Commit graph

190 commits

Author SHA1 Message Date
Marcelo Vanzin a63e7b2a21 [SPARK-25877][K8S] Move all feature logic to feature classes.
This change makes the driver and executor builders a lot simpler
by encapsulating almost all feature logic into the respective
feature classes. The only logic that remains is the creation of
the initial pod, which needs to happen before anything else so
is better to be left in the builder class.

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

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

Author: Marcelo Vanzin <vanzin@cloudera.com>

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

Unit tests.

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

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

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

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

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

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

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #23174 from vanzin/SPARK-26194.
2018-12-06 14:17:13 -08:00
Qi Shao 0889fbaf95 [SPARK-26083][K8S] Add Copy pyspark into corresponding dir cmd in pyspark Dockerfile
When I try to run `./bin/pyspark` cmd in a pod in Kubernetes(image built without change from pyspark Dockerfile), I'm getting an error:
```
$SPARK_HOME/bin/pyspark --deploy-mode client --master k8s://https://$KUBERNETES_SERVICE_HOST:$KUBERNETES_SERVICE_PORT_HTTPS ...
Python 2.7.15 (default, Aug 22 2018, 13:24:18)
[GCC 6.4.0] on linux2
Type "help", "copyright", "credits" or "license" for more information.
Could not open PYTHONSTARTUP
IOError: [Errno 2] No such file or directory: '/opt/spark/python/pyspark/shell.py'
```
This is because `pyspark` folder doesn't exist under `/opt/spark/python/`

## What changes were proposed in this pull request?

Added `COPY python/pyspark ${SPARK_HOME}/python/pyspark` to pyspark Dockerfile to resolve issue above.

## How was this patch tested?
Google Kubernetes Engine

Closes #23037 from AzureQ/master.

Authored-by: Qi Shao <qi.shao.nyu@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-03 15:36:41 -08:00
Stavros Kontopoulos a24e1a126c [SPARK-26256][K8S] Fix labels for pod deletion
## What changes were proposed in this pull request?
Adds proper labels when deleting executor pods.

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

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

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

Closes #23136 from skonto/keep_pods.

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

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

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

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

This change fixes those problems by:

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

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

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

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

- removed unnecessary type parameters in KubernetesVolumeSpec

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

- removed a lot of unnecessary mocking from tests.

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

Tested with existing unit tests and integration tests.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #22959 from vanzin/SPARK-25876.
2018-11-30 16:23:37 -08:00
Rob Vesse 1144df3b5d [SPARK-26015][K8S] Set a default UID for Spark on K8S Images
Adds USER directives to the Dockerfiles which is configurable via build argument (`spark_uid`) for easy customisation.  A `-u` flag is added to `bin/docker-image-tool.sh` to make it easy to customise this e.g.

```
> bin/docker-image-tool.sh -r rvesse -t uid -u 185 build
> bin/docker-image-tool.sh -r rvesse -t uid push
```

If no UID is explicitly specified it defaults to `185` - this is per skonto's suggestion to align with the OpenShift standard reserved UID for Java apps (
https://lists.openshift.redhat.com/openshift-archives/users/2016-March/msg00283.html)

Notes:
- We have to make the `WORKDIR` writable by the root group or otherwise jobs will fail with `AccessDeniedException`

To Do:
- [x] Debug and resolve issue with client mode test
- [x] Consider whether to always propagate `SPARK_USER_NAME` to environment of driver and executor pods so `entrypoint.sh` can insert that into `/etc/passwd` entry
- [x] Rebase once PR #23013 is merged and update documentation accordingly

Built the Docker images with the new Dockerfiles that include the `USER` directives.  Ran the Spark on K8S integration tests against the new images.  All pass except client mode which I am currently debugging further.

Also manually dropped myself into the resulting container images via `docker run` and checked `id -u` output to see that UID is as expected.

Tried customising the UID from the default via the new `-u` argument to `docker-image-tool.sh` and again checked the resulting image for the correct runtime UID.

cc felixcheung skonto vanzin

Closes #23017 from rvesse/SPARK-26015.

