Commit graph

332 commits

Author SHA1 Message Date
Stavros Kontopoulos 1fb3759f2b [SPARK-25930][K8S] Fix scala string detection in k8s tests
## What changes were proposed in this pull request?

- Issue is described in detail in [SPARK-25930](https://issues.apache.org/jira/browse/SPARK-25930). Since we rely on the std output, pick always the last line which contains the wanted value. Although minor, current implementation breaks tests.

## How was this patch tested?
manually. rm -rf ~/.m2 and then run the tests.

Closes #22931 from skonto/fix_scala_detection.

Authored-by: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-05 08:40:25 -06:00
Marcelo Vanzin 3404a73f4c [SPARK-25875][K8S] Merge code to set up driver command into a single step.
Right now there are 3 different classes dealing with building the driver
command to run inside the pod, one for each "binding" supported by Spark.
This has two main shortcomings:

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

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

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

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

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

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

Tested with new and updated unit tests + integration tests.

Author: Marcelo Vanzin <vanzin@cloudera.com>

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

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

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

To Do:

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

## How was this patch tested?

Ran integration tests with Docker for Desktop and all passed:

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

Suggested Reviewers: ifilonenko srowen

Author: Rob Vesse <rvesse@dotnetrdf.org>

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

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

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

Closes #22624 from vanzin/SPARK-23781.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2018-10-31 13:00:10 -05:00
Dongjoon Hyun e4cb42ad89
[SPARK-25891][PYTHON] Upgrade to Py4J 0.10.8.1
## What changes were proposed in this pull request?

Py4J 0.10.8.1 is released on October 21st and is the first release of Py4J to support Python 3.7 officially. We had better have this to get the official support. Also, there are some patches related to garbage collections.

https://www.py4j.org/changelog.html#py4j-0-10-8-and-py4j-0-10-8-1

## How was this patch tested?

Pass the Jenkins.

Closes #22901 from dongjoon-hyun/SPARK-25891.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-31 09:55:03 -07:00
Onur Satici f6cc354d83 [SPARK-24434][K8S] pod template files
## What changes were proposed in this pull request?

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

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

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

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

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

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

## How was this patch tested?

Unit and Integration tests

Closes #22820 from ifilonenko/SPARK-25828.

Authored-by: Ilan Filonenko <ifilondz@gmail.com>
Signed-off-by: Erik Erlandson <eerlands@redhat.com>
2018-10-26 15:59:12 -07:00
Stavros Kontopoulos 7d44bc2640 [SPARK-25835][K8S] Create kubernetes-tests profile and use the detected SCALA_VERSION
## What changes were proposed in this pull request?

- Fixes the scala version propagation issue.
- Disables the tests under the k8s profile, now we will run them manually. Adds a test specific profile otherwise tests will not run if we just remove the module from the kubernetes profile (quickest solution I can think of).
## How was this patch tested?
Manually by running the tests with different versions of scala.

Closes #22838 from skonto/propagate-scala2.12.

Authored-by: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-26 08:49:27 -05:00
Ilan Filonenko 19ada15d1b [SPARK-24516][K8S] Change Python default to Python3
## What changes were proposed in this pull request?

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

## How was this patch tested?

Unit and Integration tests

Author: Ilan Filonenko <ifilondz@gmail.com>

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

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

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

## How was this patch tested?

Running on my local cluster.

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

Closes #22720 from mikekap/patch-1.
2018-10-21 11:32:33 -07:00
Marcelo Vanzin 15524c41b2 [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 <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-10-18 10:21:37 -07:00
Marcelo Vanzin 7d425b190a [SPARK-20327][YARN] Follow up: fix resource request tests on Hadoop 3.
The test fix is to allocate a `Resource` object only after the resource
types have been initialized. Otherwise the YARN classes get in a weird
state and throw a different exception than expected, because the resource
has a different view of the registered resources.

I also removed a test for a null resource since that seems unnecessary
and made the fix more complicated.

All the other changes are just cleanup; basically simplify the tests by
defining what is being tested and deriving the resource type registration
and the SparkConf from that data, instead of having redundant definitions
in the tests.

Ran tests with Hadoop 3 (and also without it).

Closes #22751 from vanzin/SPARK-20327.fix.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2018-10-17 10:40:47 -05:00
Ilan Filonenko 6c9c84ffb9 [SPARK-23257][K8S] Kerberos Support for Spark on K8S
## What changes were proposed in this pull request?
This is the work on setting up Secure HDFS interaction with Spark-on-K8S.
The architecture is discussed in this community-wide google [doc](https://docs.google.com/document/d/1RBnXD9jMDjGonOdKJ2bA1lN4AAV_1RwpU_ewFuCNWKg)
This initiative can be broken down into 4 Stages

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

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

**STAGE 3**
- [x] Driver

**STAGE 4**
- [x] Executor

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

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

## Contribution Credit
kimoonkim skonto

Closes #21669 from ifilonenko/secure-hdfs.

Lead-authored-by: Ilan Filonenko <if56@cornell.edu>
Co-authored-by: Ilan Filonenko <ifilondz@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-10-15 15:48:51 -07:00
Szilard Nemeth 3946de7734 [SPARK-20327][CORE][YARN] Add CLI support for YARN custom resources, like GPUs
## What changes were proposed in this pull request?

This PR adds CLI support for YARN custom resources, e.g. GPUs and any other resources YARN defines.
The custom resources are defined with Spark properties, no additional CLI arguments were introduced.

The properties can be defined in the following form:

**AM resources, client mode:**
Format: `spark.yarn.am.resource.<resource-name>`
The property name follows the naming convention of YARN AM cores / memory properties: `spark.yarn.am.memory and spark.yarn.am.cores
`

**Driver resources, cluster mode:**
Format: `spark.yarn.driver.resource.<resource-name>`
The property name follows the naming convention of driver cores / memory properties: `spark.driver.memory and spark.driver.cores.`

**Executor resources:**
Format: `spark.yarn.executor.resource.<resource-name>`
The property name follows the naming convention of executor cores / memory properties: `spark.executor.memory / spark.executor.cores`.

For the driver resources (cluster mode) and executor resources properties, we use the `yarn` prefix here as custom resource types are specific to YARN, currently.

**Validation:**
Please note that a validation logic is added to avoid having requested resources defined in 2 ways, for example defining the following configs:
```
"--conf", "spark.driver.memory=2G",
"--conf", "spark.yarn.driver.resource.memory=1G"
```

will not start execution and will print an error message.

## How was this patch tested?
Unit tests + manual execution with Hadoop2 and Hadoop 3 builds.

Testing have been performed on a real cluster with Spark and YARN configured:
Cluster and client mode
Request Resource Types with lowercase and uppercase units
Start Spark job with only requesting standard resources (mem / cpu)
Error handling cases:
- Request unknown resource type
- Request Resource type (either memory / cpu) with duplicate configs at the same time (e.g. with this config:
```
--conf spark.yarn.am.resource.memory=1G \
  --conf spark.yarn.driver.resource.memory=2G \
  --conf spark.yarn.executor.resource.memory=3G \
```
), ResourceTypeValidator handles these cases well, so it is not permitted
- Request standard resource (memory / cpu) with the new style configs, e.g. --conf spark.yarn.am.resource.memory=1G,  this is not permitted and handled well.

An example about how I ran the testcases:
```
cd ~;export HADOOP_CONF_DIR=/opt/hadoop/etc/hadoop/;
./spark-2.4.0-SNAPSHOT-bin-custom-spark/bin/spark-submit \
  --class org.apache.spark.examples.SparkPi \
  --master yarn \
  --deploy-mode cluster \
  --driver-memory 1G \
  --driver-cores 1 \
  --executor-memory 1G \
  --executor-cores 1 \
  --conf spark.logConf=true \
  --conf spark.yarn.executor.resource.gpu=3G \
  --verbose \
  ./spark-2.4.0-SNAPSHOT-bin-custom-spark/examples/jars/spark-examples_2.11-2.4.0-SNAPSHOT.jar \
  10;
```

Closes #20761 from szyszy/SPARK-20327.

Authored-by: Szilard Nemeth <snemeth@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-10-12 18:14:13 -07:00
Sean Owen 80813e1980 [SPARK-25016][BUILD][CORE] Remove support for Hadoop 2.6
## What changes were proposed in this pull request?

Remove Hadoop 2.6 references and make 2.7 the default.
Obviously, this is for master/3.0.0 only.
After this we can also get rid of the separate test jobs for Hadoop 2.6.

## How was this patch tested?

Existing tests

Closes #22615 from srowen/SPARK-25016.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-10 12:07:53 -07:00
Marcelo Vanzin 58287a3986
[SPARK-25646][K8S] Fix docker-image-tool.sh on dev build.
The docker file was referencing a path that only existed in the
distribution tarball; it needs to be parameterized so that the
right path can be used in a dev build.

Tested on local dev build.

Closes #22634 from vanzin/SPARK-25646.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-05 21:15:16 -07:00
gatorsmile 9bf397c0e4 [SPARK-25592] Setting version to 3.0.0-SNAPSHOT
## What changes were proposed in this pull request?

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

## How was this patch tested?
N/A

Closes #22606 from gatorsmile/bump3.0.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-02 08:48:24 -07:00
marek.simunek a802c69b13 [SPARK-18364][YARN] Expose metrics for YarnShuffleService
## What changes were proposed in this pull request?

This PR is follow-up of closed  https://github.com/apache/spark/pull/17401 which only ended due to of inactivity, but its still nice feature to have.
Given review by jerryshao taken in consideration and edited:
- VisibleForTesting deleted because of dependency conflicts
- removed unnecessary reflection for `MetricsSystemImpl`
- added more available types for gauge

## How was this patch tested?

Manual deploy of new yarn-shuffle jar into a Node Manager and verifying that the metrics appear in the Node Manager-standard location. This is JMX with an query endpoint running on `hostname:port`

Resulting metrics look like this:
```
curl -sk -XGET hostname:port |  grep -v '#' | grep 'shuffleService'
hadoop_nodemanager_openblockrequestlatencymillis_rate15{name="shuffleService",} 0.31428910657834713
hadoop_nodemanager_blocktransferratebytes_rate15{name="shuffleService",} 566144.9983653595
hadoop_nodemanager_blocktransferratebytes_ratemean{name="shuffleService",} 2464409.9678099006
hadoop_nodemanager_openblockrequestlatencymillis_rate1{name="shuffleService",} 1.2893844732240272
hadoop_nodemanager_registeredexecutorssize{name="shuffleService",} 2.0
hadoop_nodemanager_openblockrequestlatencymillis_ratemean{name="shuffleService",} 1.255574678369966
hadoop_nodemanager_openblockrequestlatencymillis_count{name="shuffleService",} 315.0
hadoop_nodemanager_openblockrequestlatencymillis_rate5{name="shuffleService",} 0.7661929192569739
hadoop_nodemanager_registerexecutorrequestlatencymillis_ratemean{name="shuffleService",} 0.0
hadoop_nodemanager_registerexecutorrequestlatencymillis_count{name="shuffleService",} 0.0
hadoop_nodemanager_registerexecutorrequestlatencymillis_rate1{name="shuffleService",} 0.0
hadoop_nodemanager_registerexecutorrequestlatencymillis_rate5{name="shuffleService",} 0.0
hadoop_nodemanager_blocktransferratebytes_count{name="shuffleService",} 6.18271213E8
hadoop_nodemanager_registerexecutorrequestlatencymillis_rate15{name="shuffleService",} 0.0
hadoop_nodemanager_blocktransferratebytes_rate5{name="shuffleService",} 1154114.4881816586
hadoop_nodemanager_blocktransferratebytes_rate1{name="shuffleService",} 574745.0749848988
```

Closes #22485 from mareksimunek/SPARK-18364.

Lead-authored-by: marek.simunek <marek.simunek@firma.seznam.cz>
Co-authored-by: Andrew Ash <andrew@andrewash.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2018-10-01 11:04:37 -05:00
Prashant Sharma 4da541a5d2
[SPARK-25543][K8S] Print debug message iff execIdsRemovedInThisRound is not empty.
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

Tested by manually deploying to a minikube cluster.

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

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

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

It produces an error as below:

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

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

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

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

## How was this patch tested?

Manually tested, and Jenkins tests.

Closes #22581 from HyukjinKwon/SPARK-25565.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-30 14:31:04 +08:00
Mukul Murthy 9362c5cc27
[SPARK-25449][CORE] Heartbeat shouldn't include accumulators for zero metrics
## What changes were proposed in this pull request?

Heartbeat shouldn't include accumulators for zero metrics.

Heartbeats sent from executors to the driver every 10 seconds contain metrics and are generally on the order of a few KBs. However, for large jobs with lots of tasks, heartbeats can be on the order of tens of MBs, causing tasks to die with heartbeat failures. We can mitigate this by not sending zero metrics to the driver.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

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

Closes #22473 from mukulmurthy/25449-heartbeat.

Authored-by: Mukul Murthy <mukul.murthy@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2018-09-28 16:34:17 -07:00
Ilan Filonenko 123f0041d5 [SPARK-25291][K8S] Fixing Flakiness of Executor Pod tests
## What changes were proposed in this pull request?

Added fix to flakiness that was present in PySpark tests w.r.t Executors not being tested.

Important fix to executorConf which was failing tests when executors *were* tested

## How was this patch tested?

Unit and Integration tests

Closes #22415 from ifilonenko/SPARK-25291.

Authored-by: Ilan Filonenko <if56@cornell.edu>
Signed-off-by: Yinan Li <ynli@google.com>
2018-09-18 11:43:35 -07:00
gatorsmile bb2f069cf2 [SPARK-25436] Bump master branch version to 2.5.0-SNAPSHOT
## What changes were proposed in this pull request?
In the dev list, we can still discuss whether the next version is 2.5.0 or 3.0.0. Let us first bump the master branch version to `2.5.0-SNAPSHOT`.

## How was this patch tested?
N/A

Closes #22426 from gatorsmile/bumpVersionMaster.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-15 16:24:02 -07:00
Kazuaki Ishizaki f60cd7cc3c
[SPARK-25338][TEST] Ensure to call super.beforeAll() and super.afterAll() in test cases
## What changes were proposed in this pull request?

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.

## How was this patch tested?

Existing UTs

Closes #22337 from kiszk/SPARK-25338.

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

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

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

## How was this patch tested?

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

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

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

Authored-by: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>
Signed-off-by: Yinan Li <ynli@google.com>
2018-09-12 22:02:59 -07:00
Sean Owen cfbdd6a1f5 [SPARK-25398] Minor bugs from comparing unrelated types
## What changes were proposed in this pull request?

Correct some comparisons between unrelated types to what they seem to… have been trying to do

## How was this patch tested?

Existing tests.

Closes #22384 from srowen/SPARK-25398.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-11 14:46:03 -05:00
Ilan Filonenko 1cfda44825 [SPARK-25021][K8S] Add spark.executor.pyspark.memory limit for K8S
## What changes were proposed in this pull request?

Add spark.executor.pyspark.memory limit for K8S

## How was this patch tested?

Unit and Integration tests

Closes #22298 from ifilonenko/SPARK-25021.

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

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

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

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

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

## How was this patch tested?

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

Author: Rob Vesse <rvesse@dotnetrdf.org>

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

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

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

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

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

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

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

## How was this patch tested?

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

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

Suggested reviewers: liyinan926 mccheah

Author: Rob Vesse <rvesse@dotnetrdf.org>

Closes #22215 from rvesse/SPARK-25222.
2018-09-06 16:15:11 -07:00
liuxian ca861fea21 [SPARK-25300][CORE] Unified the configuration parameter spark.shuffle.service.enabled
## What changes were proposed in this pull request?

The configuration parameter "spark.shuffle.service.enabled"  has defined in `package.scala`,  and it  is also used in many place,  so we can replace it with `SHUFFLE_SERVICE_ENABLED`.
and unified  this configuration parameter "spark.shuffle.service.port"  together.

## How was this patch tested?
N/A

Closes #22306 from 10110346/unifiedserviceenable.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-05 10:43:46 +08:00
Ilan Filonenko e1d72f2c07 [SPARK-25264][K8S] Fix comma-delineated arguments passed into PythonRunner and RRunner
## What changes were proposed in this pull request?

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

## How was this patch tested?

Modified unit test to test this change.

Author: Ilan Filonenko <if56@cornell.edu>

Closes #22257 from ifilonenko/SPARK-25264.
2018-08-31 15:46:45 -07:00
Erik Erlandson d6d1224ffa [SPARK-25275][K8S] require memberhip in wheel to run 'su' in dockerfiles
## What changes were proposed in this pull request?
Add a PAM configuration in k8s dockerfile to require authentication into wheel to run as `su`

## How was this patch tested?
Verify against CI that PAM config succeeds & causes no regressions

Closes #22285 from erikerlandson/spark-25275.

Authored-by: Erik Erlandson <eerlands@redhat.com>
Signed-off-by: Erik Erlandson <eerlands@redhat.com>
2018-08-30 14:07:04 -07:00
Ryan Blue 7ad18ee9f2 [SPARK-25004][CORE] Add spark.executor.pyspark.memory limit.
## What changes were proposed in this pull request?

This adds `spark.executor.pyspark.memory` to configure Python's address space limit, [`resource.RLIMIT_AS`](https://docs.python.org/3/library/resource.html#resource.RLIMIT_AS). Limiting Python's address space allows Python to participate in memory management. In practice, we see fewer cases of Python taking too much memory because it doesn't know to run garbage collection. This results in YARN killing fewer containers. This also improves error messages so users know that Python is consuming too much memory:

```
  File "build/bdist.linux-x86_64/egg/package/library.py", line 265, in fe_engineer
    fe_eval_rec.update(f(src_rec_prep, mat_rec_prep))
  File "build/bdist.linux-x86_64/egg/package/library.py", line 163, in fe_comp
    comparisons = EvaluationUtils.leven_list_compare(src_rec_prep.get(item, []), mat_rec_prep.get(item, []))
  File "build/bdist.linux-x86_64/egg/package/evaluationutils.py", line 25, in leven_list_compare
    permutations = sorted(permutations, reverse=True)
  MemoryError
```

The new pyspark memory setting is used to increase requested YARN container memory, instead of sharing overhead memory between python and off-heap JVM activity.

## How was this patch tested?

Tested memory limits in our YARN cluster and verified that MemoryError is thrown.

Author: Ryan Blue <blue@apache.org>

Closes #21977 from rdblue/SPARK-25004-add-python-memory-limit.
2018-08-28 12:31:33 -07:00
jerryshao 4e3f3cebe4 [SPARK-23679][YARN] Setting RM_HA_URLS for AmIpFilter to avoid redirect failure in YARN mode
## What changes were proposed in this pull request?

YARN `AmIpFilter` adds a new parameter "RM_HA_URLS" to support RM HA, but Spark on YARN doesn't provide a such parameter, so it will be failed to redirect when running on RM HA. The detailed exception can be checked from JIRA. So here fixing this issue by adding "RM_HA_URLS" parameter.

## How was this patch tested?

Local verification.

Closes #22164 from jerryshao/SPARK-23679.

Authored-by: jerryshao <sshao@hortonworks.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-08-28 10:33:39 -07:00
Yuming Wang c3f285c939 [SPARK-24149][YARN][FOLLOW-UP] Only get the delegation tokens of the filesystem explicitly specified by the user
## What changes were proposed in this pull request?

Our HDFS cluster configured 5 nameservices: `nameservices1`, `nameservices2`, `nameservices3`, `nameservices-dev1` and `nameservices4`, but `nameservices-dev1` unstable. So sometimes an error occurred and causing the entire job failed since [SPARK-24149](https://issues.apache.org/jira/browse/SPARK-24149):

![image](https://user-images.githubusercontent.com/5399861/42434779-f10c48fc-8386-11e8-98b0-4d9786014744.png)

I think it's best to add a switch here.

## How was this patch tested?

manual tests

Closes #21734 from wangyum/SPARK-24149.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-08-27 13:26:55 -07:00
Kent Yao f8346d2fc0 [SPARK-25174][YARN] Limit the size of diagnostic message for am to unregister itself from rm
## What changes were proposed in this pull request?

When using older versions of spark releases,  a use case generated a huge code-gen file which hit the limitation `Constant pool has grown past JVM limit of 0xFFFF`.  In this situation, it should fail immediately. But the diagnosis message sent to RM is too large,  the ApplicationMaster suspended and RM's ZKStateStore was crashed. For 2.3 or later spark releases the limitation of code-gen has been removed, but maybe there are still some uncaught exceptions that contain oversized error message will cause such a problem.

This PR is aim to cut down the diagnosis message size.

## How was this patch tested?

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

Closes #22180 from yaooqinn/SPARK-25174.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-08-24 13:44:19 -07:00
s71955 c20916a5dc [SPARK-25073][YARN] AM and Executor Memory validation message is not proper while submitting spark yarn application
**## What changes were proposed in this pull request?**
When the yarn.nodemanager.resource.memory-mb or yarn.scheduler.maximum-allocation-mb
 memory assignment is insufficient, Spark always reports an error request to adjust
yarn.scheduler.maximum-allocation-mb even though in message it shows the memory value
of yarn.nodemanager.resource.memory-mb parameter,As the error Message is bit misleading to the user  we can modify the same, We can keep the error message same as executor memory validation message.

Defintion of **yarn.nodemanager.resource.memory-mb:**
Amount of physical memory, in MB, that can be allocated for containers. It means the amount of memory YARN can utilize on this node and therefore this property should be lower then the total memory of that machine.
**yarn.scheduler.maximum-allocation-mb:**
It defines the maximum memory allocation available for a container in MB
it means RM can only allocate memory to containers in increments of "yarn.scheduler.minimum-allocation-mb" and not exceed "yarn.scheduler.maximum-allocation-mb" and It should not be more than total allocated memory of the Node.

**## How was this patch tested?**
Manually tested in hdfs-Yarn clustaer

Closes #22199 from sujith71955/maste_am_log.

Authored-by: s71955 <sujithchacko.2010@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-08-24 08:58:19 -05:00
Ilan Filonenko ba84bcb2c4 [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s
## What changes were proposed in this pull request?

Introducing R Bindings for Spark R on K8s

- [x] Running SparkR Job

## How was this patch tested?

This patch was tested with

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

## Example:

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

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

Author: Ilan Filonenko <if56@cornell.edu>

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

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

## How was this patch tested?

Tested with unit and integration tests

Author: Ilan Filonenko <if56@cornell.edu>

Closes #22095 from ifilonenko/spark-py-edits.
2018-08-15 17:52:12 -07:00
Xingbo Jiang bfb74394a5 [SPARK-24819][CORE] Fail fast when no enough slots to launch the barrier stage on job submitted
## What changes were proposed in this pull request?

We shall check whether the barrier stage requires more slots (to be able to launch all tasks in the barrier stage together) than the total number of active slots currently, and fail fast if trying to submit a barrier stage that requires more slots than current total number.

This PR proposes to add a new method `getNumSlots()` to try to get the total number of currently active slots in `SchedulerBackend`, support of this new method has been added to all the first-class scheduler backends except `MesosFineGrainedSchedulerBackend`.

## How was this patch tested?

Added new test cases in `BarrierStageOnSubmittedSuite`.

Closes #22001 from jiangxb1987/SPARK-24819.

Lead-authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Co-authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2018-08-15 13:31:28 -07:00
Imran Rashid 1024875843 [SPARK-25088][CORE][MESOS][DOCS] Update Rest Server docs & defaults.
## What changes were proposed in this pull request?

(a) disabled rest submission server by default in standalone mode
(b) fails the standalone master if rest server enabled & authentication secret set
(c) fails the mesos cluster dispatcher if authentication secret set
(d) doc updates
(e) when submitting a standalone app, only try the rest submission first if spark.master.rest.enabled=true

otherwise you'd see a 10 second pause like
18/08/09 08:13:22 INFO RestSubmissionClient: Submitting a request to launch an application in spark://...
18/08/09 08:13:33 WARN RestSubmissionClient: Unable to connect to server spark://...

I also made sure the mesos cluster dispatcher failed with the secret enabled, though I had to do that on slightly different code as I don't have mesos native libs around.

## How was this patch tested?

I ran the tests in the mesos module & in core for org.apache.spark.deploy.*

I ran a test on a cluster with standalone master to make sure I could still start with the right configs, and would fail the right way too.

Closes #22071 from squito/rest_doc_updates.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-08-14 13:02:33 -05:00
Kazuhiro Sera 8ec25cd67e Fix typos detected by github.com/client9/misspell
## What changes were proposed in this pull request?

Fixing typos is sometimes very hard. It's not so easy to visually review them. Recently, I discovered a very useful tool for it, [misspell](https://github.com/client9/misspell).

This pull request fixes minor typos detected by [misspell](https://github.com/client9/misspell) except for the false positives. If you would like me to work on other files as well, let me know.

## How was this patch tested?

### before

```
$ misspell . | grep -v '.js'
R/pkg/R/SQLContext.R:354:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:424:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:445:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:495:43: "definiton" is a misspelling of "definition"
NOTICE-binary:454:16: "containd" is a misspelling of "contained"
R/pkg/R/context.R:46:43: "definiton" is a misspelling of "definition"
R/pkg/R/context.R:74:43: "definiton" is a misspelling of "definition"
R/pkg/R/DataFrame.R:591:48: "persistance" is a misspelling of "persistence"
R/pkg/R/streaming.R:166:44: "occured" is a misspelling of "occurred"
R/pkg/inst/worker/worker.R:65:22: "ouput" is a misspelling of "output"
R/pkg/tests/fulltests/test_utils.R:106:25: "environemnt" is a misspelling of "environment"
common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java:38:39: "existant" is a misspelling of "existent"
common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java:83:39: "existant" is a misspelling of "existent"
common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java:243:46: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:234:19: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:238:63: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:244:46: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:276:39: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java:27:20: "transfered" is a misspelling of "transferred"
common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala:195:15: "orgin" is a misspelling of "origin"
core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala:621:39: "gauranteed" is a misspelling of "guaranteed"
core/src/main/scala/org/apache/spark/status/storeTypes.scala:113:29: "ect" is a misspelling of "etc"
core/src/main/scala/org/apache/spark/storage/DiskStore.scala:282:18: "transfered" is a misspelling of "transferred"
core/src/main/scala/org/apache/spark/util/ListenerBus.scala:64:17: "overriden" is a misspelling of "overridden"
core/src/test/scala/org/apache/spark/ShuffleSuite.scala:211:7: "substracted" is a misspelling of "subtracted"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:1922:49: "agriculteur" is a misspelling of "agriculture"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:2468:84: "truely" is a misspelling of "truly"
core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala:25:18: "persistance" is a misspelling of "persistence"
core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala:26:69: "persistance" is a misspelling of "persistence"
data/streaming/AFINN-111.txt:1219:0: "humerous" is a misspelling of "humorous"
dev/run-pip-tests:55:28: "enviroments" is a misspelling of "environments"
dev/run-pip-tests:91:37: "virutal" is a misspelling of "virtual"
dev/merge_spark_pr.py:377:72: "accross" is a misspelling of "across"
dev/merge_spark_pr.py:378:66: "accross" is a misspelling of "across"
dev/run-pip-tests:126:25: "enviroments" is a misspelling of "environments"
docs/configuration.md:1830:82: "overriden" is a misspelling of "overridden"
docs/structured-streaming-programming-guide.md:525:45: "processs" is a misspelling of "processes"
docs/structured-streaming-programming-guide.md:1165:61: "BETWEN" is a misspelling of "BETWEEN"
docs/sql-programming-guide.md:1891:810: "behaivor" is a misspelling of "behavior"
examples/src/main/python/sql/arrow.py:98:8: "substract" is a misspelling of "subtract"
examples/src/main/python/sql/arrow.py:103:27: "substract" is a misspelling of "subtract"
licenses/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt:170:0: "teh" is a misspelling of "the"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt:53:0: "eles" is a misspelling of "eels"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:99:20: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:539:11: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala:77:36: "Teh" is a misspelling of "The"
mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala:230:24: "inital" is a misspelling of "initial"
mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala:276:9: "Euclidian" is a misspelling of "Euclidean"
mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala:237:26: "descripiton" is a misspelling of "descriptions"
python/pyspark/find_spark_home.py:30:13: "enviroment" is a misspelling of "environment"
python/pyspark/context.py:937:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:938:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:939:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:940:12: "supress" is a misspelling of "suppress"
python/pyspark/heapq3.py:6:63: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:7:2: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:263:29: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:263:39: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:270:49: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:270:59: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:275:2: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:275:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:277:29: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:277:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:713:8: "probabilty" is a misspelling of "probability"
python/pyspark/ml/clustering.py:1038:8: "Currenlty" is a misspelling of "Currently"
python/pyspark/ml/stat.py:339:23: "Euclidian" is a misspelling of "Euclidean"
python/pyspark/ml/regression.py:1378:20: "paramter" is a misspelling of "parameter"
python/pyspark/mllib/stat/_statistics.py:262:8: "probabilty" is a misspelling of "probability"
python/pyspark/rdd.py:1363:32: "paramter" is a misspelling of "parameter"
python/pyspark/streaming/tests.py:825:42: "retuns" is a misspelling of "returns"
python/pyspark/sql/tests.py:768:29: "initalization" is a misspelling of "initialization"
python/pyspark/sql/tests.py:3616:31: "initalize" is a misspelling of "initialize"
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala:120:39: "arbitary" is a misspelling of "arbitrary"
resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala:26:45: "sucessfully" is a misspelling of "successfully"
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala:358:27: "constaints" is a misspelling of "constraints"
resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala:111:24: "senstive" is a misspelling of "sensitive"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:1063:5: "overwirte" is a misspelling of "overwrite"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala:1348:17: "compatability" is a misspelling of "compatibility"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:77:36: "paramter" is a misspelling of "parameter"
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:1374:22: "precendence" is a misspelling of "precedence"
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:238:27: "unnecassary" is a misspelling of "unnecessary"
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala:212:17: "whn" is a misspelling of "when"
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala:147:60: "timestmap" is a misspelling of "timestamp"
sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala:150:45: "precentage" is a misspelling of "percentage"
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala:135:29: "infered" is a misspelling of "inferred"
sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922:1:52: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182:1:52: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e:1:63: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478:1:63: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8:9:79: "occurence" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8:13:110: "occurence" is a misspelling of "occurrence"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q:46:105: "distint" is a misspelling of "distinct"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q:29:3: "Currenly" is a misspelling of "Currently"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q:72:15: "existant" is a misspelling of "existent"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q:25:3: "substraction" is a misspelling of "subtraction"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q:16:51: "funtion" is a misspelling of "function"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q:15:30: "issueing" is a misspelling of "issuing"
sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala:669:52: "wiht" is a misspelling of "with"
sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java:474:9: "Refering" is a misspelling of "Referring"
```

### after

```
$ misspell . | grep -v '.js'
common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java:27:20: "transfered" is a misspelling of "transferred"
core/src/main/scala/org/apache/spark/status/storeTypes.scala:113:29: "ect" is a misspelling of "etc"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:1922:49: "agriculteur" is a misspelling of "agriculture"
data/streaming/AFINN-111.txt:1219:0: "humerous" is a misspelling of "humorous"
licenses/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt:170:0: "teh" is a misspelling of "the"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt:53:0: "eles" is a misspelling of "eels"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:99:20: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:539:11: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala:77:36: "Teh" is a misspelling of "The"
mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala:276:9: "Euclidian" is a misspelling of "Euclidean"
python/pyspark/heapq3.py:6:63: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:7:2: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:263:29: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:263:39: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:270:49: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:270:59: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:275:2: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:275:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:277:29: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:277:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/ml/stat.py:339:23: "Euclidian" is a misspelling of "Euclidean"
```

Closes #22070 from seratch/fix-typo.

Authored-by: Kazuhiro Sera <seratch@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2018-08-11 21:23:36 -05:00
Kazuaki Ishizaki 132bcceebb [SPARK-25036][SQL] Avoid discarding unmoored doc comment in Scala-2.12.
## What changes were proposed in this pull request?

This PR avoid the following compilation error using sbt in Scala-2.12.

```
[error] [warn] /home/ishizaki/Spark/PR/scala212/spark/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala:410: discarding unmoored doc comment
[error] [warn]     /**
[error] [warn]
[error] [warn] /home/ishizaki/Spark/PR/scala212/spark/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala:441: discarding unmoored doc comment
[error] [warn]     /**
[error] [warn]
...
[error] [warn] /home/ishizaki/Spark/PR/scala212/spark/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala:440: discarding unmoored doc comment
[error] [warn]     /**
[error] [warn]
```

## How was this patch tested?

Existing UTs

Closes #22059 from kiszk/SPARK-25036d.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2018-08-10 07:32:52 -05:00
Adelbert Chang f5113ea8d7 [SPARK-24960][K8S] explicitly expose ports on driver container
https://issues.apache.org/jira/browse/SPARK-24960
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

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

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

Closes #21884 from adelbertc/k8s-expose-driver-ports.
2018-08-01 13:57:33 -07:00
mcheah 2fbe294cf0 [SPARK-24963][K8S][TESTS] Add user-specified service account name for client mode test driver pod
## What changes were proposed in this pull request?

Adds the user-set service account name for the driver pod in the client mode integration test

## How was this patch tested?

Manual test against a custom Kubernetes cluster

Author: mcheah <mcheah@palantir.com>

Closes #21924 from mccheah/fix-service-account.
2018-07-30 15:57:54 -07:00
mcheah d6b7545b5f [SPARK-24963][K8S][TESTS] Don't set service account name for client mode test
## What changes were proposed in this pull request?

Don't set service account name for the pod created in client mode

## How was this patch tested?

Test should continue running smoothly in Jenkins.

Author: mcheah <mcheah@palantir.com>

Closes #21900 from mccheah/fix-integration-test-service-account.
2018-07-30 11:41:02 -07:00
Xingbo Jiang e3486e1b95 [SPARK-24795][CORE] Implement barrier execution mode
## What changes were proposed in this pull request?

Propose new APIs and modify job/task scheduling to support barrier execution mode, which requires all tasks in a same barrier stage start at the same time, and retry all tasks in case some tasks fail in the middle. The barrier execution mode is useful for some ML/DL workloads.

The proposed API changes include:

- `RDDBarrier` that marks an RDD as barrier (Spark must launch all the tasks together for the current stage).
- `BarrierTaskContext` that support global sync of all tasks in a barrier stage, and provide extra `BarrierTaskInfo`s.

In DAGScheduler, we retry all tasks of a barrier stage in case some tasks fail in the middle, this is achieved by unregistering map outputs for a shuffleId (for ShuffleMapStage) or clear the finished partitions in an active job (for ResultStage).

## How was this patch tested?

Add `RDDBarrierSuite` to ensure we convert RDDs correctly;
Add new test cases in `DAGSchedulerSuite` to ensure we do task scheduling correctly;
Add new test cases in `SparkContextSuite` to ensure the barrier execution mode actually works (both under local mode and local cluster mode).
Add new test cases in `TaskSchedulerImplSuite` to ensure we schedule tasks for barrier taskSet together.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #21758 from jiangxb1987/barrier-execution-mode.
2018-07-26 12:09:01 -07:00
mcheah 0c83f718ee [SPARK-23146][K8S][TESTS] Enable client mode integration test.
## What changes were proposed in this pull request?

Enable client mode integration test after merging from master.

## How was this patch tested?

Check the integration test runs in the build.

Author: mcheah <mcheah@palantir.com>

Closes #21874 from mccheah/enable-client-mode-test.
2018-07-25 12:10:23 -07:00
mcheah 571a6f0574 [SPARK-23146][K8S] Support client mode.
## What changes were proposed in this pull request?

Support client mode for the Kubernetes scheduler.

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

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

We also change the authentication configuration prefixes for client mode.

## How was this patch tested?

Adding an integration test to exercise client mode support.

Author: mcheah <mcheah@palantir.com>

Closes #21748 from mccheah/k8s-client-mode.
2018-07-25 11:08:41 -07:00
“attilapiros” d2436a8529 [SPARK-24594][YARN] Introducing metrics for YARN
## What changes were proposed in this pull request?

In this PR metrics are introduced for YARN.  As up to now there was no metrics in the YARN module a new metric system is created with the name "applicationMaster".
To support both client and cluster mode the metric system lifecycle is bound to the AM.

## How was this patch tested?

Both client and cluster mode was tested manually.
Before the test on one of the YARN node spark-core was removed to cause the allocation failure.
Spark was started as (in case of client mode):

```
spark2-submit \
  --class org.apache.spark.examples.SparkPi \
  --conf "spark.yarn.blacklist.executor.launch.blacklisting.enabled=true" --conf "spark.blacklist.application.maxFailedExecutorsPerNode=2" --conf "spark.dynamicAllocation.enabled=true" --conf "spark.metrics.conf.*.sink.console.class=org.apache.spark.metrics.sink.ConsoleSink" \
  --master yarn \
  --deploy-mode client \
  original-spark-examples_2.11-2.4.0-SNAPSHOT.jar \
  1000
```

In both cases the YARN logs contained the new metrics as:

```
$ yarn logs --applicationId application_1529926424933_0015
...
-- Gauges ----------------------------------------------------------------------
application_1531751594108_0046.applicationMaster.numContainersPendingAllocate
             value = 0
application_1531751594108_0046.applicationMaster.numExecutorsFailed
             value = 3
application_1531751594108_0046.applicationMaster.numExecutorsRunning
             value = 9
application_1531751594108_0046.applicationMaster.numLocalityAwareTasks
             value = 0
application_1531751594108_0046.applicationMaster.numReleasedContainers
             value = 0
...

```

Author: “attilapiros” <piros.attila.zsolt@gmail.com>
Author: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com>

Closes #21635 from attilapiros/SPARK-24594.
2018-07-24 09:33:10 +08:00
Yuming Wang d7ae4247ea [SPARK-24873][YARN] Turn off spark-shell noisy log output
## What changes were proposed in this pull request?

[SPARK-24182](https://github.com/apache/spark/pull/21243) changed the `logApplicationReport` from `false` to `true`. This pr revert it to `false`. otherwise `spark-shell` will show noisy log output:
```java
...
18/07/16 04:46:25 INFO Client: Application report for application_1530676576026_54551 (state: RUNNING)
18/07/16 04:46:26 INFO Client: Application report for application_1530676576026_54551 (state: RUNNING)
...
```

Closes https://github.com/apache/spark/pull/21827

## How was this patch tested?

 manual tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21784 from wangyum/SPARK-24182.
2018-07-21 16:43:10 +08:00
zsxwing f765bb7823 [SPARK-24880][BUILD] Fix the group id for spark-kubernetes-integration-tests
## What changes were proposed in this pull request?

The correct group id should be `org.apache.spark`. This is causing the nightly build failure: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-maven-snapshots/2295/console

`
[ERROR] Failed to execute goal org.apache.maven.plugins:maven-deploy-plugin:2.8.2:deploy (default-deploy) on project spark-kubernetes-integration-tests_2.11: Failed to deploy artifacts: Could not transfer artifact spark-kubernetes-integration-tests:spark-kubernetes-integration-tests_2.11🫙2.4.0-20180720.101629-1 from/to apache.snapshots.https (https://repository.apache.org/content/repositories/snapshots): Access denied to: https://repository.apache.org/content/repositories/snapshots/spark-kubernetes-integration-tests/spark-kubernetes-integration-tests_2.11/2.4.0-SNAPSHOT/spark-kubernetes-integration-tests_2.11-2.4.0-20180720.101629-1.jar, ReasonPhrase: Forbidden. -> [Help 1]
[ERROR]
`

## How was this patch tested?

Jenkins.

Author: zsxwing <zsxwing@gmail.com>

Closes #21831 from zsxwing/fix-k8s-test.
2018-07-20 15:23:04 -07:00
Stavros Kontopoulos 20ce1a8f8b [SPARK-24551][K8S] Add integration tests for secrets
## What changes were proposed in this pull request?

- Adds integration tests for env and mount secrets.

## How was this patch tested?

Manually by checking that secrets were added to the containers and by tuning the tests.

![image](https://user-images.githubusercontent.com/7945591/42968472-fee3740a-8bab-11e8-9eac-573f67d861fc.png)

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

Closes #21652 from skonto/add-secret-its.
2018-07-20 07:55:58 -05:00
pgandhi 1272b2034d [SPARK-22151] PYTHONPATH not picked up from the spark.yarn.appMaste…
…rEnv properly

Running in yarn cluster mode and trying to set pythonpath via spark.yarn.appMasterEnv.PYTHONPATH doesn't work.

the yarn Client code looks at the env variables:
val pythonPathStr = (sys.env.get("PYTHONPATH") ++ pythonPath)
But when you set spark.yarn.appMasterEnv it puts it into the local env.

So the python path set in spark.yarn.appMasterEnv isn't properly set.

You can work around if you are running in cluster mode by setting it on the client like:

PYTHONPATH=./addon/python/ spark-submit

## What changes were proposed in this pull request?
In Client.scala, PYTHONPATH was being overridden, so changed code to append values to PYTHONPATH instead of overriding them.

## How was this patch tested?
Added log statements to ApplicationMaster.scala to check for environment variable PYTHONPATH, ran a spark job in cluster mode before the change and verified the issue. Performed the same test after the change and verified the fix.

Author: pgandhi <pgandhi@oath.com>

Closes #21468 from pgandhi999/SPARK-22151.
2018-07-18 14:07:03 -05:00
mcheah fc0c8c9717 [SPARK-24825][K8S][TEST] Kubernetes integration tests build the whole reactor
## What changes were proposed in this pull request?

Make the integration test script build all modules.

In order to not run all the non-Kubernetes integration tests in the build, support specifying tags and tag all integration tests specifically with "k8s". Supply the k8s tag in the dev/dev-run-integration-tests.sh script.

## How was this patch tested?

The build system will test this.

Author: mcheah <mcheah@palantir.com>

Closes #21800 from mccheah/k8s-integration-tests-maven-fix.
2018-07-18 10:01:39 -07:00
Ilan Filonenko f1a99ad582 [SPARK-23984][K8S][TEST] Added Integration Tests for PySpark on Kubernetes
## What changes were proposed in this pull request?

I added integration tests for PySpark ( + checking JVM options + RemoteFileTest) which wasn't properly merged in the initial integration test PR.

## How was this patch tested?

I tested this with integration tests using:

`dev/dev-run-integration-tests.sh --spark-tgz spark-2.4.0-SNAPSHOT-bin-2.7.3.tgz`

Author: Ilan Filonenko <if56@cornell.edu>

Closes #21583 from ifilonenko/master.
2018-07-13 17:19:28 -07:00
Kazuaki Ishizaki 5ad4735bda [SPARK-24529][BUILD][TEST-MAVEN] Add spotbugs into maven build process
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

Existing UTs

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

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

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

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

----

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

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

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

Closes #21260 from andrusha/k8s-vol.
2018-07-10 22:53:44 -07:00
Stavros Kontopoulos e71e93aaaa [SPARK-24694][K8S] Pass all app args to integration tests
## What changes were proposed in this pull request?
- Allows to pass more than one app args to tests.
## How was this patch tested?
Manually tested it with a spark test that requires more than on app args.

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

Closes #21672 from skonto/fix_itsets-args.
2018-07-05 16:35:16 -05:00
Stavros Kontopoulos 7bd6d54120 [SPARK-24711][K8S] Fix tags for integration tests
## What changes were proposed in this pull request?

- disables maven surfire plugin to allow tags function properly, doc here: http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin

## How was this patch tested?

Manually by adding tags.

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

Closes #21697 from skonto/fix-tags.
2018-07-05 08:38:26 -05:00
Stavros Kontopoulos 85fe1297e3 [SPARK-24428][K8S] Fix unused code
## What changes were proposed in this pull request?

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

## How was this patch tested?
Manually.

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

Closes #21462 from skonto/fix-k8s-docs.
2018-07-02 13:08:16 -07:00
mcheah 42815548c7 [SPARK-24683][K8S] Fix k8s no resource
## What changes were proposed in this pull request?

Make SparkSubmit pass in the main class even if `SparkLauncher.NO_RESOURCE` is the primary resource.

## How was this patch tested?

New integration test written to capture this case.

Author: mcheah <mcheah@palantir.com>

Closes #21660 from mccheah/fix-k8s-no-resource.
2018-07-02 10:24:04 -07:00
xueyu f71e8da5ef [SPARK-24566][CORE] Fix spark.storage.blockManagerSlaveTimeoutMs default config
This PR use spark.network.timeout in place of spark.storage.blockManagerSlaveTimeoutMs when it is not configured, as configuration doc said

manual test

Author: xueyu <278006819@qq.com>

Closes #21575 from xueyumusic/slaveTimeOutConfig.
2018-06-29 10:44:49 -07:00
Marcelo Vanzin 78ecb6d457 [SPARK-24446][YARN] Properly quote library path for YARN.
Because the way YARN executes commands via bash -c, everything needs
to be quoted so that the whole command is fully contained inside a
bash string and is interpreted correctly when the string is read by
bash. This is a bit different than the quoting done when executing
things as if typing in a bash shell.

Tweaked unit tests to exercise the bad behavior, which would cause
existing tests to time out without the fix. Also tested on a real
cluster, verifying the shell script created by YARN to run the
container.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21476 from vanzin/SPARK-24446.
2018-06-27 10:57:29 -07:00
“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
Marcelo Vanzin 95f9659abe [SPARK-22948][K8S] Move SparkPodInitContainer to correct package.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20156 from vanzin/SPARK-22948.
2018-01-04 15:00:09 -08:00
Sean Owen c284c4e1f6 [MINOR] Fix a bunch of typos 2018-01-02 07:10:19 +09:00
Marcelo Vanzin cfcd746689 [SPARK-11035][CORE] Add in-process Spark app launcher.
This change adds a new launcher that allows applications to be run
in a separate thread in the same process as the calling code. To
achieve that, some code from the child process implementation was
moved to abstract classes that implement the common functionality,
and the new launcher inherits from those.

The new launcher was added as a new class, instead of implemented
as a new option to the existing SparkLauncher, to avoid ambigous
APIs. For example, SparkLauncher has ways to set the child app's
environment, modify SPARK_HOME, or control the logging of the
child process, none of which apply to in-process apps.

The in-process launcher has limitations: it needs Spark in the
context class loader of the calling thread, and it's bound by
Spark's current limitation of a single client-mode application
per JVM. It also relies on the recently added SparkApplication
trait to make sure different apps don't mess up each other's
configuration, so config isolation is currently limited to cluster mode.

I also chose to keep the same socket-based communication for in-process
apps, even though it might be possible to avoid it for in-process
mode. That helps both implementations share more code.

Tested with new and existing unit tests, and with a simple app that
uses the launcher; also made sure the app ran fine with older launcher
jar to check binary compatibility.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19591 from vanzin/SPARK-11035.
2017-12-28 17:00:49 -06:00
Yinan Li 171f6ddadc [SPARK-22757][KUBERNETES] Enable use of remote dependencies (http, s3, gcs, etc.) in Kubernetes mode
## What changes were proposed in this pull request?

This PR expands the Kubernetes mode to be able to use remote dependencies on http/https endpoints, GCS, S3, etc. It adds steps for configuring and appending the Kubernetes init-container into the driver and executor pods for downloading remote dependencies.
[Init-containers](https://kubernetes.io/docs/concepts/workloads/pods/init-containers/), as the name suggests, are containers that are run to completion before the main containers start, and are often used to perform initialization tasks prior to starting the main containers. We use init-containers to localize remote application dependencies before the driver/executors start running. The code that the init-container runs is also included. This PR also adds a step to the driver and executors for mounting user-specified secrets that may store credentials for accessing data storage, e.g., S3 and Google Cloud Storage (GCS), into the driver and executors.

## How was this patch tested?

* The patch contains unit tests which are passing.
* Manual testing: `./build/mvn -Pkubernetes clean package` succeeded.
* Manual testing of the following cases:
  * [x] Running SparkPi using container-local spark-example jar.
  * [x] Running SparkPi using container-local spark-example jar with user-specific secret mounted.
  * [x] Running SparkPi using spark-example jar hosted remotely on an https endpoint.

cc rxin felixcheung mateiz (shepherd)
k8s-big-data SIG members & contributors: mccheah foxish ash211 ssuchter varunkatta kimoonkim erikerlandson tnachen ifilonenko liyinan926
reviewers: vanzin felixcheung jiangxb1987 mridulm

Author: Yinan Li <liyinan926@gmail.com>

Closes #19954 from liyinan926/init-container.
2017-12-28 13:44:44 +09:00
Anirudh Ramanathan 22e1849bcf [SPARK-22866][K8S] Fix path issue in Kubernetes dockerfile
## What changes were proposed in this pull request?

The path was recently changed in https://github.com/apache/spark/pull/19946, but the dockerfile was not updated.
This is a trivial 1 line fix.

## How was this patch tested?

`./sbin/build-push-docker-images.sh -r spark-repo -t latest build`

cc/ vanzin mridulm rxin jiangxb1987 liyinan926

Author: Anirudh Ramanathan <ramanathana@google.com>
Author: foxish <ramanathana@google.com>

Closes #20051 from foxish/patch-1.
2017-12-21 21:03:10 -08:00
foxish 0114c89d04 [SPARK-22845][SCHEDULER] Modify spark.kubernetes.allocation.batch.delay to take time instead of int
## What changes were proposed in this pull request?

Fixing configuration that was taking an int which should take time. Discussion in https://github.com/apache/spark/pull/19946#discussion_r156682354
Made the granularity milliseconds as opposed to seconds since there's a use-case for sub-second reactions to scale-up rapidly especially with dynamic allocation.

## How was this patch tested?

TODO: manual run of integration tests against this PR.
PTAL

cc/ mccheah liyinan926 kimoonkim vanzin mridulm jiangxb1987 ueshin

Author: foxish <ramanathana@google.com>

Closes #20032 from foxish/fix-time-conf.
2017-12-20 16:14:36 -08:00
foxish 0609dcc038 [SPARK-22777][SCHEDULER] Kubernetes mode dockerfile permission and distribution
# What changes were proposed in this pull request?
1. entrypoint.sh for Kubernetes spark-base image is marked as executable (644 -> 755)
2. make-distribution script will now create kubernetes/dockerfiles directory when Kubernetes support is compiled.

## How was this patch tested?
Manual testing

cc/ ueshin jiangxb1987 mridulm vanzin rxin liyinan926

Author: foxish <ramanathana@google.com>

Closes #20007 from foxish/fix-dockerfiles.
2017-12-18 15:31:47 -08:00
foxish fb3636b482 [SPARK-22807][SCHEDULER] Remove config that says docker and replace with container
## What changes were proposed in this pull request?
Changes discussed in https://github.com/apache/spark/pull/19946#discussion_r157063535
docker -> container, since with CRI, we are not limited to running only docker images.

## How was this patch tested?
Manual testing

Author: foxish <ramanathana@google.com>

Closes #19995 from foxish/make-docker-container.
2017-12-18 11:29:32 -08:00
Yinan Li 2fe16333d5 [SPARK-22778][KUBERNETES] Added the missing service metadata for KubernetesClusterManager
## What changes were proposed in this pull request?

This PR added the missing service metadata for `KubernetesClusterManager`. Without the metadata, the service loader couldn't load `KubernetesClusterManager`, and caused the driver to fail to create a `ExternalClusterManager`, as being reported in SPARK-22778. The PR also changed the `k8s:` prefix used to `k8s://`, which is what existing Spark on k8s users are familiar and used to.

## How was this patch tested?

Manual testing verified that the fix resolved the issue in SPARK-22778.

/cc vanzin felixcheung jiangxb1987

Author: Yinan Li <liyinan926@gmail.com>

Closes #19972 from liyinan926/fix-22778.
2017-12-14 14:03:08 -08:00
German Schiavon 0bdb4e516c [SPARK-22574][MESOS][SUBMIT] Check submission request parameters
## What changes were proposed in this pull request?

PR closed with all the comments -> https://github.com/apache/spark/pull/19793

It solves the problem when submitting a wrong CreateSubmissionRequest to Spark Dispatcher was causing a bad state of Dispatcher and making it inactive as a Mesos framework.

https://issues.apache.org/jira/browse/SPARK-22574

## How was this patch tested?

All spark test passed successfully.

It was tested sending a wrong request (without appArgs) before and after the change. The point is easy, check if the value is null before being accessed.

This was before the change, leaving the dispatcher inactive:

```
Exception in thread "Thread-22" java.lang.NullPointerException
	at org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler.getDriverCommandValue(MesosClusterScheduler.scala:444)
	at org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler.buildDriverCommand(MesosClusterScheduler.scala:451)
	at org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler.org$apache$spark$scheduler$cluster$mesos$MesosClusterScheduler$$createTaskInfo(MesosClusterScheduler.scala:538)
	at org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler$$anonfun$scheduleTasks$1.apply(MesosClusterScheduler.scala:570)
	at org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler$$anonfun$scheduleTasks$1.apply(MesosClusterScheduler.scala:555)
	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
	at org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler.scheduleTasks(MesosClusterScheduler.scala:555)
	at org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler.resourceOffers(MesosClusterScheduler.scala:621)
```

And after:

```
  "message" : "Malformed request: org.apache.spark.deploy.rest.SubmitRestProtocolException: Validation of message CreateSubmissionRequest failed!\n\torg.apache.spark.deploy.rest.SubmitRestProtocolMessage.validate(SubmitRestProtocolMessage.scala:70)\n\torg.apache.spark.deploy.rest.SubmitRequestServlet.doPost(RestSubmissionServer.scala:272)\n\tjavax.servlet.http.HttpServlet.service(HttpServlet.java:707)\n\tjavax.servlet.http.HttpServlet.service(HttpServlet.java:790)\n\torg.spark_project.jetty.servlet.ServletHolder.handle(ServletHolder.java:845)\n\torg.spark_project.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:583)\n\torg.spark_project.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1180)\n\torg.spark_project.jetty.servlet.ServletHandler.doScope(ServletHandler.java:511)\n\torg.spark_project.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1112)\n\torg.spark_project.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:141)\n\torg.spark_project.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:134)\n\torg.spark_project.jetty.server.Server.handle(Server.java:524)\n\torg.spark_project.jetty.server.HttpChannel.handle(HttpChannel.java:319)\n\torg.spark_project.jetty.server.HttpConnection.onFillable(HttpConnection.java:253)\n\torg.spark_project.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:273)\n\torg.spark_project.jetty.io.FillInterest.fillable(FillInterest.java:95)\n\torg.spark_project.jetty.io.SelectChannelEndPoint$2.run(SelectChannelEndPoint.java:93)\n\torg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.executeProduceConsume(ExecuteProduceConsume.java:303)\n\torg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.produceConsume(ExecuteProduceConsume.java:148)\n\torg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:136)\n\torg.spark_project.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)\n\torg.spark_project.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)\n\tjava.lang.Thread.run(Thread.java:745)"
```

Author: German Schiavon <germanschiavon@gmail.com>

Closes #19966 from Gschiavon/fix-submission-request.
2017-12-13 13:37:25 -08:00
gatorsmile 704af4bd67 Revert "[SPARK-22574][MESOS][SUBMIT] Check submission request parameters"
This reverts commit 7a51e71355.
2017-12-12 13:40:01 -08:00
German Schiavon 7a51e71355 [SPARK-22574][MESOS][SUBMIT] Check submission request parameters
## What changes were proposed in this pull request?

It solves the problem when submitting a wrong CreateSubmissionRequest to Spark Dispatcher was causing a bad state of Dispatcher and making it inactive as a Mesos framework.

https://issues.apache.org/jira/browse/SPARK-22574

## How was this patch tested?

All spark test passed successfully.

It was tested sending a wrong request (without appArgs) before and after the change. The point is easy, check if the value is null before being accessed.

This was before the change, leaving the dispatcher inactive:

```
Exception in thread "Thread-22" java.lang.NullPointerException
	at org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler.getDriverCommandValue(MesosClusterScheduler.scala:444)
	at org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler.buildDriverCommand(MesosClusterScheduler.scala:451)
	at org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler.org$apache$spark$scheduler$cluster$mesos$MesosClusterScheduler$$createTaskInfo(MesosClusterScheduler.scala:538)
	at org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler$$anonfun$scheduleTasks$1.apply(MesosClusterScheduler.scala:570)
	at org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler$$anonfun$scheduleTasks$1.apply(MesosClusterScheduler.scala:555)
	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
	at org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler.scheduleTasks(MesosClusterScheduler.scala:555)
	at org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler.resourceOffers(MesosClusterScheduler.scala:621)
```

And after:

```
  "message" : "Malformed request: org.apache.spark.deploy.rest.SubmitRestProtocolException: Validation of message CreateSubmissionRequest failed!\n\torg.apache.spark.deploy.rest.SubmitRestProtocolMessage.validate(SubmitRestProtocolMessage.scala:70)\n\torg.apache.spark.deploy.rest.SubmitRequestServlet.doPost(RestSubmissionServer.scala:272)\n\tjavax.servlet.http.HttpServlet.service(HttpServlet.java:707)\n\tjavax.servlet.http.HttpServlet.service(HttpServlet.java:790)\n\torg.spark_project.jetty.servlet.ServletHolder.handle(ServletHolder.java:845)\n\torg.spark_project.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:583)\n\torg.spark_project.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1180)\n\torg.spark_project.jetty.servlet.ServletHandler.doScope(ServletHandler.java:511)\n\torg.spark_project.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1112)\n\torg.spark_project.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:141)\n\torg.spark_project.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:134)\n\torg.spark_project.jetty.server.Server.handle(Server.java:524)\n\torg.spark_project.jetty.server.HttpChannel.handle(HttpChannel.java:319)\n\torg.spark_project.jetty.server.HttpConnection.onFillable(HttpConnection.java:253)\n\torg.spark_project.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:273)\n\torg.spark_project.jetty.io.FillInterest.fillable(FillInterest.java:95)\n\torg.spark_project.jetty.io.SelectChannelEndPoint$2.run(SelectChannelEndPoint.java:93)\n\torg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.executeProduceConsume(ExecuteProduceConsume.java:303)\n\torg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.produceConsume(ExecuteProduceConsume.java:148)\n\torg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:136)\n\torg.spark_project.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)\n\torg.spark_project.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)\n\tjava.lang.Thread.run(Thread.java:745)"
```

Author: German Schiavon <germanschiavon@gmail.com>

Closes #19793 from Gschiavon/fix-submission-request.
2017-12-12 11:46:57 -08:00
Yinan Li 3f4060c340 [SPARK-22646][K8S] Spark on Kubernetes - basic submission client
This PR contains implementation of the basic submission client for the cluster mode of Spark on Kubernetes. It's step 2 from the step-wise plan documented [here](https://github.com/apache-spark-on-k8s/spark/issues/441#issuecomment-330802935).
This addition is covered by the [SPIP](http://apache-spark-developers-list.1001551.n3.nabble.com/SPIP-Spark-on-Kubernetes-td22147.html) vote which passed on Aug 31.

This PR and #19468 together form a MVP of Spark on Kubernetes that allows users to run Spark applications that use resources locally within the driver and executor containers on Kubernetes 1.6 and up. Some changes on pom and build/test setup are copied over from #19468 to make this PR self contained and testable.

The submission client is mainly responsible for creating the Kubernetes pod that runs the Spark driver. It follows a step-based approach to construct the driver pod, as the code under the `submit.steps` package shows. The steps are orchestrated by `DriverConfigurationStepsOrchestrator`. `Client` creates the driver pod and waits for the application to complete if it's configured to do so, which is the case by default.

This PR also contains Dockerfiles of the driver and executor images. They are included because some of the environment variables set in the code would not make sense without referring to the Dockerfiles.

* The patch contains unit tests which are passing.
* Manual testing: ./build/mvn -Pkubernetes clean package succeeded.
* It is a subset of the entire changelist hosted at http://github.com/apache-spark-on-k8s/spark which is in active use in several organizations.
* There is integration testing enabled in the fork currently hosted by PepperData which is being moved over to RiseLAB CI.
* Detailed documentation on trying out the patch in its entirety is in: https://apache-spark-on-k8s.github.io/userdocs/running-on-kubernetes.html

cc rxin felixcheung mateiz (shepherd)
k8s-big-data SIG members & contributors: mccheah foxish ash211 ssuchter varunkatta kimoonkim erikerlandson tnachen ifilonenko liyinan926

Author: Yinan Li <liyinan926@gmail.com>

Closes #19717 from liyinan926/spark-kubernetes-4.
2017-12-11 15:15:05 -08:00
Devaraj K 51066b437b [SPARK-14228][CORE][YARN] Lost executor of RPC disassociated, and occurs exception: Could not find CoarseGrainedScheduler or it has been stopped
## What changes were proposed in this pull request?
I see the two instances where the exception is occurring.

**Instance 1:**

```
17/11/10 15:49:32 ERROR util.Utils: Uncaught exception in thread driver-revive-thread
org.apache.spark.SparkException: Could not find CoarseGrainedScheduler.
        at org.apache.spark.rpc.netty.Dispatcher.postMessage(Dispatcher.scala:160)
        at org.apache.spark.rpc.netty.Dispatcher.postOneWayMessage(Dispatcher.scala:140)
        at org.apache.spark.rpc.netty.NettyRpcEnv.send(NettyRpcEnv.scala:187)
        at org.apache.spark.rpc.netty.NettyRpcEndpointRef.send(NettyRpcEnv.scala:521)
        at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anon$1$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(CoarseGrainedSchedulerBackend.scala:125)
        at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anon$1$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(CoarseGrainedSchedulerBackend.scala:125)
        at scala.Option.foreach(Option.scala:257)
        at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anon$1$$anonfun$run$1.apply$mcV$sp(CoarseGrainedSchedulerBackend.scala:125)
        at org.apache.spark.util.Utils$.tryLogNonFatalError(Utils.scala:1344)
        at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anon$1.run(CoarseGrainedSchedulerBackend.scala:124)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
        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)
```

In CoarseGrainedSchedulerBackend.scala, driver-revive-thread starts with DriverEndpoint.onStart() and keeps sending the ReviveOffers messages periodically till it gets shutdown as part DriverEndpoint.onStop(). There is no proper coordination between the driver-revive-thread(shutdown) and the RpcEndpoint unregister, RpcEndpoint unregister happens first and then driver-revive-thread shuts down as part of DriverEndpoint.onStop(), In-between driver-revive-thread may try to send the ReviveOffers message which is leading to the above exception.

To fix this issue, this PR moves the shutting down of driver-revive-thread to CoarseGrainedSchedulerBackend.stop() which executes before the DriverEndpoint unregister.

**Instance 2:**

```
17/11/10 16:31:38 ERROR cluster.YarnSchedulerBackend$YarnSchedulerEndpoint: Error requesting driver to remove executor 1 for reason Executor for container container_1508535467865_0226_01_000002 exited because of a YARN event (e.g., pre-emption) and not because of an error in the running job.
org.apache.spark.SparkException: Could not find CoarseGrainedScheduler.
        at org.apache.spark.rpc.netty.Dispatcher.postMessage(Dispatcher.scala:160)
        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:516)
        at org.apache.spark.rpc.RpcEndpointRef.ask(RpcEndpointRef.scala:63)
        at org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnSchedulerEndpoint$$anonfun$receive$1.applyOrElse(YarnSchedulerBackend.scala:269)
        at org.apache.spark.rpc.netty.Inbox$$anonfun$process$1.apply$mcV$sp(Inbox.scala:117)
        at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:205)
        at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:101)
        at org.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:221)
        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)
```

Here YarnDriverEndpoint tries to send remove executor messages after the Yarn scheduler backend service stop, which is leading to the above exception. To avoid the above exception,
1) We may add a condition(which checks whether service has stopped or not) before sending executor remove message
2) Add a warn log message in onFailure case when the service is already stopped

In this PR, chosen the 2) option which adds a log message in the case of onFailure without the exception stack trace since the option 1) would need to to go through for every remove executor message.

## How was this patch tested?
I verified it manually, I don't see these exceptions with the PR changes.

Author: Devaraj K <devaraj@apache.org>

Closes #19741 from devaraj-kavali/SPARK-14228.
2017-12-06 10:39:15 -08:00
Marcelo Vanzin e1dd03e42c [SPARK-22372][CORE, YARN] Make cluster submission use SparkApplication.
The main goal of this change is to allow multiple cluster-mode
submissions from the same JVM, without having them end up with
mixed configuration. That is done by extending the SparkApplication
trait, and doing so was reasonably trivial for standalone and
mesos modes.

For YARN mode, there was a complication. YARN used a "SPARK_YARN_MODE"
system property to control behavior indirectly in a whole bunch of
places, mainly in the SparkHadoopUtil / YarnSparkHadoopUtil classes.
Most of the changes here are removing that.

Since we removed support for Hadoop 1.x, some methods that lived in
YarnSparkHadoopUtil can now live in SparkHadoopUtil. The remaining
methods don't need to be part of the class, and can be called directly
from the YarnSparkHadoopUtil object, so now there's a single
implementation of SparkHadoopUtil.

There were two places in the code that relied on  SPARK_YARN_MODE to
make decisions about YARN-specific functionality, and now explicitly check
the master from the configuration for that instead:

* fetching the external shuffle service port, which can come from the YARN
  configuration.

* propagation of the authentication secret using Hadoop credentials. This also
  was cleaned up a little to not need so many methods in `SparkHadoopUtil`.

With those out of the way, actually changing the YARN client
to extend SparkApplication was easy.

Tested with existing unit tests, and also by running YARN apps
with auth and kerberos both on and off in a real cluster.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19631 from vanzin/SPARK-22372.
2017-12-04 11:05:03 -08:00
Stavros Kontopoulos 193555f79c [SPARK-18935][MESOS] Fix dynamic reservations on mesos
## What changes were proposed in this pull request?

- Solves the issue described in the ticket by preserving reservation and allocation info in all cases (port handling included).
- upgrades to 1.4
- Adds extra debug level logging to make debugging easier in the future, for example we add reservation info when applicable.
```
 17/09/29 14:53:07 DEBUG MesosCoarseGrainedSchedulerBackend: Accepting offer: f20de49b-dee3-45dd-a3c1-73418b7de891-O32 with attributes: Map() allocation info: role: "spark-prive"
  reservation info: name: "ports"
 type: RANGES
 ranges {
   range {
     begin: 31000
     end: 32000
   }
 }
 role: "spark-prive"
 reservation {
   principal: "test"
 }
 allocation_info {
   role: "spark-prive"
 }
```
- Some style cleanup.

## How was this patch tested?

Manually by running the example in the ticket with and without a principal. Specifically I tested it on a dc/os 1.10 cluster with 7 nodes and played with reservations. From the master node in order to reserve resources I executed:

```for i in 0 1 2 3 4 5 6
do
curl -i \
      -d slaveId=90ec65ea-1f7b-479f-a824-35d2527d6d26-S$i \
      -d resources='[
        {
          "name": "cpus",
          "type": "SCALAR",
          "scalar": { "value": 2 },
          "role": "spark-role",
          "reservation": {
            "principal": ""
          }
        },
        {
          "name": "mem",
          "type": "SCALAR",
          "scalar": { "value": 8026 },
          "role": "spark-role",
          "reservation": {
            "principal": ""
          }
        }
      ]' \
      -X POST http://master.mesos:5050/master/reserve
done
```
Nodes had 4 cpus (m3.xlarge instances)  and I reserved either 2 or 4 cpus (all for a role).
I verified it launches tasks on nodes with reserved resources under `spark-role` role  only if
a) there are remaining resources for (*) default role and the spark driver has no role assigned to it.
b) the spark driver has a role assigned to it and it is the same role used in reservations.
I also tested this locally on my machine.

Author: Stavros Kontopoulos <st.kontopoulos@gmail.com>

Closes #19390 from skonto/fix_dynamic_reservation.
2017-11-29 14:15:35 -08:00
Yinan Li e9b2070ab2 [SPARK-18278][SCHEDULER] Spark on Kubernetes - Basic Scheduler Backend
## What changes were proposed in this pull request?

This is a stripped down version of the `KubernetesClusterSchedulerBackend` for Spark with the following components:
- Static Allocation of Executors
- Executor Pod Factory
- Executor Recovery Semantics

It's step 1 from the step-wise plan documented [here](https://github.com/apache-spark-on-k8s/spark/issues/441#issuecomment-330802935).
This addition is covered by the [SPIP vote](http://apache-spark-developers-list.1001551.n3.nabble.com/SPIP-Spark-on-Kubernetes-td22147.html) which passed on Aug 31 .

## How was this patch tested?

- The patch contains unit tests which are passing.
- Manual testing: `./build/mvn -Pkubernetes clean package` succeeded.
- It is a **subset** of the entire changelist hosted in http://github.com/apache-spark-on-k8s/spark which is in active use in several organizations.
- There is integration testing enabled in the fork currently [hosted by PepperData](spark-k8s-jenkins.pepperdata.org:8080) which is being moved over to RiseLAB CI.
- Detailed documentation on trying out the patch in its entirety is in: https://apache-spark-on-k8s.github.io/userdocs/running-on-kubernetes.html

cc rxin felixcheung mateiz (shepherd)
k8s-big-data SIG members & contributors: mccheah ash211 ssuchter varunkatta kimoonkim erikerlandson liyinan926 tnachen ifilonenko

Author: Yinan Li <liyinan926@gmail.com>
Author: foxish <ramanathana@google.com>
Author: mcheah <mcheah@palantir.com>

Closes #19468 from foxish/spark-kubernetes-3.
2017-11-28 23:02:09 -08:00
Kalvin Chau 4d8ace4869 [SPARK-22583] First delegation token renewal time is not 75% of renewal time in Mesos
The first scheduled renewal time is is set to the exact expiration time,
and all subsequent renewal times are 75% of the renewal time. This makes
it so that the inital renewal time is also 75%.

## What changes were proposed in this pull request?

Set the initial renewal time to be 75% of renewal time.

## How was this patch tested?

Tested locally in a test HDFS cluster, checking various renewal times.

Author: Kalvin Chau <kalvin.chau@viasat.com>

Closes #19798 from kalvinnchau/fix-inital-renewal-time.
2017-11-25 07:32:28 -06:00
ArtRand 1e82335413 [SPARK-21842][MESOS] Support Kerberos ticket renewal and creation in Mesos
## What changes were proposed in this pull request?
tl;dr: Add a class, `MesosHadoopDelegationTokenManager` that updates delegation tokens on a schedule on the behalf of Spark Drivers. Broadcast renewed credentials to the executors.

## The problem
We recently added Kerberos support to Mesos-based Spark jobs as well as Secrets support to the Mesos Dispatcher (SPARK-16742, SPARK-20812, respectively). However the delegation tokens have a defined expiration. This poses a problem for long running Spark jobs (e.g. Spark Streaming applications). YARN has a solution for this where a thread is scheduled to renew the tokens they reach 75% of their way to expiration. It then writes the tokens to HDFS for the executors to find (uses a monotonically increasing suffix).

## This solution
We replace the current method in `CoarseGrainedSchedulerBackend` which used to discard the token renewal time with a protected method `fetchHadoopDelegationTokens`. Now the individual cluster backends are responsible for overriding this method to fetch and manage token renewal. The delegation tokens themselves, are still part of the `CoarseGrainedSchedulerBackend` as before.
In the case of Mesos renewed Credentials are broadcasted to the executors. This maintains all transfer of Credentials within Spark (as opposed to Spark-to-HDFS). It also does not require any writing of Credentials to disk. It also does not require any GC of old files.

## How was this patch tested?
Manually against a Kerberized HDFS cluster.

Thank you for the reviews.

Author: ArtRand <arand@soe.ucsc.edu>

Closes #19272 from ArtRand/spark-21842-450-kerberos-ticket-renewal.
2017-11-15 15:53:05 -08:00
Devaraj K eaff295a23 [SPARK-22519][YARN] Remove unnecessary stagingDirPath null check in ApplicationMaster.cleanupStagingDir()
## What changes were proposed in this pull request?
Removed the unnecessary stagingDirPath null check in ApplicationMaster.cleanupStagingDir().

## How was this patch tested?
I verified with the existing test cases.

Author: Devaraj K <devaraj@apache.org>

Closes #19749 from devaraj-kavali/SPARK-22519.
2017-11-14 15:20:03 -08:00
Paul Mackles b3f9dbf48e [SPARK-19606][MESOS] Support constraints in spark-dispatcher
## What changes were proposed in this pull request?

A discussed in SPARK-19606, the addition of a new config property named "spark.mesos.constraints.driver" for constraining drivers running on a Mesos cluster

## How was this patch tested?

Corresponding unit test added also tested locally on a Mesos cluster

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

Author: Paul Mackles <pmackles@adobe.com>

Closes #19543 from pmackles/SPARK-19606.
2017-11-12 11:21:23 -08:00
Kent Yao c755b0d910 [SPARK-22463][YARN][SQL][HIVE] add hadoop/hive/hbase/etc configuration files in SPARK_CONF_DIR to distribute archive
## What changes were proposed in this pull request?
When I ran self contained sql apps, such as
```scala
import org.apache.spark.sql.SparkSession

object ShowHiveTables {
  def main(args: Array[String]): Unit = {
    val spark = SparkSession
      .builder()
      .appName("Show Hive Tables")
      .enableHiveSupport()
      .getOrCreate()
    spark.sql("show tables").show()
    spark.stop()
  }
}
```
with **yarn cluster** mode and `hive-site.xml` correctly within `$SPARK_HOME/conf`,they failed to connect the right hive metestore for not seeing hive-site.xml in AM/Driver's classpath.

Although submitting them with `--files/--jars local/path/to/hive-site.xml` or puting it to `$HADOOP_CONF_DIR/YARN_CONF_DIR` can make these apps works well in cluster mode as client mode, according to the official doc, see  http://spark.apache.org/docs/latest/sql-programming-guide.html#hive-tables
> Configuration of Hive is done by placing your hive-site.xml, core-site.xml (for security configuration), and hdfs-site.xml (for HDFS configuration) file in conf/.

We may respect these configuration files too or modify the doc for hive-tables in cluster mode.
## How was this patch tested?

cc cloud-fan gatorsmile

Author: Kent Yao <yaooqinn@hotmail.com>

Closes #19663 from yaooqinn/SPARK-21888.
2017-11-09 09:22:33 +01:00
Stavros Kontopoulos b2463fad71 [SPARK-22145][MESOS] fix supervise with checkpointing on mesos
## What changes were proposed in this pull request?

- Fixes the issue with the frameworkId being recovered by checkpointed data overwriting the one sent by the dipatcher.
- Keeps submission driver id as the only index for all data structures in the dispatcher.
Allocates a different task id per driver retry to satisfy the mesos requirements. Check the relevant ticket for the details on that.
## How was this patch tested?

Manually tested this with DC/OS 1.10. Launched a streaming job with checkpointing to hdfs, made the driver fail several times and observed behavior:
![image](https://user-images.githubusercontent.com/7945591/30940500-f7d2a744-a3e9-11e7-8c56-f2ccbb271e80.png)

![image](https://user-images.githubusercontent.com/7945591/30940550-19bc15de-a3ea-11e7-8a11-f48abfe36720.png)

![image](https://user-images.githubusercontent.com/7945591/30940524-083ea308-a3ea-11e7-83ae-00d3fa17b928.png)

![image](https://user-images.githubusercontent.com/7945591/30940579-2f0fb242-a3ea-11e7-82f9-86179da28b8c.png)

![image](https://user-images.githubusercontent.com/7945591/30940591-3b561b0e-a3ea-11e7-9dbd-e71912bb2ef3.png)

![image](https://user-images.githubusercontent.com/7945591/30940605-49c810ca-a3ea-11e7-8af5-67930851fd38.png)

![image](https://user-images.githubusercontent.com/7945591/30940631-59f4a288-a3ea-11e7-88cb-c3741b72bb13.png)

![image](https://user-images.githubusercontent.com/7945591/30940642-62346c9e-a3ea-11e7-8935-82e494925f67.png)

![image](https://user-images.githubusercontent.com/7945591/30940653-6c46d53c-a3ea-11e7-8dd1-5840d484d28c.png)

Author: Stavros Kontopoulos <st.kontopoulos@gmail.com>

Closes #19374 from skonto/fix_retry.
2017-11-02 13:25:48 +00:00
Susan X. Huynh 5415963d2c [SPARK-22131][MESOS] Mesos driver secrets
## Background

In #18837 , ArtRand added Mesos secrets support to the dispatcher. **This PR is to add the same secrets support to the drivers.** This means if the secret configs are set, the driver will launch executors that have access to either env or file-based secrets.

One use case for this is to support TLS in the driver <=> executor communication.

## What changes were proposed in this pull request?

Most of the changes are a refactor of the dispatcher secrets support (#18837) - moving it to a common place that can be used by both the dispatcher and drivers. The same goes for the unit tests.

## How was this patch tested?

There are four config combinations: [env or file-based] x [value or reference secret]. For each combination:
- Added a unit test.
- Tested in DC/OS.

Author: Susan X. Huynh <xhuynh@mesosphere.com>

Closes #19437 from susanxhuynh/sh-mesos-driver-secret.
2017-10-26 16:13:48 -07:00
Marcelo Vanzin d212ef14be [SPARK-22341][YARN] Impersonate correct user when preparing resources.
The bug was introduced in SPARK-22290, which changed how the app's user
is impersonated in the AM. The changed missed an initialization function
that needs to be run as the app owner (who has the right credentials to
read from HDFS).

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19566 from vanzin/SPARK-22341.
2017-10-25 13:53:01 -07:00
Marcelo Vanzin dc2714da50 [SPARK-22290][CORE] Avoid creating Hive delegation tokens when not necessary.
Hive delegation tokens are only needed when the Spark driver has no access
to the kerberos TGT. That happens only in two situations:

- when using a proxy user
- when using cluster mode without a keytab

This change modifies the Hive provider so that it only generates delegation
tokens in those situations, and tweaks the YARN AM so that it makes the proper
user visible to the Hive code when running with keytabs, so that the TGT
can be used instead of a delegation token.

The effect of this change is that now it's possible to initialize multiple,
non-concurrent SparkContext instances in the same JVM. Before, the second
invocation would fail to fetch a new Hive delegation token, which then could
make the second (or third or...) application fail once the token expired.
With this change, the TGT will be used to authenticate to the HMS instead.

This change also avoids polluting the current logged in user's credentials
when launching applications. The credentials are copied only when running
applications as a proxy user. This makes it possible to implement SPARK-11035
later, where multiple threads might be launching applications, and each app
should have its own set of credentials.

Tested by verifying HDFS and Hive access in following scenarios:
- client and cluster mode
- client and cluster mode with proxy user
- client and cluster mode with principal / keytab
- long-running cluster app with principal / keytab
- pyspark app that creates (and stops) multiple SparkContext instances
  through its lifetime

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19509 from vanzin/SPARK-22290.
2017-10-19 14:56:48 +08:00
Devaraj K 06df34d35e [SPARK-11034][LAUNCHER][MESOS] Launcher: add support for monitoring Mesos apps
## What changes were proposed in this pull request?

Added Launcher support for monitoring Mesos apps in Client mode. SPARK-11033 can handle the support for Mesos/Cluster mode since the Standalone/Cluster and Mesos/Cluster modes use the same code at client side.

## How was this patch tested?

I verified it manually by running launcher application, able to launch, stop and kill the mesos applications and also can invoke other launcher API's.

Author: Devaraj K <devaraj@apache.org>

Closes #19385 from devaraj-kavali/SPARK-11034.
2017-10-13 17:12:50 -07:00
Gene Pang 4329eb2e73 [SPARK-16944][Mesos] Improve data locality when launching new executors when dynamic allocation is enabled
## What changes were proposed in this pull request?

Improve the Spark-Mesos coarse-grained scheduler to consider the preferred locations when dynamic allocation is enabled.

## How was this patch tested?

Added a unittest, and performed manual testing on AWS.

Author: Gene Pang <gene.pang@gmail.com>

Closes #18098 from gpang/mesos_data_locality.
2017-10-02 15:09:11 -07:00
Paul Mackles f20be4d70b [SPARK-22135][MESOS] metrics in spark-dispatcher not being registered properly
## What changes were proposed in this pull request?

Fix a trivial bug with how metrics are registered in the mesos dispatcher. Bug resulted in creating a new registry each time the metricRegistry() method was called.

## How was this patch tested?

Verified manually on local mesos setup

Author: Paul Mackles <pmackles@adobe.com>

Closes #19358 from pmackles/SPARK-22135.
2017-09-28 14:43:31 +08:00
Devaraj K 55d5fa79db [SPARK-21384][YARN] Spark + YARN fails with LocalFileSystem as default FS
## What changes were proposed in this pull request?

When the libraries temp directory(i.e. __spark_libs__*.zip dir) file system and staging dir(destination) file systems are the same then the __spark_libs__*.zip is not copying to the staging directory. But after making this decision the libraries zip file is getting deleted immediately and becoming unavailable for the Node Manager's localization.

With this change, client copies the files to remote always when the source scheme is "file".

## How was this patch tested?

I have verified it manually in yarn/cluster and yarn/client modes with hdfs and local file systems.

Author: Devaraj K <devaraj@apache.org>

Closes #19141 from devaraj-kavali/SPARK-21384.
2017-09-20 16:22:36 -07:00
Marcelo Vanzin bb9c0697d5 [SPARK-18838][HOTFIX][YARN] Check internal context state before stopping it.
The live listener bus now cleans up after itself and releases listeners
after stopping, so code cannot get references to listeners after the
Spark context is stopped.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19297 from vanzin/SPARK-18838.hotfix.
2017-09-20 15:53:28 -07:00
Sean Owen 12ab7f7e89 [SPARK-14280][BUILD][WIP] Update change-version.sh and pom.xml to add Scala 2.12 profiles and enable 2.12 compilation
…build; fix some things that will be warnings or errors in 2.12; restore Scala 2.12 profile infrastructure

## What changes were proposed in this pull request?

This change adds back the infrastructure for a Scala 2.12 build, but does not enable it in the release or Python test scripts.

In order to make that meaningful, it also resolves compile errors that the code hits in 2.12 only, in a way that still works with 2.11.

It also updates dependencies to the earliest minor release of dependencies whose current version does not yet support Scala 2.12. This is in a sense covered by other JIRAs under the main umbrella, but implemented here. The versions below still work with 2.11, and are the _latest_ maintenance release in the _earliest_ viable minor release.

- Scalatest 2.x -> 3.0.3
- Chill 0.8.0 -> 0.8.4
- Clapper 1.0.x -> 1.1.2
- json4s 3.2.x -> 3.4.2
- Jackson 2.6.x -> 2.7.9 (required by json4s)

This change does _not_ fully enable a Scala 2.12 build:

- It will also require dropping support for Kafka before 0.10. Easy enough, just didn't do it yet here
- It will require recreating `SparkILoop` and `Main` for REPL 2.12, which is SPARK-14650. Possible to do here too.

What it does do is make changes that resolve much of the remaining gap without affecting the current 2.11 build.

## How was this patch tested?

Existing tests and build. Manually tested with `./dev/change-scala-version.sh 2.12` to verify it compiles, modulo the exceptions above.

Author: Sean Owen <sowen@cloudera.com>

Closes #18645 from srowen/SPARK-14280.
2017-09-01 19:21:21 +01:00
ArtRand fc45c2c88a [SPARK-20812][MESOS] Add secrets support to the dispatcher
Mesos has secrets primitives for environment and file-based secrets, this PR adds that functionality to the Spark dispatcher and the appropriate configuration flags.
Unit tested and manually tested against a DC/OS cluster with Mesos 1.4.

Author: ArtRand <arand@soe.ucsc.edu>

Closes #18837 from ArtRand/spark-20812-dispatcher-secrets-and-labels.
2017-08-31 10:58:41 -07:00
jerryshao 4482ff23ad [SPARK-17321][YARN] Avoid writing shuffle metadata to disk if NM recovery is disabled
In the current code, if NM recovery is not enabled then `YarnShuffleService` will write shuffle metadata to NM local dir-1, if this local dir-1 is on bad disk, then `YarnShuffleService` will be failed to start. So to solve this issue, in Spark side if NM recovery is not enabled, then Spark will not persist data into leveldb, in that case yarn shuffle service can still be served but lose the ability for recovery, (it is fine because the failure of NM will kill the containers as well as applications).

Tested in the local cluster with NM recovery off and on to see if folder is created or not. MiniCluster UT isn't added because in MiniCluster NM will always set port to 0, but NM recovery requires non-ephemeral port.

Author: jerryshao <sshao@hortonworks.com>

Closes #19032 from jerryshao/SPARK-17321.

Change-Id: I8f2fe73d175e2ad2c4e380caede3873e0192d027
2017-08-31 09:26:20 +08:00
Susan X. Huynh ce0d3bb377 [SPARK-21694][MESOS] Support Mesos CNI network labels
JIRA ticket: https://issues.apache.org/jira/browse/SPARK-21694

## What changes were proposed in this pull request?

Spark already supports launching containers attached to a given CNI network by specifying it via the config `spark.mesos.network.name`.

This PR adds support to pass in network labels to CNI plugins via a new config option `spark.mesos.network.labels`. These network labels are key-value pairs that are set in the `NetworkInfo` of both the driver and executor tasks. More details in the related Mesos documentation:  http://mesos.apache.org/documentation/latest/cni/#mesos-meta-data-to-cni-plugins

## How was this patch tested?

Unit tests, for both driver and executor tasks.
Manual integration test to submit a job with the `spark.mesos.network.labels` option, hit the mesos/state.json endpoint, and check that the labels are set in the driver and executor tasks.

ArtRand skonto

Author: Susan X. Huynh <xhuynh@mesosphere.com>

Closes #18910 from susanxhuynh/sh-mesos-cni-labels.
2017-08-24 10:05:38 +01:00
Andrew Ash a2db5c5761 [MINOR][TYPO] Fix typos: runnning and Excecutors
## What changes were proposed in this pull request?

Fix typos

## How was this patch tested?

Existing tests

Author: Andrew Ash <andrew@andrewash.com>

Closes #18996 from ash211/patch-2.
2017-08-18 13:43:42 -07:00