Commit graph

168 commits

Author SHA1 Message Date
“attilapiros” b56e9c613f [SPARK-16630][YARN] Blacklist a node if executors won't launch on it
## What changes were proposed in this pull request?

This change extends YARN resource allocation handling with blacklisting functionality.
This handles cases when node is messed up or misconfigured such that a container won't launch on it. Before this change backlisting only focused on task execution but this change introduces YarnAllocatorBlacklistTracker which tracks allocation failures per host (when enabled via "spark.yarn.blacklist.executor.launch.blacklisting.enabled").

## How was this patch tested?

### With unit tests

Including a new suite: YarnAllocatorBlacklistTrackerSuite.

#### Manually

It was tested on a cluster by deleting the Spark jars on one of the node.

#### Behaviour before these changes

Starting Spark as:
```
spark2-shell --master yarn --deploy-mode client --num-executors 4  --conf spark.executor.memory=4g --conf "spark.yarn.max.executor.failures=6"
```

Log is:
```
18/04/12 06:49:36 INFO yarn.ApplicationMaster: Final app status: FAILED, exitCode: 11, (reason: Max number of executor failures (6) reached)
18/04/12 06:49:39 INFO yarn.ApplicationMaster: Unregistering ApplicationMaster with FAILED (diag message: Max number of executor failures (6) reached)
18/04/12 06:49:39 INFO impl.AMRMClientImpl: Waiting for application to be successfully unregistered.
18/04/12 06:49:39 INFO yarn.ApplicationMaster: Deleting staging directory hdfs://apiros-1.gce.test.com:8020/user/systest/.sparkStaging/application_1523459048274_0016
18/04/12 06:49:39 INFO util.ShutdownHookManager: Shutdown hook called
```

#### Behaviour after these changes

Starting Spark as:
```
spark2-shell --master yarn --deploy-mode client --num-executors 4  --conf spark.executor.memory=4g --conf "spark.yarn.max.executor.failures=6" --conf "spark.yarn.blacklist.executor.launch.blacklisting.enabled=true"
```

And the log is:
```
18/04/13 05:37:43 INFO yarn.YarnAllocator: Will request 1 executor container(s), each with 1 core(s) and 4505 MB memory (including 409 MB of overhead)
18/04/13 05:37:43 INFO yarn.YarnAllocator: Submitted 1 unlocalized container requests.
18/04/13 05:37:43 INFO yarn.YarnAllocator: Launching container container_1523459048274_0025_01_000008 on host apiros-4.gce.test.com for executor with ID 6
18/04/13 05:37:43 INFO yarn.YarnAllocator: Received 1 containers from YARN, launching executors on 1 of them.
18/04/13 05:37:43 INFO yarn.YarnAllocator: Completed container container_1523459048274_0025_01_000007 on host: apiros-4.gce.test.com (state: COMPLETE, exit status: 1)
18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: blacklisting host as YARN allocation failed: apiros-4.gce.test.com
18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: adding nodes to YARN application master's blacklist: List(apiros-4.gce.test.com)
18/04/13 05:37:43 WARN yarn.YarnAllocator: Container marked as failed: container_1523459048274_0025_01_000007 on host: apiros-4.gce.test.com. Exit status: 1. Diagnostics: Exception from container-launch.
Container id: container_1523459048274_0025_01_000007
Exit code: 1
Stack trace: ExitCodeException exitCode=1:
        at org.apache.hadoop.util.Shell.runCommand(Shell.java:604)
        at org.apache.hadoop.util.Shell.run(Shell.java:507)
        at org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:789)
        at org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)
        at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)
        at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
```

Where the most important part is:

```
18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: blacklisting host as YARN allocation failed: apiros-4.gce.test.com
18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: adding nodes to YARN application master's blacklist: List(apiros-4.gce.test.com)
```

And execution was continued (no shutdown called).

### Testing the backlisting of the whole cluster

Starting Spark with YARN blacklisting enabled then removing a the Spark core jar one by one from all the cluster nodes. Then executing a simple spark job which fails checking the yarn log the expected exit status is contained:

```
18/06/15 01:07:10 INFO yarn.ApplicationMaster: Final app status: FAILED, exitCode: 11, (reason: Due to executor failures all available nodes are blacklisted)
18/06/15 01:07:13 INFO util.ShutdownHookManager: Shutdown hook called
```

Author: “attilapiros” <piros.attila.zsolt@gmail.com>

Closes #21068 from attilapiros/SPARK-16630.
2018-06-21 09:17:18 -05:00
rimolive 13092d7337 [SPARK-24534][K8S] Bypass non spark-on-k8s commands
## What changes were proposed in this pull request?
This PR changes the entrypoint.sh to provide an option to run non spark-on-k8s commands (init, driver, executor) in order to let the user keep with the normal workflow without hacking the image to bypass the entrypoint

## How was this patch tested?
This patch was built manually in my local machine and I ran some tests with a combination of ```docker run``` commands.

Author: rimolive <ricardo.martinelli.oliveira@gmail.com>

Closes #21572 from rimolive/rimolive-spark-24534.
2018-06-19 13:25:00 -07:00
Jacek Laskowski 495d8cf09a [SPARK-24490][WEBUI] Use WebUI.addStaticHandler in web UIs
`WebUI` defines `addStaticHandler` that web UIs don't use (and simply introduce duplication). Let's clean them up and remove duplications.

Local build and waiting for Jenkins

Author: Jacek Laskowski <jacek@japila.pl>

Closes #21510 from jaceklaskowski/SPARK-24490-Use-WebUI.addStaticHandler.
2018-06-15 09:59:02 -07:00
mcheah 270a9a3cac [SPARK-24248][K8S] Use level triggering and state reconciliation in scheduling and lifecycle
## What changes were proposed in this pull request?

Previously, the scheduler backend was maintaining state in many places, not only for reading state but also writing to it. For example, state had to be managed in both the watch and in the executor allocator runnable. Furthermore, one had to keep track of multiple hash tables.

We can do better here by:

1. Consolidating the places where we manage state. Here, we take inspiration from traditional Kubernetes controllers. These controllers tend to follow a level-triggered mechanism. This means that the controller will continuously monitor the API server via watches and polling, and on periodic passes, the controller will reconcile the current state of the cluster with the desired state. We implement this by introducing the concept of a pod snapshot, which is a given state of the executors in the Kubernetes cluster. We operate periodically on snapshots. To prevent overloading the API server with polling requests to get the state of the cluster (particularly for executor allocation where we want to be checking frequently to get executors to launch without unbearably bad latency), we use watches to populate snapshots by applying observed events to a previous snapshot to get a new snapshot. Whenever we do poll the cluster, the polled state replaces any existing snapshot - this ensures eventual consistency and mirroring of the cluster, as is desired in a level triggered architecture.

2. Storing less specialized in-memory state in general. Previously we were creating hash tables to represent the state of executors. Instead, it's easier to represent state solely by the snapshots.

## How was this patch tested?