Authored-by: Rob Vesse <rvesse@dotnetrdf.org>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-29 10:00:12 -08:00
Marcelo Vanzin 2d89d109e1 [SPARK-26025][K8S] Speed up docker image build on dev repo.
The "build context" for a docker image - basically the whole contents of the
current directory where "docker" is invoked - can be huge in a dev build,
easily breaking a couple of gigs.

Doing that copy 3 times during the build of docker images severely slows
down the process.

This patch creates a smaller build context - basically mimicking what the
make-distribution.sh script does, so that when building the docker images,
only the necessary bits are in the current directory. For PySpark and R that
is optimized further, since those images are built based on the previously
built Spark main image.

In my current local clone, the dir size is about 2G, but with this script
the "context" sent to docker is about 250M for the main image, 1M for the
pyspark image and 8M for the R image. That speeds up the image builds
considerably.

I also snuck in a fix to the k8s integration test dependencies in the sbt
build, so that the examples are properly built (without having to do it
manually).

Closes #23019 from vanzin/SPARK-26025.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-27 09:09:16 -08:00
Nihar Sheth 3df307aa51 [SPARK-25960][K8S] Support subpath mounting with Kubernetes
## What changes were proposed in this pull request?

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

## How was this patch tested?

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

Closes #23026 from NiharS/k8s_subpath.

Authored-by: Nihar Sheth <niharrsheth@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-26 11:06:02 -08:00
Takanobu Asanuma 15c0384977
[SPARK-26134][CORE] Upgrading Hadoop to 2.7.4 to fix java.version problem
## What changes were proposed in this pull request?

When I ran spark-shell on JDK11+28(2018-09-25), It failed with the error below.

```
Exception in thread "main" java.lang.ExceptionInInitializerError
	at org.apache.hadoop.util.StringUtils.<clinit>(StringUtils.java:80)
	at org.apache.hadoop.security.SecurityUtil.getAuthenticationMethod(SecurityUtil.java:611)
	at org.apache.hadoop.security.UserGroupInformation.initialize(UserGroupInformation.java:273)
	at org.apache.hadoop.security.UserGroupInformation.ensureInitialized(UserGroupInformation.java:261)
	at org.apache.hadoop.security.UserGroupInformation.loginUserFromSubject(UserGroupInformation.java:791)
	at org.apache.hadoop.security.UserGroupInformation.getLoginUser(UserGroupInformation.java:761)
	at org.apache.hadoop.security.UserGroupInformation.getCurrentUser(UserGroupInformation.java:634)
	at org.apache.spark.util.Utils$.$anonfun$getCurrentUserName$1(Utils.scala:2427)
	at scala.Option.getOrElse(Option.scala:121)
	at org.apache.spark.util.Utils$.getCurrentUserName(Utils.scala:2427)
	at org.apache.spark.SecurityManager.<init>(SecurityManager.scala:79)
	at org.apache.spark.deploy.SparkSubmit.secMgr$lzycompute$1(SparkSubmit.scala:359)
	at org.apache.spark.deploy.SparkSubmit.secMgr$1(SparkSubmit.scala:359)
	at org.apache.spark.deploy.SparkSubmit.$anonfun$prepareSubmitEnvironment$9(SparkSubmit.scala:367)
	at scala.Option.map(Option.scala:146)
	at org.apache.spark.deploy.SparkSubmit.prepareSubmitEnvironment(SparkSubmit.scala:367)
	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:143)
	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:86)
	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:927)
	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:936)
	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
Caused by: java.lang.StringIndexOutOfBoundsException: begin 0, end 3, length 2
	at java.base/java.lang.String.checkBoundsBeginEnd(String.java:3319)
	at java.base/java.lang.String.substring(String.java:1874)
	at org.apache.hadoop.util.Shell.<clinit>(Shell.java:52)
```
This is a Hadoop issue that fails to parse some java.version. It has been fixed from Hadoop-2.7.4(see [HADOOP-14586](https://issues.apache.org/jira/browse/HADOOP-14586)).

Note, Hadoop-2.7.5 or upper have another problem with Spark ([SPARK-25330](https://issues.apache.org/jira/browse/SPARK-25330)). So upgrading to 2.7.4 would be fine for now.

## How was this patch tested?
Existing tests.

Closes #23101 from tasanuma/SPARK-26134.

Authored-by: Takanobu Asanuma <tasanuma@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-21 23:09:57 -08:00
Nagaram Prasad Addepally 9b48107f9c [SPARK-25957][K8S] Make building alternate language binding docker images optional
## What changes were proposed in this pull request?
bin/docker-image-tool.sh tries to build all docker images (JVM, PySpark
and SparkR) by default. But not all spark distributions are built with
SparkR and hence this script will fail on such distros.

With this change, we make building alternate language binding docker images (PySpark and SparkR) optional. User has to specify dockerfile for those language bindings using -p and -R flags accordingly, to build the binding docker images.

## How was this patch tested?

Tested following scenarios.
*bin/docker-image-tool.sh -r <repo> -t <tag> build* --> Builds only JVM docker image (default behavior)

*bin/docker-image-tool.sh -r <repo> -t <tag> -p kubernetes/dockerfiles/spark/bindings/python/Dockerfile build* --> Builds both JVM and PySpark docker images

*bin/docker-image-tool.sh -r <repo> -t <tag> -p kubernetes/dockerfiles/spark/bindings/python/Dockerfile -R kubernetes/dockerfiles/spark/bindings/R/Dockerfile build* --> Builds JVM, PySpark and SparkR docker images.

Author: Nagaram Prasad Addepally <ram@cloudera.com>

Closes #23053 from ramaddepally/SPARK-25957.
2018-11-21 15:51:37 -08:00
Sean Owen 32365f8177 [SPARK-26090][CORE][SQL][ML] Resolve most miscellaneous deprecation and build warnings for Spark 3
## What changes were proposed in this pull request?

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

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

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

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

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

## How was this patch tested?

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

Closes #23065 from srowen/SPARK-26090.

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

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

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

## How was this patch tested?

existing tests

Closes #22967 from dbtsai/scala2.12.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-14 16:22:23 -08:00
Marcelo Vanzin e4561e1c55 [SPARK-25897][K8S] Hook up k8s integration tests to sbt build.
The integration tests can now be run in sbt if the right profile
is enabled, using the "test" task under the respective project.

This avoids having to fall back to maven to run the tests, which
invalidates all your compiled stuff when you go back to sbt, making
development way slower than it should.

There's also a task to run the tests directly without refreshing
the docker images, which is helpful if you just made a change to
the submission code which should not affect the code in the images.

The sbt tasks currently are not very customizable; there's some
very minor things you can set in the sbt shell itself, but otherwise
it's hardcoded to run on minikube.

I also had to make some slight adjustments to the IT code itself,
mostly to remove assumptions about the existing harness.

Tested on sbt and maven.

Closes #22909 from vanzin/SPARK-25897.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-07 13:19:31 -08:00
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
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
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
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
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
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
Ilan Filonenko 1cfda44825 [SPARK-25021][K8S] Add spark.executor.pyspark.memory limit for K8S
## What changes were proposed in this pull request?

Add spark.executor.pyspark.memory limit for K8S

## How was this patch tested?

Unit and Integration tests

Closes #22298 from ifilonenko/SPARK-25021.

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

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

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

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

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

## How was this patch tested?

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

Author: Rob Vesse <rvesse@dotnetrdf.org>

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

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

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

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

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

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

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

## How was this patch tested?

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

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

Suggested reviewers: liyinan926 mccheah

Author: Rob Vesse <rvesse@dotnetrdf.org>

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

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

## How was this patch tested?

Modified unit test to test this change.

Author: Ilan Filonenko <if56@cornell.edu>

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

Introducing R Bindings for Spark R on K8s

- [x] Running SparkR Job

## How was this patch tested?

This patch was tested with

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

## Example:

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

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

Author: Ilan Filonenko <if56@cornell.edu>

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

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

## How was this patch tested?

Tested with unit and integration tests

Author: Ilan Filonenko <if56@cornell.edu>

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

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

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

## How was this patch tested?

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

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

Closes #21884 from adelbertc/k8s-expose-driver-ports.
2018-08-01 13:57:33 -07:00
mcheah 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
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
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
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
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
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 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
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
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
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
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
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
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
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
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
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
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
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
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