Integration tests should test there's no regressions end to end. Unit tests to be updated, in particular focusing on different orderings of events, particularly accounting for when events come in unexpected ordering.

Author: mcheah <mcheah@palantir.com>

Closes #21366 from mccheah/event-queue-driven-scheduling.
2018-06-14 15:56:21 -07:00
Sean Suchter f433ef7867 [SPARK-23010][K8S] Initial checkin of k8s integration tests.
These tests were developed in the https://github.com/apache-spark-on-k8s/spark-integration repo
by several contributors. This is a copy of the current state into the main apache spark repo.
The only changes from the current spark-integration repo state are:
* Move the files from the repo root into resource-managers/kubernetes/integration-tests
* Add a reference to these tests in the root README.md
* Fix a path reference in dev/dev-run-integration-tests.sh
* Add a TODO in include/util.sh

## What changes were proposed in this pull request?

Incorporation of Kubernetes integration tests.

## How was this patch tested?

This code has its own unit tests, but the main purpose is to provide the integration tests.
I tested this on my laptop by running dev/dev-run-integration-tests.sh --spark-tgz ~/spark-2.4.0-SNAPSHOT-bin--.tgz

The spark-integration tests have already been running for months in AMPLab, here is an example:
https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-scheduled-spark-integration-master/

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Sean Suchter <sean-github@suchter.com>
Author: Sean Suchter <ssuchter@pepperdata.com>

Closes #20697 from ssuchter/ssuchter-k8s-integration-tests.
2018-06-08 15:15:24 -07:00
Ilan Filonenko 1a644afbac [SPARK-23984][K8S] Initial Python Bindings for PySpark on K8s
## What changes were proposed in this pull request?

Introducing Python Bindings for PySpark.

- [x] Running PySpark Jobs
- [x] Increased Default Memory Overhead value
- [ ] Dependency Management for virtualenv/conda

## How was this patch tested?

This patch was tested with

- [x] Unit Tests
- [x] Integration tests with [this addition](https://github.com/apache-spark-on-k8s/spark-integration/pull/46)
```
KubernetesSuite:
- Run SparkPi with no resources
- Run SparkPi with a very long application name.
- Run SparkPi with a master URL without a scheme.
- Run SparkPi with an argument.
- Run SparkPi with custom labels, annotations, and environment variables.
- Run SparkPi with a test secret mounted into the driver and executor pods
- Run extraJVMOptions check on driver
- Run SparkRemoteFileTest using a remote data file
- 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 completed in 4 minutes, 28 seconds.
Total number of tests run: 11
Suites: completed 2, aborted 0
Tests: succeeded 11, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

Author: Ilan Filonenko <if56@cornell.edu>
Author: Ilan Filonenko <ifilondz@gmail.com>

Closes #21092 from ifilonenko/master.
2018-06-08 11:18:34 -07:00
Stavros Kontopoulos 22df953f6b [SPARK-24326][MESOS] add support for local:// scheme for the app jar
## What changes were proposed in this pull request?

* Adds support for local:// scheme like in k8s case for image based deployments where the jar is already in the image. Affects cluster mode and the mesos dispatcher.  Covers also file:// scheme. Keeps the default case where jar resolution happens on the host.

## How was this patch tested?

Dispatcher image with the patch, use it to start DC/OS Spark service:
skonto/spark-local-disp:test

Test image with my application jar located at the root folder:
skonto/spark-local:test

Dockerfile for that image.

From mesosphere/spark:2.3.0-2.2.1-2-hadoop-2.6
COPY spark-examples_2.11-2.2.1.jar /
WORKDIR /opt/spark/dist

Tests:

The following work as expected:

* local normal example
```
dcos spark run --submit-args="--conf spark.mesos.appJar.local.resolution.mode=container --conf spark.executor.memory=1g --conf spark.mesos.executor.docker.image=skonto/spark-local:test
 --conf spark.executor.cores=2 --conf spark.cores.max=8
 --class org.apache.spark.examples.SparkPi local:///spark-examples_2.11-2.2.1.jar"
```

* make sure the flag does not affect other uris
```
dcos spark run --submit-args="--conf spark.mesos.appJar.local.resolution.mode=container --conf spark.executor.memory=1g  --conf spark.executor.cores=2 --conf spark.cores.max=8
 --class org.apache.spark.examples.SparkPi https://s3-eu-west-1.amazonaws.com/fdp-stavros-test/spark-examples_2.11-2.1.1.jar"
```

* normal example no local
```
dcos spark run --submit-args="--conf spark.executor.memory=1g  --conf spark.executor.cores=2 --conf spark.cores.max=8
 --class org.apache.spark.examples.SparkPi https://s3-eu-west-1.amazonaws.com/fdp-stavros-test/spark-examples_2.11-2.1.1.jar"

```

The following fails

 * uses local with no setting, default is host.
```
dcos spark run --submit-args="--conf spark.executor.memory=1g --conf spark.mesos.executor.docker.image=skonto/spark-local:test
  --conf spark.executor.cores=2 --conf spark.cores.max=8
  --class org.apache.spark.examples.SparkPi local:///spark-examples_2.11-2.2.1.jar"
```
![image](https://user-images.githubusercontent.com/7945591/40283021-8d349762-5c80-11e8-9d62-2a61a4318fd5.png)

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

Closes #21378 from skonto/local-upstream.
2018-05-31 21:25:45 -07:00
hyukjinkwon 2c9c8629b7 [MINOR][YARN] Add YARN-specific credential providers in debug logging message
This PR adds a debugging log for YARN-specific credential providers which is loaded by service loader mechanism.

It took me a while to debug if it's actually loaded or not. I had to explicitly set the deprecated configuration and check if that's actually being loaded.

The change scope is manually tested. Logs are like:

```
Using the following builtin delegation token providers: hadoopfs, hive, hbase.
Using the following YARN-specific credential providers: yarn-test.
```

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21466 from HyukjinKwon/minor-log.

Change-Id: I18e2fb8eeb3289b148f24c47bb3130a560a881cf
2018-06-01 08:44:57 +08:00
Stavros Kontopoulos 21e1fc7d4a [SPARK-24232][K8S] Add support for secret env vars
## What changes were proposed in this pull request?

* Allows to refer a secret as an env var.
* Introduces new config properties in the form: spark.kubernetes{driver,executor}.secretKeyRef.ENV_NAME=name:key
  ENV_NAME is case sensitive.

* Updates docs.
* Adds required unit tests.

## How was this patch tested?
Manually tested and confirmed that the secrets exist in driver's and executor's container env.
Also job finished successfully.
First created a secret with the following yaml:
```
apiVersion: v1
kind: Secret
metadata:
  name: test-secret
data:
  username: c3RhdnJvcwo=
  password: Mzk1MjgkdmRnN0pi

-------

$ echo -n 'stavros' | base64
c3RhdnJvcw==
$ echo -n '39528$vdg7Jb' | base64
MWYyZDFlMmU2N2Rm
```
Run a job as follows:
```./bin/spark-submit \
      --master k8s://http://localhost:9000 \
      --deploy-mode cluster \
      --name spark-pi \
      --class org.apache.spark.examples.SparkPi \
      --conf spark.executor.instances=1 \
      --conf spark.kubernetes.container.image=skonto/spark:k8envs3 \
      --conf spark.kubernetes.driver.secretKeyRef.MY_USERNAME=test-secret:username \
      --conf spark.kubernetes.driver.secretKeyRef.My_password=test-secret:password \
      --conf spark.kubernetes.executor.secretKeyRef.MY_USERNAME=test-secret:username \
      --conf spark.kubernetes.executor.secretKeyRef.My_password=test-secret:password \
      local:///opt/spark/examples/jars/spark-examples_2.11-2.4.0-SNAPSHOT.jar 10000
```

Secret loaded correctly at the driver container:
![image](https://user-images.githubusercontent.com/7945591/40174346-7fee70c8-59dd-11e8-8705-995a5472716f.png)

Also if I log into the exec container:

kubectl exec -it spark-pi-1526555613156-exec-1 bash
bash-4.4# env

> SPARK_EXECUTOR_MEMORY=1g
> SPARK_EXECUTOR_CORES=1
> LANG=C.UTF-8
> HOSTNAME=spark-pi-1526555613156-exec-1
> SPARK_APPLICATION_ID=spark-application-1526555618626
> **MY_USERNAME=stavros**
>
> JAVA_HOME=/usr/lib/jvm/java-1.8-openjdk
> KUBERNETES_PORT_443_TCP_PROTO=tcp
> KUBERNETES_PORT_443_TCP_ADDR=10.100.0.1
> JAVA_VERSION=8u151
> KUBERNETES_PORT=tcp://10.100.0.1:443
> PWD=/opt/spark/work-dir
> HOME=/root
> SPARK_LOCAL_DIRS=/var/data/spark-b569b0ae-b7ef-4f91-bcd5-0f55535d3564
> KUBERNETES_SERVICE_PORT_HTTPS=443
> KUBERNETES_PORT_443_TCP_PORT=443
> SPARK_HOME=/opt/spark
> SPARK_DRIVER_URL=spark://CoarseGrainedSchedulerspark-pi-1526555613156-driver-svc.default.svc:7078
> KUBERNETES_PORT_443_TCP=tcp://10.100.0.1:443
> SPARK_EXECUTOR_POD_IP=9.0.9.77
> TERM=xterm
> SPARK_EXECUTOR_ID=1
> SHLVL=1
> KUBERNETES_SERVICE_PORT=443
> SPARK_CONF_DIR=/opt/spark/conf
> PATH=/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin:/usr/lib/jvm/java-1.8-openjdk/jre/bin:/usr/lib/jvm/java-1.8-openjdk/bin
> JAVA_ALPINE_VERSION=8.151.12-r0
> KUBERNETES_SERVICE_HOST=10.100.0.1
> **My_password=39528$vdg7Jb**
> _=/usr/bin/env
>

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

Closes #21317 from skonto/k8s-fix-env-secrets.
2018-05-31 14:28:33 -07:00
hyukjinkwon b142157dcc [SPARK-24384][PYTHON][SPARK SUBMIT] Add .py files correctly into PythonRunner in submit with client mode in spark-submit
## What changes were proposed in this pull request?

In client side before context initialization specifically,  .py file doesn't work in client side before context initialization when the application is a Python file. See below:

```
$ cat /home/spark/tmp.py
def testtest():
    return 1
```

This works:

```
$ cat app.py
import pyspark
pyspark.sql.SparkSession.builder.getOrCreate()
import tmp
print("************************%s" % tmp.testtest())

$ ./bin/spark-submit --master yarn --deploy-mode client --py-files /home/spark/tmp.py app.py
...
************************1
```

but this doesn't:

```
$ cat app.py
import pyspark
import tmp
pyspark.sql.SparkSession.builder.getOrCreate()
print("************************%s" % tmp.testtest())

$ ./bin/spark-submit --master yarn --deploy-mode client --py-files /home/spark/tmp.py app.py
Traceback (most recent call last):
  File "/home/spark/spark/app.py", line 2, in <module>
    import tmp
ImportError: No module named tmp
```

### How did it happen?

In client mode specifically, the paths are being added into PythonRunner as are:

628c7b5179/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala (L430)

628c7b5179/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala (L49-L88)

The problem here is, .py file shouldn't be added as are since `PYTHONPATH` expects a directory or an archive like zip or egg.

### How does this PR fix?

We shouldn't simply just add its parent directory because other files in the parent directory could also be added into the `PYTHONPATH` in client mode before context initialization.

Therefore, we copy .py files into a temp directory for .py files and add it to `PYTHONPATH`.

## How was this patch tested?

Unit tests are added and manually tested in both standalond and yarn client modes with submit.

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21426 from HyukjinKwon/SPARK-24384.
2018-05-30 10:33:34 -07:00
Shixiong Zhu 53c06ddabb [SPARK-24332][SS][MESOS] Fix places reading 'spark.network.timeout' as milliseconds
## What changes were proposed in this pull request?

This PR replaces `getTimeAsMs` with `getTimeAsSeconds` to fix the issue that reading "spark.network.timeout" using a wrong time unit when the user doesn't specify a time out.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #21382 from zsxwing/fix-network-timeout-conf.
2018-05-24 13:00:24 -07:00
Marco Gaido 84d31aa5d4 [SPARK-24209][SHS] Automatic retrieve proxyBase from Knox headers
## What changes were proposed in this pull request?

The PR retrieves the proxyBase automatically from the header `X-Forwarded-Context` (if available). This is the header used by Knox to inform the proxied service about the base path.

This provides 0-configuration support for Knox gateway (instead of having to properly set `spark.ui.proxyBase`) and it allows to access directly SHS when it is proxied by Knox. In the previous scenario, indeed, after setting `spark.ui.proxyBase`, direct access to SHS was not working fine (due to bad link generated).

## How was this patch tested?

added UT + manual tests

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21268 from mgaido91/SPARK-24209.
2018-05-21 18:11:05 -07:00
Marco Gaido 3159ee085b [SPARK-24149][YARN] Retrieve all federated namespaces tokens
## What changes were proposed in this pull request?

Hadoop 3 introduces HDFS federation. This means that multiple namespaces are allowed on the same HDFS cluster. In Spark, we need to ask the delegation token for all the namenodes (for each namespace), otherwise accessing any other namespace different from the default one (for which we already fetch the delegation token) fails.

The PR adds the automatic discovery of all the namenodes related to all the namespaces available according to the configs in hdfs-site.xml.

## How was this patch tested?

manual tests in dockerized env

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21216 from mgaido91/SPARK-24149.
2018-05-18 13:04:00 -07:00
Marcelo Vanzin 54032682b9 [SPARK-24182][YARN] Improve error message when client AM fails.
Instead of always throwing a generic exception when the AM fails,
print a generic error and throw the exception with the YARN
diagnostics containing the reason for the failure.

There was an issue with YARN sometimes providing a generic diagnostic
message, even though the AM provides a failure reason when
unregistering. That was happening because the AM was registering
too late, and if errors happened before the registration, YARN would
just create a generic "ExitCodeException" which wasn't very helpful.

Since most errors in this path are a result of not being able to
connect to the driver, this change modifies the AM registration
a bit so that the AM is registered before the connection to the
driver is established. That way, errors are properly propagated
through YARN back to the driver.

As part of that, I also removed the code that retried connections
to the driver from the client AM. At that point, the driver should
already be up and waiting for connections, so it's unlikely that
retrying would help - and in case it does, that means a flaky
network, which would mean problems would probably show up again.
The effect of that is that connection-related errors are reported
back to the driver much faster now (through the YARN report).

One thing to note is that there seems to be a race on the YARN
side that causes a report to be sent to the client without the
corresponding diagnostics string from the AM; the diagnostics are
available later from the RM web page. For that reason, the generic
error messages are kept in the Spark scheduler code, to help
guide users to a way of debugging their failure.

Also of note is that if YARN's max attempts configuration is lower
than Spark's, Spark will not unregister the AM with a proper
diagnostics message. Unfortunately there seems to be no way to
unregister the AM and still allow further re-attempts to happen.

Testing:
- existing unit tests
- some of our integration tests
- hardcoded an invalid driver address in the code and verified
  the error in the shell. e.g.

```
scala> 18/05/04 15:09:34 ERROR cluster.YarnClientSchedulerBackend: YARN application has exited unexpectedly with state FAILED! Check the YARN application logs for more details.
18/05/04 15:09:34 ERROR cluster.YarnClientSchedulerBackend: Diagnostics message: Uncaught exception: org.apache.spark.SparkException: Exception thrown in awaitResult:
  <AM stack trace>
Caused by: java.io.IOException: Failed to connect to localhost/127.0.0.1:1234
  <More stack trace>
```

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21243 from vanzin/SPARK-24182.
2018-05-11 17:40:35 +08:00
mcheah 6282fc64e3 [SPARK-24137][K8S] Mount local directories as empty dir volumes.
## What changes were proposed in this pull request?

Drastically improves performance and won't cause Spark applications to fail because they write too much data to the Docker image's specific file system. The file system's directories that back emptydir volumes are generally larger and more performant.

## How was this patch tested?

Has been in use via the prototype version of Kubernetes support, but lost in the transition to here.

Author: mcheah <mcheah@palantir.com>

Closes #21238 from mccheah/mount-local-dirs.
2018-05-10 11:36:41 -07:00
Marcelo Vanzin cc613b552e [PYSPARK] Update py4j to version 0.10.7. 2018-05-09 10:47:35 -07:00
Bounkong Khamphousone 6782359a04 [SPARK-23941][MESOS] Mesos task failed on specific spark app name
## What changes were proposed in this pull request?
Shell escaped the name passed to spark-submit and change how conf attributes are shell escaped.

## How was this patch tested?
This test has been tested manually with Hive-on-spark with mesos or with the use case described in the issue with the sparkPi application with a custom name which contains illegal shell characters.

With this PR, hive-on-spark on mesos works like a charm with hive 3.0.0-SNAPSHOT.

I state that this contribution is my original work and that I license the work to the project under the project’s open source license

Author: Bounkong Khamphousone <bounkong.khamphousone@ebiznext.com>

Closes #21014 from tiboun/fix/SPARK-23941.
2018-05-01 08:28:21 -07:00
Devaraj K 007ae6878f [SPARK-24003][CORE] Add support to provide spark.executor.extraJavaOptions in terms of App Id and/or Executor Id's
## What changes were proposed in this pull request?

Added support to specify the 'spark.executor.extraJavaOptions' value in terms of the `{{APP_ID}}` and/or `{{EXECUTOR_ID}}`,  `{{APP_ID}}` will be replaced by Application Id and `{{EXECUTOR_ID}}` will be replaced by Executor Id while starting the executor.

## How was this patch tested?

I have verified this by checking the executor process command and gc logs. I verified the same in different deployment modes(Standalone, YARN, Mesos) client and cluster modes.

Author: Devaraj K <devaraj@apache.org>

Closes #21088 from devaraj-kavali/SPARK-24003.
2018-04-30 13:40:03 -07:00
eric-maynard 109935fc5d [SPARK-23830][YARN] added check to ensure main method is found
## What changes were proposed in this pull request?

When a user specifies the wrong class -- or, in fact, a class instead of an object -- Spark throws an NPE which is not useful for debugging. This was reported in [SPARK-23830](https://issues.apache.org/jira/browse/SPARK-23830). This PR adds a check to ensure the main method was found and logs a useful error in the event that it's null.

## How was this patch tested?

* Unit tests + Manual testing
* The scope of the changes is very limited

Author: eric-maynard <emaynard@cloudera.com>
Author: Eric Maynard <emaynard@cloudera.com>

Closes #21168 from eric-maynard/feature/SPARK-23830.
2018-04-27 15:25:07 +08:00
zhoukang 9ee9fcf522 [SPARK-24083][YARN] Log stacktrace for uncaught exception
## What changes were proposed in this pull request?

Log stacktrace for uncaught exception

## How was this patch tested?
UT and manually test

Author: zhoukang <zhoukang199191@gmail.com>

Closes #21151 from caneGuy/zhoukang/log-stacktrace.
2018-04-26 15:38:11 -07:00
Gera Shegalov c0964935d6 [SPARK-23956][YARN] Use effective RPC port in AM registration
## What changes were proposed in this pull request?

We propose not to hard-code the RPC port in the AM registration.

## How was this patch tested?

Tested application reports from a pseudo-distributed cluster
```
18/04/10 14:56:21 INFO Client:
client token: N/A
diagnostics: N/A
ApplicationMaster host: localhost
ApplicationMaster RPC port: 58338
queue: default
start time: 1523397373659
final status: UNDEFINED
tracking URL: http://localhost:8088/proxy/application_1523370127531_0016/
```

Author: Gera Shegalov <gera@apache.org>

Closes #21047 from gerashegalov/gera/am-to-rm-nmhost.
2018-04-16 12:01:42 +08:00
mcheah a83ae0d9bc [SPARK-22839][K8S] Refactor to unify driver and executor pod builder APIs
## What changes were proposed in this pull request?

Breaks down the construction of driver pods and executor pods in a way that uses a common abstraction for both spark-submit creating the driver and KubernetesClusterSchedulerBackend creating the executor. Encourages more code reuse and is more legible than the older approach.

The high-level design is discussed in more detail on the JIRA ticket. This pull request is the implementation of that design with some minor changes in the implementation details.

No user-facing behavior should break as a result of this change.

## How was this patch tested?

Migrated all unit tests from the old submission steps architecture to the new architecture. Integration tests should not have to change and pass given that this shouldn't change any outward behavior.

Author: mcheah <mcheah@palantir.com>

Closes #20910 from mccheah/spark-22839-incremental.
2018-04-13 08:43:58 -07:00
Marcelo Vanzin 3cb82047f2 [SPARK-22941][CORE] Do not exit JVM when submit fails with in-process launcher.
The current in-process launcher implementation just calls the SparkSubmit
object, which, in case of errors, will more often than not exit the JVM.
This is not desirable since this launcher is meant to be used inside other
applications, and that would kill the application.

The change turns SparkSubmit into a class, and abstracts aways some of
the functionality used to print error messages and abort the submission
process. The default implementation uses the logging system for messages,
and throws exceptions for errors. As part of that I also moved some code
that doesn't really belong in SparkSubmit to a better location.

The command line invocation of spark-submit now uses a special implementation
of the SparkSubmit class that overrides those behaviors to do what is expected
from the command line version (print to the terminal, exit the JVM, etc).

A lot of the changes are to replace calls to methods such as "printErrorAndExit"
with the new API.

As part of adding tests for this, I had to fix some small things in the
launcher option parser so that things like "--version" can work when
used in the launcher library.

There is still code that prints directly to the terminal, like all the
Ivy-related code in SparkSubmitUtils, and other areas where some re-factoring
would help, like the CommandLineUtils class, but I chose to leave those
alone to keep this change more focused.

Aside from existing and added unit tests, I ran command line tools with
a bunch of different arguments to make sure messages and errors behave
like before.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20925 from vanzin/SPARK-22941.
2018-04-11 10:13:44 -05:00
jinxing d3bd0435ee [SPARK-23637][YARN] Yarn might allocate more resource if a same executor is killed multiple times.
## What changes were proposed in this pull request?
`YarnAllocator` uses `numExecutorsRunning` to track the number of running executor. `numExecutorsRunning` is used to check if there're executors missing and need to allocate more.

 In current code, `numExecutorsRunning` can be negative when driver asks to kill a same idle executor multiple times.

## How was this patch tested?
UT added

Author: jinxing <jinxing6042@126.com>

Closes #20781 from jinxing64/SPARK-23637.
2018-04-04 15:51:27 -07:00
Andrew Korzhuev cccaaa14ad [SPARK-23668][K8S] Add config option for passing through k8s Pod.spec.imagePullSecrets
## What changes were proposed in this pull request?

Pass through the `imagePullSecrets` option to the k8s pod in order to allow user to access private image registries.

See https://kubernetes.io/docs/tasks/configure-pod-container/pull-image-private-registry/

## How was this patch tested?

Unit tests + manual testing.

Manual testing procedure:
1. Have private image registry.
2. Spark-submit application with no `spark.kubernetes.imagePullSecret` set. Do `kubectl describe pod ...`. See the error message:
```
Error syncing pod, skipping: failed to "StartContainer" for "spark-kubernetes-driver" with ErrImagePull: "rpc error: code = 2 desc = Error: Status 400 trying to pull repository ...: \"{\\n  \\\"errors\\\" : [ {\\n    \\\"status\\\" : 400,\\n    \\\"message\\\" : \\\"Unsupported docker v1 repository request for '...'\\\"\\n  } ]\\n}\""
```
3. Create secret `kubectl create secret docker-registry ...`
4. Spark-submit with `spark.kubernetes.imagePullSecret` set to the new secret. See that deployment was successful.

Author: Andrew Korzhuev <andrew.korzhuev@klarna.com>
Author: Andrew Korzhuev <korzhuev@andrusha.me>

Closes #20811 from andrusha/spark-23668-image-pull-secrets.
2018-04-04 12:30:52 -07:00
Yinan Li fe2b7a4568 [SPARK-23285][K8S] Add a config property for specifying physical executor cores
## What changes were proposed in this pull request?

As mentioned in SPARK-23285, this PR introduces a new configuration property `spark.kubernetes.executor.cores` for specifying the physical CPU cores requested for each executor pod. This is to avoid changing the semantics of `spark.executor.cores` and `spark.task.cpus` and their role in task scheduling, task parallelism, dynamic resource allocation, etc. The new configuration property only determines the physical CPU cores available to an executor. An executor can still run multiple tasks simultaneously by using appropriate values for `spark.executor.cores` and `spark.task.cpus`.

## How was this patch tested?

Unit tests.

felixcheung srowen jiangxb1987 jerryshao mccheah foxish

Author: Yinan Li <ynli@google.com>
Author: Yinan Li <liyinan926@gmail.com>

Closes #20553 from liyinan926/master.
2018-04-02 12:20:55 -07:00
David Vogelbacher 6151f29f9f [SPARK-23825][K8S] Requesting memory + memory overhead for pod memory
## What changes were proposed in this pull request?

Kubernetes driver and executor pods should request `memory + memoryOverhead` as their resources instead of just `memory`, see https://issues.apache.org/jira/browse/SPARK-23825

## How was this patch tested?
Existing unit tests were adapted.

Author: David Vogelbacher <dvogelbacher@palantir.com>

Closes #20943 from dvogelbacher/spark-23825.
2018-04-02 12:00:37 -07:00
Marcelo Vanzin 5fa4384711 [SPARK-23361][YARN] Allow AM to restart after initial tokens expire.
Currently, the Spark AM relies on the initial set of tokens created by
the submission client to be able to talk to HDFS and other services that
require delegation tokens. This means that after those tokens expire, a
new AM will fail to start (e.g. when there is an application failure and
re-attempts are enabled).

This PR makes it so that the first thing the AM does when the user provides
a principal and keytab is to create new delegation tokens for use. This
makes sure that the AM can be started irrespective of how old the original
token set is. It also allows all of the token management to be done by the
AM - there is no need for the submission client to set configuration values
to tell the AM when to renew tokens.

Note that even though in this case the AM will not be using the delegation
tokens created by the submission client, those tokens still need to be provided
to YARN, since they are used to do log aggregation.

To be able to re-use the code in the AMCredentialRenewal for the above
purposes, I refactored that class a bit so that it can fetch tokens into
a pre-defined UGI, insted of always logging in.

Another issue with re-attempts is that, after the fix that allows the AM
to restart correctly, new executors would get confused about when to
update credentials, because the credential updater used the update time
initially set up by the submission code. This could make the executor
fail to update credentials in time, since that value would be very out
of date in the situation described in the bug.

To fix that, I changed the YARN code to use the new RPC-based mechanism
for distributing tokens to executors. This allowed the old credential
updater code to be removed, and a lot of code in the renewer to be
simplified.

I also made two currently hardcoded values (the renewal time ratio, and
the retry wait) configurable; while this probably never needs to be set
by anyone in a production environment, it helps with testing; that's also
why they're not documented.

Tested on real cluster with a specially crafted application to test this
functionality: checked proper access to HDFS, Hive and HBase in cluster
mode with token renewal on and AM restarts. Tested things still work in
client mode too.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20657 from vanzin/SPARK-23361.
2018-03-23 13:59:21 +08:00
Gabor Somogyi 5f4deff195 [SPARK-23660] Fix exception in yarn cluster mode when application ended fast
## What changes were proposed in this pull request?

Yarn throws the following exception in cluster mode when the application is really small:

```
18/03/07 23:34:22 WARN netty.NettyRpcEnv: Ignored failure: java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask7c974942 rejected from java.util.concurrent.ScheduledThreadPoolExecutor1eea9d2d[Terminated, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 0]
18/03/07 23:34:22 ERROR yarn.ApplicationMaster: Uncaught exception:
org.apache.spark.SparkException: Exception thrown in awaitResult:
	at org.apache.spark.util.ThreadUtils$.awaitResult(ThreadUtils.scala:205)
	at org.apache.spark.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:75)
	at org.apache.spark.rpc.RpcEndpointRef.askSync(RpcEndpointRef.scala:92)
	at org.apache.spark.rpc.RpcEndpointRef.askSync(RpcEndpointRef.scala:76)
	at org.apache.spark.deploy.yarn.YarnAllocator.<init>(YarnAllocator.scala:102)
	at org.apache.spark.deploy.yarn.YarnRMClient.register(YarnRMClient.scala:77)
	at org.apache.spark.deploy.yarn.ApplicationMaster.registerAM(ApplicationMaster.scala:450)
	at org.apache.spark.deploy.yarn.ApplicationMaster.runDriver(ApplicationMaster.scala:493)
	at org.apache.spark.deploy.yarn.ApplicationMaster.org$apache$spark$deploy$yarn$ApplicationMaster$$runImpl(ApplicationMaster.scala:345)
	at org.apache.spark.deploy.yarn.ApplicationMaster$$anonfun$run$2.apply$mcV$sp(ApplicationMaster.scala:260)
	at org.apache.spark.deploy.yarn.ApplicationMaster$$anonfun$run$2.apply(ApplicationMaster.scala:260)
	at org.apache.spark.deploy.yarn.ApplicationMaster$$anonfun$run$2.apply(ApplicationMaster.scala:260)
	at org.apache.spark.deploy.yarn.ApplicationMaster$$anon$5.run(ApplicationMaster.scala:810)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:422)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1920)
	at org.apache.spark.deploy.yarn.ApplicationMaster.doAsUser(ApplicationMaster.scala:809)
	at org.apache.spark.deploy.yarn.ApplicationMaster.run(ApplicationMaster.scala:259)
	at org.apache.spark.deploy.yarn.ApplicationMaster$.main(ApplicationMaster.scala:834)
	at org.apache.spark.deploy.yarn.ApplicationMaster.main(ApplicationMaster.scala)
Caused by: org.apache.spark.rpc.RpcEnvStoppedException: RpcEnv already stopped.
	at org.apache.spark.rpc.netty.Dispatcher.postMessage(Dispatcher.scala:158)
	at org.apache.spark.rpc.netty.Dispatcher.postLocalMessage(Dispatcher.scala:135)
	at org.apache.spark.rpc.netty.NettyRpcEnv.ask(NettyRpcEnv.scala:229)
	at org.apache.spark.rpc.netty.NettyRpcEndpointRef.ask(NettyRpcEnv.scala:523)
	at org.apache.spark.rpc.RpcEndpointRef.askSync(RpcEndpointRef.scala:91)
	... 17 more
18/03/07 23:34:22 INFO yarn.ApplicationMaster: Final app status: FAILED, exitCode: 13, (reason: Uncaught exception: org.apache.spark.SparkException: Exception thrown in awaitResult: )
```

Example application:

```
object ExampleApp {
  def main(args: Array[String]): Unit = {
    val conf = new SparkConf().setAppName("ExampleApp")
    val sc = new SparkContext(conf)
    try {
      // Do nothing
    } finally {
      sc.stop()
    }
  }
```

This PR pauses user class thread after `SparkContext` created and keeps it so until application master initialises properly.

## How was this patch tested?

Automated: Existing unit tests
Manual: Application submitted into small cluster

Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>

Closes #20807 from gaborgsomogyi/SPARK-23660.
2018-03-19 18:02:04 -07:00
Ilan Filonenko f15906da15 [SPARK-22839][K8S] Remove the use of init-container for downloading remote dependencies
## What changes were proposed in this pull request?

Removal of the init-container for downloading remote dependencies. Built off of the work done by vanzin in an attempt to refactor driver/executor configuration elaborated in [this](https://issues.apache.org/jira/browse/SPARK-22839) ticket.

## How was this patch tested?

This patch was tested with unit and integration tests.

Author: Ilan Filonenko <if56@cornell.edu>

Closes #20669 from ifilonenko/remove-init-container.
2018-03-19 11:29:56 -07:00
Ricardo Martinelli de Oliveira 9945b0227e [SPARK-23680] Fix entrypoint.sh to properly support Arbitrary UIDs
## What changes were proposed in this pull request?

As described in SPARK-23680, entrypoint.sh returns an error code because of a command pipeline execution where it is expected in case of Openshift environments, where arbitrary UIDs are used to run containers

## How was this patch tested?

This patch was manually tested by using docker-image-toll.sh script to generate a Spark driver image and running an example against an OpenShift cluster.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Ricardo Martinelli de Oliveira <rmartine@rmartine.gru.redhat.com>

Closes #20822 from rimolive/rmartine-spark-23680.
2018-03-16 10:37:11 -07:00
jerryshao c952000487 [SPARK-23635][YARN] AM env variable should not overwrite same name env variable set through spark.executorEnv.
## What changes were proposed in this pull request?

In the current Spark on YARN code, AM always will copy and overwrite its env variables to executors, so we cannot set different values for executors.

To reproduce issue, user could start spark-shell like:

```
./bin/spark-shell --master yarn-client --conf spark.executorEnv.SPARK_ABC=executor_val --conf  spark.yarn.appMasterEnv.SPARK_ABC=am_val
```

Then check executor env variables by

```
sc.parallelize(1 to 1).flatMap \{ i => sys.env.toSeq }.collect.foreach(println)
```

We will always get `am_val` instead of `executor_val`. So we should not let AM to overwrite specifically set executor env variables.

## How was this patch tested?

Added UT and tested in local cluster.

Author: jerryshao <sshao@hortonworks.com>

Closes #20799 from jerryshao/SPARK-23635.
2018-03-16 16:22:03 +08:00
Marcelo Vanzin 2c3673680e [SPARK-23630][YARN] Allow user's hadoop conf customizations to take effect.
This change restores functionality that was inadvertently removed as part
of the fix for SPARK-22372.

Also modified an existing unit test to make sure the feature works as intended.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20776 from vanzin/SPARK-23630.
2018-03-09 10:36:38 -08:00
Yuming Wang 04e71c3160 [MINOR][YARN] Add disable yarn.nodemanager.vmem-check-enabled option to memLimitExceededLogMessage
My spark application sometimes will throw `Container killed by YARN for exceeding memory limits`.
Even I increased `spark.yarn.executor.memoryOverhead` to 10G, this error still happen.  The latest config:
<img width="685" alt="memory-config" src="https://user-images.githubusercontent.com/5399861/36975716-f5c548d2-20b5-11e8-95e5-b228d50917b9.png">

And error message:
```
ExecutorLostFailure (executor 121 exited caused by one of the running tasks) Reason: Container killed by YARN for exceeding memory limits. 30.7 GB of 30 GB physical memory used. Consider boosting spark.yarn.executor.memoryOverhead.
```

This is because of [Linux glibc >= 2.10 (RHEL 6) malloc may show excessive virtual memory usage](https://www.ibm.com/developerworks/community/blogs/kevgrig/entry/linux_glibc_2_10_rhel_6_malloc_may_show_excessive_virtual_memory_usage?lang=en). So disable `yarn.nodemanager.vmem-check-enabled` looks like a good option as [MapR mentioned ](https://mapr.com/blog/best-practices-yarn-resource-management).

This PR add disable `yarn.nodemanager.vmem-check-enabled` option to memLimitExceededLogMessage.

More details:
https://issues.apache.org/jira/browse/YARN-4714
https://stackoverflow.com/a/31450291
https://stackoverflow.com/a/42091255

After this PR:
<img width="898" alt="yarn" src="https://user-images.githubusercontent.com/5399861/36975949-c8e7bbbe-20b6-11e8-9513-9f903b868d8d.png">

N/A

Author: Yuming Wang <yumwang@ebay.com>
Author: Yuming Wang <wgyumg@gmail.com>

Closes #20735 from wangyum/YARN-4714.

Change-Id: Ie10836e2c07b6384d228c3f9e89f802823bd9f16
2018-03-07 17:01:29 +08:00
Andrew Korzhuev 185f5bc7dd [SPARK-23449][K8S] Preserve extraJavaOptions ordering
For some JVM options, like `-XX:+UnlockExperimentalVMOptions` ordering is necessary.

## What changes were proposed in this pull request?

Keep original `extraJavaOptions` ordering, when passing them through environment variables inside the Docker container.

## How was this patch tested?

Ran base branch a couple of times and checked startup command in logs. Ordering differed every time. Added sorting, ordering was consistent to what user had in `extraJavaOptions`.

Author: Andrew Korzhuev <korzhuev@andrusha.me>

Closes #20628 from andrusha/patch-2.
2018-02-26 10:28:45 -08:00
Rob Vesse 7f10cf83f3 [SPARK-16501][MESOS] Allow providing Mesos principal & secret via files
This commit modifies the Mesos submission client to allow the principal
and secret to be provided indirectly via files.  The path to these files
can be specified either via Spark configuration or via environment
variable.

Assuming these files are appropriately protected by FS/OS permissions
this means we don't ever leak the actual values in process info like ps

Environment variable specification is useful because it allows you to
interpolate the location of this file when using per-user Mesos
credentials.

For some background as to why we have taken this approach I will briefly describe our set up.  On our systems we provide each authorised user account with their own Mesos credentials to provide certain security and audit guarantees to our customers. These credentials are managed by a central Secret management service. In our `spark-env.sh` we determine the appropriate secret and principal files to use depending on the user who is invoking Spark hence the need to inject these via environment variables as well as by configuration properties. So we set these environment variables appropriately and our Spark read in the contents of those files to authenticate itself with Mesos.

This is functionality we have been using it in production across multiple customer sites for some time. This has been in the field for around 18 months with no reported issues. These changes have been sufficient to meet our customer security and audit requirements.

We have been building and deploying custom builds of Apache Spark with various minor tweaks like this which we are now looking to contribute back into the community in order that we can rely upon stock Apache Spark builds and stop maintaining our own internal fork.

Author: Rob Vesse <rvesse@dotnetrdf.org>

Closes #20167 from rvesse/SPARK-16501.
2018-02-09 11:23:06 -08:00
Gera Shegalov 032c11b83f [SPARK-23296][YARN] Include stacktrace in YARN-app diagnostic
## What changes were proposed in this pull request?

Include stacktrace in the diagnostics message upon abnormal unregister from RM

## How was this patch tested?
Tested with a failing job, and confirmed a stacktrace in the client output and YARN webUI.

Author: Gera Shegalov <gera@apache.org>

Closes #20470 from gerashegalov/gera/stacktrace-diagnostics.
2018-02-01 15:26:59 -08:00
Marcelo Vanzin 70a68b328b [SPARK-23020][CORE] Fix race in SparkAppHandle cleanup, again.
Third time is the charm?

There was still a race that was left in previous attempts. If the handle
closes the connection, the close() implementation would clean up state
that would prevent the thread from waiting on the connection thread to
finish. That could cause the race causing the test flakiness reported
in the bug.

The fix is to move the "wait for connection thread" code to a separate
close method that is used by the handle; that also simplifies the code
a bit and makes it also easier to follow.

I included an unrelated, but correct, change to a YARN test so that
it triggers when the PR is built.

Tested by inserting a sleep in the connection thread to mimic the race;
test failed reliably with the sleep, passes now. (Sleep not included in
the patch.) Also ran YARN tests to make sure.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20388 from vanzin/SPARK-23020.
2018-01-26 11:58:20 +08:00
Marcelo Vanzin ec22897615 [SPARK-23020][CORE] Fix races in launcher code, test.
The race in the code is because the handle might update
its state to the wrong state if the connection handling
thread is still processing incoming data; so the handle
needs to wait for the connection to finish up before
checking the final state.

The race in the test is because when waiting for a handle
to reach a final state, the waitFor() method needs to wait
until all handle state is updated (which also includes
waiting for the connection thread above to finish).
Otherwise, waitFor() may return too early, which would cause
a bunch of different races (like the listener not being yet
notified of the state change, or being in the middle of
being notified, or the handle not being properly disposed
and causing postChecks() to assert).

On top of that I found, by code inspection, a couple of
potential races that could make a handle end up in the
wrong state when being killed.

The original version of this fix introduced the flipped
version of the first race described above; the connection
closing might override the handle state before the
handle might have a chance to do cleanup. The fix there
is to only dispose of the handle from the connection
when there is an error, and let the handle dispose
itself in the normal case.

The fix also caused a bug in YarnClusterSuite to be surfaced;
the code was checking for a file in the classpath that was
not expected to be there in client mode. Because of the above
issues, the error was not propagating correctly and the (buggy)
test was incorrectly passing.

Tested by running the existing unit tests a lot (and not
seeing the errors I was seeing before).

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20297 from vanzin/SPARK-23020.
2018-01-22 14:49:12 +08:00
Yinan Li 5d7c4ba4d7 [SPARK-22962][K8S] Fail fast if submission client local files are used
## What changes were proposed in this pull request?

In the Kubernetes mode, fails fast in the submission process if any submission client local dependencies are used as the use case is not supported yet.

## How was this patch tested?

Unit tests, integration tests, and manual tests.

vanzin foxish

Author: Yinan Li <liyinan926@gmail.com>

Closes #20320 from liyinan926/master.
2018-01-18 14:44:22 -08:00
Andrew Korzhuev f568e9cf76 [SPARK-23133][K8S] Fix passing java options to Executor
Pass through spark java options to the executor in context of docker image.
Closes #20296

andrusha: Deployed two version of containers to local k8s, checked that java options were present in the updated image on the running executor.
Manual test

Author: Andrew Korzhuev <korzhuev@andrusha.me>

Closes #20322 from foxish/patch-1.
2018-01-18 14:00:43 -08:00
gatorsmile 651f76153f [SPARK-23028] Bump master branch version to 2.4.0-SNAPSHOT
## What changes were proposed in this pull request?
This patch bumps the master branch version to `2.4.0-SNAPSHOT`.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20222 from gatorsmile/bump24.
2018-01-13 00:37:59 +08:00
Marcelo Vanzin 0b2eefb674 [SPARK-22994][K8S] Use a single image for all Spark containers.
This change allows a user to submit a Spark application on kubernetes
having to provide a single image, instead of one image for each type
of container. The image's entry point now takes an extra argument that
identifies the process that is being started.

The configuration still allows the user to provide different images
for each container type if they so desire.

On top of that, the entry point was simplified a bit to share more
code; mainly, the same env variable is used to propagate the user-defined
classpath to the different containers.

Aside from being modified to match the new behavior, the
'build-push-docker-images.sh' script was renamed to 'docker-image-tool.sh'
to more closely match its purpose; the old name was a little awkward
and now also not entirely correct, since there is a single image. It
was also moved to 'bin' since it's not necessarily an admin tool.

Docs have been updated to match the new behavior.

Tested locally with minikube.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20192 from vanzin/SPARK-22994.
2018-01-11 10:37:35 -08:00
Mingjie Tang a6647ffbf7 [SPARK-22587] Spark job fails if fs.defaultFS and application jar are different url
## What changes were proposed in this pull request?

Two filesystems comparing does not consider the authority of URI. This is specific for
WASB file storage system, where userInfo is honored to differentiate filesystems.
For example: wasbs://user1xyz.net, wasbs://user2xyz.net would consider as two filesystem.
Therefore, we have to add the authority to compare two filesystem, and  two filesystem with different authority can not be the same FS.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Mingjie Tang <mtang@hortonworks.com>

Closes #19885 from merlintang/EAR-7377.
2018-01-11 11:51:03 +08:00
Yinan Li 6a4206ff04 [SPARK-22998][K8S] Set missing value for SPARK_MOUNTED_CLASSPATH in the executors
## What changes were proposed in this pull request?

The environment variable `SPARK_MOUNTED_CLASSPATH` is referenced in the executor's Dockerfile, where its value is added to the classpath of the executor. However, the scheduler backend code missed setting it when creating the executor pods. This PR fixes it.

## How was this patch tested?

Unit tested.

vanzin Can you help take a look? Thanks!
foxish

Author: Yinan Li <liyinan926@gmail.com>

Closes #20193 from liyinan926/master.
2018-01-09 01:32:48 -08:00
foxish eed82a0b21 [SPARK-22992][K8S] Remove assumption of the DNS domain
## What changes were proposed in this pull request?

Remove the use of FQDN to access the driver because it assumes that it's set up in a DNS zone - `cluster.local` which is common but not ubiquitous
Note that we already access the in-cluster API server through `kubernetes.default.svc`, so, by extension, this should work as well.
The alternative is to introduce DNS zones for both of those addresses.

## How was this patch tested?
Unit tests

cc vanzin liyinan926 mridulm mccheah

Author: foxish <ramanathana@google.com>

Closes #20187 from foxish/cluster.local.
2018-01-08 13:01:45 -08:00
Yinan Li bf65cd3cda [SPARK-22960][K8S] Revert use of ARG base_image in images
## What changes were proposed in this pull request?

This PR reverts the `ARG base_image` before `FROM` in the images of driver, executor, and init-container, introduced in https://github.com/apache/spark/pull/20154. The reason is Docker versions before 17.06 do not support this use (`ARG` before `FROM`).

## How was this patch tested?

Tested manually.

vanzin foxish kimoonkim

Author: Yinan Li <liyinan926@gmail.com>

Closes #20170 from liyinan926/master.
2018-01-05 17:29:27 -08:00
Gera Shegalov ea95683301 [SPARK-22914][DEPLOY] Register history.ui.port
## What changes were proposed in this pull request?

Register spark.history.ui.port as a known spark conf to be used in substitution expressions even if it's not set explicitly.

## How was this patch tested?

Added unit test to demonstrate the issue

Author: Gera Shegalov <gera@apache.org>
Author: Gera Shegalov <gshegalov@salesforce.com>

Closes #20098 from gerashegalov/gera/register-SHS-port-conf.
2018-01-05 17:25:28 -08:00
Marcelo Vanzin 0428368c2c [SPARK-22960][K8S] Make build-push-docker-images.sh more dev-friendly.
- Make it possible to build images from a git clone.
- Make it easy to use minikube to test things.

Also fixed what seemed like a bug: the base image wasn't getting the tag
provided in the command line. Adding the tag allows users to use multiple
Spark builds in the same kubernetes cluster.

Tested by deploying images on minikube and running spark-submit from a dev
environment; also by building the images with different tags and verifying
"docker images" in minikube.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20154 from vanzin/SPARK-22960.
2018-01-04 16:34:56 -08:00
Yinan Li e288fc87a0 [SPARK-22953][K8S] Avoids adding duplicated secret volumes when init-container is used
## What changes were proposed in this pull request?

User-specified secrets are mounted into both the main container and init-container (when it is used) in a Spark driver/executor pod, using the `MountSecretsBootstrap`. Because `MountSecretsBootstrap` always adds new secret volumes for the secrets to the pod, the same secret volumes get added twice, one when mounting the secrets to the main container, and the other when mounting the secrets to the init-container. This PR fixes the issue by separating `MountSecretsBootstrap.mountSecrets` out into two methods: `addSecretVolumes` for adding secret volumes to a pod and `mountSecrets` for mounting secret volumes to a container, respectively. `addSecretVolumes` is only called once for each pod, whereas `mountSecrets` is called individually for the main container and the init-container (if it is used).

Ref: https://github.com/apache-spark-on-k8s/spark/issues/594.

## How was this patch tested?
Unit tested and manually tested.

vanzin This replaces https://github.com/apache/spark/pull/20148.
hex108 foxish kimoonkim

Author: Yinan Li <liyinan926@gmail.com>

Closes #20159 from liyinan926/master.
2018-01-04 15:35:20 -08:00