Repository: spark
Updated Branches:
  refs/heads/master 7beb375bf -> 7ab165b70


[SPARK-22648][K8S] Spark on Kubernetes - Documentation

What changes were proposed in this pull request?

This PR contains documentation on the usage of Kubernetes scheduler in Spark 
2.3, and a shell script to make it easier to build docker images required to 
use the integration. The changes detailed here are covered by 
https://github.com/apache/spark/pull/19717 and 
https://github.com/apache/spark/pull/19468 which have merged already.

How was this patch tested?
The script has been in use for releases on our fork. Rest is documentation.

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

TODO:
- [x] Add dockerfiles directory to built distribution. 
(https://github.com/apache/spark/pull/20007)
- [x] Change references to docker to instead say "container" 
(https://github.com/apache/spark/pull/19995)
- [x] Update configuration table.
- [x] Modify spark.kubernetes.allocation.batch.delay to take time instead of 
int (#20032)

Author: foxish <ramanath...@google.com>

Closes #19946 from foxish/update-k8s-docs.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7ab165b7
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7ab165b7
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7ab165b7

Branch: refs/heads/master
Commit: 7ab165b7061d9acc26523227076056e94354d204
Parents: 7beb375
Author: foxish <ramanath...@google.com>
Authored: Thu Dec 21 17:21:11 2017 -0800
Committer: Reynold Xin <r...@databricks.com>
Committed: Thu Dec 21 17:21:11 2017 -0800

----------------------------------------------------------------------
 docs/_layouts/global.html        |   1 +
 docs/building-spark.md           |   6 +-
 docs/cluster-overview.md         |   7 +-
 docs/configuration.md            |   2 +
 docs/img/k8s-cluster-mode.png    | Bin 0 -> 55538 bytes
 docs/index.md                    |   3 +-
 docs/running-on-kubernetes.md    | 578 ++++++++++++++++++++++++++++++++++
 docs/running-on-yarn.md          |   4 +-
 docs/submitting-applications.md  |  16 +
 sbin/build-push-docker-images.sh |  68 ++++
 10 files changed, 677 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/7ab165b7/docs/_layouts/global.html
----------------------------------------------------------------------
diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html
index 67b05ec..e5af5ae 100755
--- a/docs/_layouts/global.html
+++ b/docs/_layouts/global.html
@@ -99,6 +99,7 @@
                                 <li><a href="spark-standalone.html">Spark 
Standalone</a></li>
                                 <li><a 
href="running-on-mesos.html">Mesos</a></li>
                                 <li><a 
href="running-on-yarn.html">YARN</a></li>
+                                <li><a 
href="running-on-kubernetes.html">Kubernetes</a></li>
                             </ul>
                         </li>
 

http://git-wip-us.apache.org/repos/asf/spark/blob/7ab165b7/docs/building-spark.md
----------------------------------------------------------------------
diff --git a/docs/building-spark.md b/docs/building-spark.md
index 98f7df1..c391255 100644
--- a/docs/building-spark.md
+++ b/docs/building-spark.md
@@ -49,7 +49,7 @@ To create a Spark distribution like those distributed by the
 to be runnable, use `./dev/make-distribution.sh` in the project root 
directory. It can be configured
 with Maven profile settings and so on like the direct Maven build. Example:
 
-    ./dev/make-distribution.sh --name custom-spark --pip --r --tgz -Psparkr 
-Phadoop-2.7 -Phive -Phive-thriftserver -Pmesos -Pyarn
+    ./dev/make-distribution.sh --name custom-spark --pip --r --tgz -Psparkr 
-Phadoop-2.7 -Phive -Phive-thriftserver -Pmesos -Pyarn -Pkubernetes
 
 This will build Spark distribution along with Python pip and R packages. For 
more information on usage, run `./dev/make-distribution.sh --help`
 
@@ -90,6 +90,10 @@ like ZooKeeper and Hadoop itself.
 ## Building with Mesos support
 
     ./build/mvn -Pmesos -DskipTests clean package
+
+## Building with Kubernetes support
+
+    ./build/mvn -Pkubernetes -DskipTests clean package
     
 ## Building with Kafka 0.8 support
 

http://git-wip-us.apache.org/repos/asf/spark/blob/7ab165b7/docs/cluster-overview.md
----------------------------------------------------------------------
diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md
index c42bb4b..658e67f 100644
--- a/docs/cluster-overview.md
+++ b/docs/cluster-overview.md
@@ -52,11 +52,8 @@ The system currently supports three cluster managers:
 * [Apache Mesos](running-on-mesos.html) -- a general cluster manager that can 
also run Hadoop MapReduce
   and service applications.
 * [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 2.
-* [Kubernetes (experimental)](https://github.com/apache-spark-on-k8s/spark) -- 
In addition to the above,
-there is experimental support for Kubernetes. Kubernetes is an open-source 
platform
-for providing container-centric infrastructure. Kubernetes support is being 
actively
-developed in an [apache-spark-on-k8s](https://github.com/apache-spark-on-k8s/) 
Github organization. 
-For documentation, refer to that project's README.
+* [Kubernetes](running-on-kubernetes.html) -- 
[Kubernetes](https://kubernetes.io/docs/concepts/overview/what-is-kubernetes/)
+is an open-source platform that provides container-centric infrastructure.
 
 A third-party project (not supported by the Spark project) exists to add 
support for
 [Nomad](https://github.com/hashicorp/nomad-spark) as a cluster manager.

http://git-wip-us.apache.org/repos/asf/spark/blob/7ab165b7/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index d70bac1..1189aea 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -2376,6 +2376,8 @@ can be found on the pages for each mode:
 
 #### [Mesos](running-on-mesos.html#configuration)
 
+#### [Kubernetes](running-on-kubernetes.html#configuration)
+
 #### [Standalone Mode](spark-standalone.html#cluster-launch-scripts)
 
 # Environment Variables

http://git-wip-us.apache.org/repos/asf/spark/blob/7ab165b7/docs/img/k8s-cluster-mode.png
----------------------------------------------------------------------
diff --git a/docs/img/k8s-cluster-mode.png b/docs/img/k8s-cluster-mode.png
new file mode 100644
index 0000000..12a6288
Binary files /dev/null and b/docs/img/k8s-cluster-mode.png differ

http://git-wip-us.apache.org/repos/asf/spark/blob/7ab165b7/docs/index.md
----------------------------------------------------------------------
diff --git a/docs/index.md b/docs/index.md
index b867c97..2f00941 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -81,6 +81,7 @@ options for deployment:
 * [Standalone Deploy Mode](spark-standalone.html): simplest way to deploy 
Spark on a private cluster
 * [Apache Mesos](running-on-mesos.html)
 * [Hadoop YARN](running-on-yarn.html)
+* [Kubernetes](running-on-kubernetes.html)
 
 # Where to Go from Here
 
@@ -112,7 +113,7 @@ options for deployment:
   * [Mesos](running-on-mesos.html): deploy a private cluster using
       [Apache Mesos](http://mesos.apache.org)
   * [YARN](running-on-yarn.html): deploy Spark on top of Hadoop NextGen (YARN)
-  * [Kubernetes (experimental)](https://github.com/apache-spark-on-k8s/spark): 
deploy Spark on top of Kubernetes
+  * [Kubernetes](running-on-kubernetes.html): deploy Spark on top of Kubernetes
 
 **Other Documents:**
 

http://git-wip-us.apache.org/repos/asf/spark/blob/7ab165b7/docs/running-on-kubernetes.md
----------------------------------------------------------------------
diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md
new file mode 100644
index 0000000..0048bd9
--- /dev/null
+++ b/docs/running-on-kubernetes.md
@@ -0,0 +1,578 @@
+---
+layout: global
+title: Running Spark on Kubernetes
+---
+* This will become a table of contents (this text will be scraped).
+{:toc}
+
+Spark can run on clusters managed by [Kubernetes](https://kubernetes.io). This 
feature makes use of native
+Kubernetes scheduler that has been added to Spark.
+
+# Prerequisites
+
+* A runnable distribution of Spark 2.3 or above.
+* A running Kubernetes cluster at version >= 1.6 with access configured to it 
using
+[kubectl](https://kubernetes.io/docs/user-guide/prereqs/).  If you do not 
already have a working Kubernetes cluster,
+you may setup a test cluster on your local machine using
+[minikube](https://kubernetes.io/docs/getting-started-guides/minikube/).
+  * We recommend using the latest release of minikube with the DNS addon 
enabled.
+* You must have appropriate permissions to list, create, edit and delete
+[pods](https://kubernetes.io/docs/user-guide/pods/) in your cluster. You can 
verify that you can list these resources
+by running `kubectl auth can-i <list|create|edit|delete> pods`.
+  * The service account credentials used by the driver pods must be allowed to 
create pods, services and configmaps.
+* You must have [Kubernetes 
DNS](https://kubernetes.io/docs/concepts/services-networking/dns-pod-service/) 
configured in your cluster.
+
+# How it works
+
+<p style="text-align: center;">
+  <img src="img/k8s-cluster-mode.png" title="Spark cluster components" 
alt="Spark cluster components" />
+</p>
+
+<code>spark-submit</code> can be directly used to submit a Spark application 
to a Kubernetes cluster.
+The submission mechanism works as follows:
+
+* Spark creates a Spark driver running within a [Kubernetes 
pod](https://kubernetes.io/docs/concepts/workloads/pods/pod/).
+* The driver creates executors which are also running within Kubernetes pods 
and connects to them, and executes application code.
+* When the application completes, the executor pods terminate and are cleaned 
up, but the driver pod persists
+logs and remains in "completed" state in the Kubernetes API until it's 
eventually garbage collected or manually cleaned up.
+
+Note that in the completed state, the driver pod does *not* use any 
computational or memory resources.
+
+The driver and executor pod scheduling is handled by Kubernetes. It will be 
possible to affect Kubernetes scheduling
+decisions for driver and executor pods using advanced primitives like
+[node 
selectors](https://kubernetes.io/docs/concepts/configuration/assign-pod-node/#nodeselector)
+and [node/pod 
affinities](https://kubernetes.io/docs/concepts/configuration/assign-pod-node/#affinity-and-anti-affinity)
+in a future release.
+
+# Submitting Applications to Kubernetes
+
+## Docker Images
+
+Kubernetes requires users to supply images that can be deployed into 
containers within pods. The images are built to
+be run in a container runtime environment that Kubernetes supports. Docker is 
a container runtime environment that is
+frequently used with Kubernetes. With Spark 2.3, there are Dockerfiles 
provided in the runnable distribution that can be customized
+and built for your usage.
+
+You may build these docker images from sources.
+There is a script, `sbin/build-push-docker-images.sh` that you can use to 
build and push
+customized Spark distribution images consisting of all the above components.
+
+Example usage is:
+
+    ./sbin/build-push-docker-images.sh -r <repo> -t my-tag build
+    ./sbin/build-push-docker-images.sh -r <repo> -t my-tag push
+
+Docker files are under the `kubernetes/dockerfiles/` directory and can be 
customized further before
+building using the supplied script, or manually.
+
+## Cluster Mode
+
+To launch Spark Pi in cluster mode,
+
+{% highlight bash %}
+$ bin/spark-submit \
+    --master k8s://https://<k8s-apiserver-host>:<k8s-apiserver-port> \
+    --deploy-mode cluster \
+    --name spark-pi \
+    --class org.apache.spark.examples.SparkPi \
+    --conf spark.executor.instances=5 \
+    --conf spark.kubernetes.driver.docker.image=<driver-image> \
+    --conf spark.kubernetes.executor.docker.image=<executor-image> \
+    local:///path/to/examples.jar
+{% endhighlight %}
+
+The Spark master, specified either via passing the `--master` command line 
argument to `spark-submit` or by setting
+`spark.master` in the application's configuration, must be a URL with the 
format `k8s://<api_server_url>`. Prefixing the
+master string with `k8s://` will cause the Spark application to launch on the 
Kubernetes cluster, with the API server
+being contacted at `api_server_url`. If no HTTP protocol is specified in the 
URL, it defaults to `https`. For example,
+setting the master to `k8s://example.com:443` is equivalent to setting it to 
`k8s://https://example.com:443`, but to
+connect without TLS on a different port, the master would be set to 
`k8s://http://example.com:8080`.
+
+In Kubernetes mode, the Spark application name that is specified by 
`spark.app.name` or the `--name` argument to
+`spark-submit` is used by default to name the Kubernetes resources created 
like drivers and executors. So, application names
+must consist of lower case alphanumeric characters, `-`, and `.`  and must 
start and end with an alphanumeric character.
+
+If you have a Kubernetes cluster setup, one way to discover the apiserver URL 
is by executing `kubectl cluster-info`.
+
+```bash
+kubectl cluster-info
+Kubernetes master is running at http://127.0.0.1:6443
+```
+
+In the above example, the specific Kubernetes cluster can be used with 
<code>spark-submit</code> by specifying
+`--master k8s://http://127.0.0.1:6443` as an argument to spark-submit. 
Additionally, it is also possible to use the
+authenticating proxy, `kubectl proxy` to communicate to the Kubernetes API.
+
+The local proxy can be started by:
+
+```bash
+kubectl proxy
+```
+
+If the local proxy is running at localhost:8001, `--master 
k8s://http://127.0.0.1:8001` can be used as the argument to
+spark-submit. Finally, notice that in the above example we specify a jar with 
a specific URI with a scheme of `local://`.
+This URI is the location of the example jar that is already in the Docker 
image.
+
+## Dependency Management
+
+If your application's dependencies are all hosted in remote locations like 
HDFS or HTTP servers, they may be referred to
+by their appropriate remote URIs. Also, application dependencies can be 
pre-mounted into custom-built Docker images.
+Those dependencies can be added to the classpath by referencing them with 
`local://` URIs and/or setting the
+`SPARK_EXTRA_CLASSPATH` environment variable in your Dockerfiles.
+
+## Introspection and Debugging
+
+These are the different ways in which you can investigate a running/completed 
Spark application, monitor progress, and
+take actions.
+
+### Accessing Logs
+
+Logs can be accessed using the Kubernetes API and the `kubectl` CLI. When a 
Spark application is running, it's possible
+to stream logs from the application using:
+
+```bash
+kubectl -n=<namespace> logs -f <driver-pod-name>
+```
+
+The same logs can also be accessed through the
+[Kubernetes 
dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/)
 if installed on
+the cluster.
+
+### Accessing Driver UI
+
+The UI associated with any application can be accessed locally using
+[`kubectl 
port-forward`](https://kubernetes.io/docs/tasks/access-application-cluster/port-forward-access-application-cluster/#forward-a-local-port-to-a-port-on-the-pod).
+
+```bash
+kubectl port-forward <driver-pod-name> 4040:4040
+```
+
+Then, the Spark driver UI can be accessed on `http://localhost:4040`.
+
+### Debugging 
+
+There may be several kinds of failures. If the Kubernetes API server rejects 
the request made from spark-submit, or the
+connection is refused for a different reason, the submission logic should 
indicate the error encountered. However, if there
+are errors during the running of the application, often, the best way to 
investigate may be through the Kubernetes CLI.
+
+To get some basic information about the scheduling decisions made around the 
driver pod, you can run:
+
+```bash
+kubectl describe pod <spark-driver-pod>
+```
+
+If the pod has encountered a runtime error, the status can be probed further 
using:
+
+```bash
+kubectl logs <spark-driver-pod>
+```
+
+Status and logs of failed executor pods can be checked in similar ways. 
Finally, deleting the driver pod will clean up the entire spark 
+application, includling all executors, associated service, etc. The driver pod 
can be thought of as the Kubernetes representation of 
+the Spark application.
+
+## Kubernetes Features
+
+### Namespaces
+
+Kubernetes has the concept of 
[namespaces](https://kubernetes.io/docs/concepts/overview/working-with-objects/namespaces/).
+Namespaces are ways to divide cluster resources between multiple users (via 
resource quota). Spark on Kubernetes can
+use namespaces to launch Spark applications. This can be made use of through 
the `spark.kubernetes.namespace` configuration.
+
+Kubernetes allows using 
[ResourceQuota](https://kubernetes.io/docs/concepts/policy/resource-quotas/) to 
set limits on
+resources, number of objects, etc on individual namespaces. Namespaces and 
ResourceQuota can be used in combination by
+administrator to control sharing and resource allocation in a Kubernetes 
cluster running Spark applications.
+
+### RBAC
+
+In Kubernetes clusters with 
[RBAC](https://kubernetes.io/docs/admin/authorization/rbac/) enabled, users can 
configure
+Kubernetes RBAC roles and service accounts used by the various Spark on 
Kubernetes components to access the Kubernetes
+API server.
+
+The Spark driver pod uses a Kubernetes service account to access the 
Kubernetes API server to create and watch executor
+pods. The service account used by the driver pod must have the appropriate 
permission for the driver to be able to do
+its work. Specifically, at minimum, the service account must be granted a
+[`Role` or 
`ClusterRole`](https://kubernetes.io/docs/admin/authorization/rbac/#role-and-clusterrole)
 that allows driver
+pods to create pods and services. By default, the driver pod is automatically 
assigned the `default` service account in
+the namespace specified by `spark.kubernetes.namespace`, if no service account 
is specified when the pod gets created.
+
+Depending on the version and setup of Kubernetes deployed, this `default` 
service account may or may not have the role
+that allows driver pods to create pods and services under the default 
Kubernetes
+[RBAC](https://kubernetes.io/docs/admin/authorization/rbac/) policies. 
Sometimes users may need to specify a custom
+service account that has the right role granted. Spark on Kubernetes supports 
specifying a custom service account to
+be used by the driver pod through the configuration property
+`spark.kubernetes.authenticate.driver.serviceAccountName=<service account 
name>`. For example to make the driver pod
+use the `spark` service account, a user simply adds the following option to 
the `spark-submit` command:
+
+```
+--conf spark.kubernetes.authenticate.driver.serviceAccountName=spark
+```
+
+To create a custom service account, a user can use the `kubectl create 
serviceaccount` command. For example, the
+following command creates a service account named `spark`:
+
+```bash
+kubectl create serviceaccount spark
+```
+
+To grant a service account a `Role` or `ClusterRole`, a `RoleBinding` or 
`ClusterRoleBinding` is needed. To create
+a `RoleBinding` or `ClusterRoleBinding`, a user can use the `kubectl create 
rolebinding` (or `clusterrolebinding`
+for `ClusterRoleBinding`) command. For example, the following command creates 
an `edit` `ClusterRole` in the `default`
+namespace and grants it to the `spark` service account created above:
+
+```bash
+kubectl create clusterrolebinding spark-role --clusterrole=edit 
--serviceaccount=default:spark --namespace=default
+```
+
+Note that a `Role` can only be used to grant access to resources (like pods) 
within a single namespace, whereas a
+`ClusterRole` can be used to grant access to cluster-scoped resources (like 
nodes) as well as namespaced resources
+(like pods) across all namespaces. For Spark on Kubernetes, since the driver 
always creates executor pods in the
+same namespace, a `Role` is sufficient, although users may use a `ClusterRole` 
instead. For more information on
+RBAC authorization and how to configure Kubernetes service accounts for pods, 
please refer to
+[Using RBAC 
Authorization](https://kubernetes.io/docs/admin/authorization/rbac/) and
+[Configure Service Accounts for 
Pods](https://kubernetes.io/docs/tasks/configure-pod-container/configure-service-account/).
+
+## Client Mode
+
+Client mode is not currently supported.
+
+## Future Work
+
+There are several Spark on Kubernetes features that are currently being 
incubated in a fork -
+[apache-spark-on-k8s/spark](https://github.com/apache-spark-on-k8s/spark), 
which are expected to eventually make it into
+future versions of the spark-kubernetes integration.
+
+Some of these include:
+
+* PySpark
+* R
+* Dynamic Executor Scaling
+* Local File Dependency Management
+* Spark Application Management
+* Job Queues and Resource Management
+
+You can refer to the 
[documentation](https://apache-spark-on-k8s.github.io/userdocs/) if you want to 
try these features
+and provide feedback to the development team.
+
+# Configuration
+
+See the [configuration page](configuration.html) for information on Spark 
configurations.  The following configurations are
+specific to Spark on Kubernetes.
+
+#### Spark Properties
+
+<table class="table">
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+<tr>
+  <td><code>spark.kubernetes.namespace</code></td>
+  <td><code>default</code></td>
+  <td>
+    The namespace that will be used for running the driver and executor pods.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.driver.container.image</code></td>
+  <td><code>(none)</code></td>
+  <td>
+    Container image to use for the driver.
+    This is usually of the form `example.com/repo/spark-driver:v1.0.0`.
+    This configuration is required and must be provided by the user.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.executor.container.image</code></td>
+  <td><code>(none)</code></td>
+  <td>
+    Container image to use for the executors.
+    This is usually of the form `example.com/repo/spark-executor:v1.0.0`.
+    This configuration is required and must be provided by the user.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.container.image.pullPolicy</code></td>
+  <td><code>IfNotPresent</code></td>
+  <td>
+    Container image pull policy used when pulling images within Kubernetes.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.allocation.batch.size</code></td>
+  <td><code>5</code></td>
+  <td>
+    Number of pods to launch at once in each round of executor pod allocation.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.allocation.batch.delay</code></td>
+  <td><code>1s</code></td>
+  <td>
+    Time to wait between each round of executor pod allocation. Specifying 
values less than 1 second may lead to
+    excessive CPU usage on the spark driver.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.authenticate.submission.caCertFile</code></td>
+  <td>(none)</td>
+  <td>
+    Path to the CA cert file for connecting to the Kubernetes API server over 
TLS when starting the driver. This file
+    must be located on the submitting machine's disk. Specify this as a path 
as opposed to a URI (i.e. do not provide
+    a scheme).
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.authenticate.submission.clientKeyFile</code></td>
+  <td>(none)</td>
+  <td>
+    Path to the client key file for authenticating against the Kubernetes API 
server when starting the driver. This file
+    must be located on the submitting machine's disk. Specify this as a path 
as opposed to a URI (i.e. do not provide
+    a scheme).
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.authenticate.submission.clientCertFile</code></td>
+  <td>(none)</td>
+  <td>
+    Path to the client cert file for authenticating against the Kubernetes API 
server when starting the driver. This
+    file must be located on the submitting machine's disk. Specify this as a 
path as opposed to a URI (i.e. do not
+    provide a scheme).
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.authenticate.submission.oauthToken</code></td>
+  <td>(none)</td>
+  <td>
+    OAuth token to use when authenticating against the Kubernetes API server 
when starting the driver. Note
+    that unlike the other authentication options, this is expected to be the 
exact string value of the token to use for
+    the authentication.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.authenticate.submission.oauthTokenFile</code></td>
+  <td>(none)</td>
+  <td>
+    Path to the OAuth token file containing the token to use when 
authenticating against the Kubernetes API server when starting the driver.
+    This file must be located on the submitting machine's disk. Specify this 
as a path as opposed to a URI (i.e. do not
+    provide a scheme).
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.authenticate.driver.caCertFile</code></td>
+  <td>(none)</td>
+  <td>
+    Path to the CA cert file for connecting to the Kubernetes API server over 
TLS from the driver pod when requesting
+    executors. This file must be located on the submitting machine's disk, and 
will be uploaded to the driver pod.
+    Specify this as a path as opposed to a URI (i.e. do not provide a scheme).
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.authenticate.driver.clientKeyFile</code></td>
+  <td>(none)</td>
+  <td>
+    Path to the client key file for authenticating against the Kubernetes API 
server from the driver pod when requesting
+    executors. This file must be located on the submitting machine's disk, and 
will be uploaded to the driver pod.
+    Specify this as a path as opposed to a URI (i.e. do not provide a scheme). 
If this is specified, it is highly
+    recommended to set up TLS for the driver submission server, as this value 
is sensitive information that would be
+    passed to the driver pod in plaintext otherwise.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.authenticate.driver.clientCertFile</code></td>
+  <td>(none)</td>
+  <td>
+    Path to the client cert file for authenticating against the Kubernetes API 
server from the driver pod when
+    requesting executors. This file must be located on the submitting 
machine's disk, and will be uploaded to the
+    driver pod. Specify this as a path as opposed to a URI (i.e. do not 
provide a scheme).
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.authenticate.driver.oauthToken</code></td>
+  <td>(none)</td>
+  <td>
+    OAuth token to use when authenticating against the Kubernetes API server 
from the driver pod when
+    requesting executors. Note that unlike the other authentication options, 
this must be the exact string value of
+    the token to use for the authentication. This token value is uploaded to 
the driver pod. If this is specified, it is
+    highly recommended to set up TLS for the driver submission server, as this 
value is sensitive information that would
+    be passed to the driver pod in plaintext otherwise.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.authenticate.driver.oauthTokenFile</code></td>
+  <td>(none)</td>
+  <td>
+    Path to the OAuth token file containing the token to use when 
authenticating against the Kubernetes API server from the driver pod when
+    requesting executors. Note that unlike the other authentication options, 
this file must contain the exact string value of
+    the token to use for the authentication. This token value is uploaded to 
the driver pod. If this is specified, it is
+    highly recommended to set up TLS for the driver submission server, as this 
value is sensitive information that would
+    be passed to the driver pod in plaintext otherwise.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.authenticate.driver.mounted.caCertFile</code></td>
+  <td>(none)</td>
+  <td>
+    Path to the CA cert file for connecting to the Kubernetes API server over 
TLS from the driver pod when requesting
+    executors. This path must be accessible from the driver pod.
+    Specify this as a path as opposed to a URI (i.e. do not provide a scheme).
+  </td>
+</tr>
+<tr>
+  
<td><code>spark.kubernetes.authenticate.driver.mounted.clientKeyFile</code></td>
+  <td>(none)</td>
+  <td>
+    Path to the client key file for authenticating against the Kubernetes API 
server from the driver pod when requesting
+    executors. This path must be accessible from the driver pod.
+    Specify this as a path as opposed to a URI (i.e. do not provide a scheme).
+  </td>
+</tr>
+<tr>
+  
<td><code>spark.kubernetes.authenticate.driver.mounted.clientCertFile</code></td>
+  <td>(none)</td>
+  <td>
+    Path to the client cert file for authenticating against the Kubernetes API 
server from the driver pod when
+    requesting executors. This path must be accessible from the driver pod.
+    Specify this as a path as opposed to a URI (i.e. do not provide a scheme).
+  </td>
+</tr>
+<tr>
+  
<td><code>spark.kubernetes.authenticate.driver.mounted.oauthTokenFile</code></td>
+  <td>(none)</td>
+  <td>
+    Path to the file containing the OAuth token to use when authenticating 
against the Kubernetes API server from the driver pod when
+    requesting executors. This path must be accessible from the driver pod.
+    Note that unlike the other authentication options, this file must contain 
the exact string value of the token to use for the authentication.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.authenticate.driver.serviceAccountName</code></td>
+  <td><code>default</code></td>
+  <td>
+    Service account that is used when running the driver pod. The driver pod 
uses this service account when requesting
+    executor pods from the API server. Note that this cannot be specified 
alongside a CA cert file, client key file,
+    client cert file, and/or OAuth token.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.driver.label.[LabelName]</code></td>
+  <td>(none)</td>
+  <td>
+    Add the label specified by <code>LabelName</code> to the driver pod.
+    For example, <code>spark.kubernetes.driver.label.something=true</code>.
+    Note that Spark also adds its own labels to the driver pod
+    for bookkeeping purposes.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.driver.annotation.[AnnotationName]</code></td>
+  <td>(none)</td>
+  <td>
+    Add the annotation specified by <code>AnnotationName</code> to the driver 
pod.
+    For example, 
<code>spark.kubernetes.driver.annotation.something=true</code>.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.executor.label.[LabelName]</code></td>
+  <td>(none)</td>
+  <td>
+    Add the label specified by <code>LabelName</code> to the executor pods.
+    For example, <code>spark.kubernetes.executor.label.something=true</code>.
+    Note that Spark also adds its own labels to the driver pod
+    for bookkeeping purposes.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.executor.annotation.[AnnotationName]</code></td>
+  <td>(none)</td>
+  <td>
+    Add the annotation specified by <code>AnnotationName</code> to the 
executor pods.
+    For example, 
<code>spark.kubernetes.executor.annotation.something=true</code>.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.driver.pod.name</code></td>
+  <td>(none)</td>
+  <td>
+    Name of the driver pod. If not set, the driver pod name is set to 
"spark.app.name" suffixed by the current timestamp
+    to avoid name conflicts.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.executor.podNamePrefix</code></td>
+  <td>(none)</td>
+  <td>
+    Prefix for naming the executor pods.
+    If not set, the executor pod name is set to driver pod name suffixed by an 
integer.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.executor.lostCheck.maxAttempts</code></td>
+  <td><code>10</code></td>
+  <td>
+    Number of times that the driver will try to ascertain the loss reason for 
a specific executor.
+    The loss reason is used to ascertain whether the executor failure is due 
to a framework or an application error
+    which in turn decides whether the executor is removed and replaced, or 
placed into a failed state for debugging.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.submission.waitAppCompletion</code></td>
+  <td><code>true</code></td>
+  <td>
+    In cluster mode, whether to wait for the application to finish before 
exiting the launcher process.  When changed to
+    false, the launcher has a "fire-and-forget" behavior when launching the 
Spark job.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.kubernetes.report.interval</code></td>
+  <td><code>1s</code></td>
+  <td>
+    Interval between reports of the current Spark job status in cluster mode.
+  </td>
+</tr>
+<tr>
+   <td><code>spark.kubernetes.driver.limit.cores</code></td>
+   <td>(none)</td>
+   <td>
+     Specify the hard CPU 
[limit](https://kubernetes.io/docs/concepts/configuration/manage-compute-resources-container/#resource-requests-and-limits-of-pod-and-container)
 for the driver pod.
+   </td>
+ </tr>
+ <tr>
+   <td><code>spark.kubernetes.executor.limit.cores</code></td>
+   <td>(none)</td>
+   <td>
+     Specify the hard CPU 
[limit](https://kubernetes.io/docs/concepts/configuration/manage-compute-resources-container/#resource-requests-and-limits-of-pod-and-container)
 for each executor pod launched for the Spark Application.
+   </td>
+ </tr>
+ <tr>
+   <td><code>spark.kubernetes.node.selector.[labelKey]</code></td>
+   <td>(none)</td>
+   <td>
+     Adds to the node selector of the driver pod and executor pods, with key 
<code>labelKey</code> and the value as the
+     configuration's value. For example, setting 
<code>spark.kubernetes.node.selector.identifier</code> to 
<code>myIdentifier</code>
+     will result in the driver pod and executors having a node selector with 
key <code>identifier</code> and value
+      <code>myIdentifier</code>. Multiple node selector keys can be added by 
setting multiple configurations with this prefix.
+    </td>
+  </tr>
+ <tr>
+   <td><code>spark.kubernetes.driverEnv.[EnvironmentVariableName]</code></td>
+   <td>(none)</td>
+   <td>
+     Add the environment variable specified by 
<code>EnvironmentVariableName</code> to
+     the Driver process. The user can specify multiple of these to set 
multiple environment variables.
+   </td>
+ </tr>
+  <tr>
+    <td><code>spark.kubernetes.mountDependencies.jarsDownloadDir</code></td>
+    <td><code>/var/spark-data/spark-jars</code></td>
+    <td>
+      Location to download jars to in the driver and executors.
+      This directory must be empty and will be mounted as an empty directory 
volume on the driver and executor pods.
+    </td>
+  </tr>
+   <tr>
+     <td><code>spark.kubernetes.mountDependencies.filesDownloadDir</code></td>
+     <td><code>/var/spark-data/spark-files</code></td>
+     <td>
+       Location to download jars to in the driver and executors.
+       This directory must be empty and will be mounted as an empty directory 
volume on the driver and executor pods.
+     </td>
+   </tr>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/7ab165b7/docs/running-on-yarn.md
----------------------------------------------------------------------
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index 7e2386f..e7edec5 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -18,7 +18,9 @@ Spark application's configuration (driver, executors, and the 
AM when running in
 
 There are two deploy modes that can be used to launch Spark applications on 
YARN. In `cluster` mode, the Spark driver runs inside an application master 
process which is managed by YARN on the cluster, and the client can go away 
after initiating the application. In `client` mode, the driver runs in the 
client process, and the application master is only used for requesting 
resources from YARN.
 
-Unlike [Spark standalone](spark-standalone.html) and 
[Mesos](running-on-mesos.html) modes, in which the master's address is 
specified in the `--master` parameter, in YARN mode the ResourceManager's 
address is picked up from the Hadoop configuration. Thus, the `--master` 
parameter is `yarn`.
+Unlike other cluster managers supported by Spark in which the master's address 
is specified in the `--master`
+parameter, in YARN mode the ResourceManager's address is picked up from the 
Hadoop configuration.
+Thus, the `--master` parameter is `yarn`.
 
 To launch a Spark application in `cluster` mode:
 

http://git-wip-us.apache.org/repos/asf/spark/blob/7ab165b7/docs/submitting-applications.md
----------------------------------------------------------------------
diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md
index 866d6e5..0473ab7 100644
--- a/docs/submitting-applications.md
+++ b/docs/submitting-applications.md
@@ -127,6 +127,16 @@ export HADOOP_CONF_DIR=XXX
   http://path/to/examples.jar \
   1000
 
+# Run on a Kubernetes cluster in cluster deploy mode
+./bin/spark-submit \
+  --class org.apache.spark.examples.SparkPi \
+  --master k8s://xx.yy.zz.ww:443 \
+  --deploy-mode cluster \
+  --executor-memory 20G \
+  --num-executors 50 \
+  http://path/to/examples.jar \
+  1000
+
 {% endhighlight %}
 
 # Master URLs
@@ -155,6 +165,12 @@ The master URL passed to Spark can be in one of the 
following formats:
         <code>client</code> or <code>cluster</code> mode depending on the 
value of <code>--deploy-mode</code>.
         The cluster location will be found based on the 
<code>HADOOP_CONF_DIR</code> or <code>YARN_CONF_DIR</code> variable.
 </td></tr>
+<tr><td> <code>k8s://HOST:PORT</code> </td><td> Connect to a <a 
href="running-on-kubernetes.html">Kubernetes</a> cluster in
+        <code>cluster</code> mode. Client mode is currently unsupported and 
will be supported in future releases.
+        The <code>HOST</code> and <code>PORT</code> refer to the [Kubernetes 
API Server](https://kubernetes.io/docs/reference/generated/kube-apiserver/).
+        It connects using TLS by default. In order to force it to use an 
unsecured connection, you can use
+        <code>k8s://http://HOST:PORT</code>.
+</td></tr>
 </table>
 
 

http://git-wip-us.apache.org/repos/asf/spark/blob/7ab165b7/sbin/build-push-docker-images.sh
----------------------------------------------------------------------
diff --git a/sbin/build-push-docker-images.sh b/sbin/build-push-docker-images.sh
new file mode 100755
index 0000000..4546e98
--- /dev/null
+++ b/sbin/build-push-docker-images.sh
@@ -0,0 +1,68 @@
+#!/usr/bin/env bash
+
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# This script builds and pushes docker images when run from a release of Spark
+# with Kubernetes support.
+
+declare -A path=( [spark-driver]=kubernetes/dockerfiles/driver/Dockerfile \
+                  [spark-executor]=kubernetes/dockerfiles/executor/Dockerfile )
+
+function build {
+  docker build -t spark-base -f kubernetes/dockerfiles/spark-base/Dockerfile .
+  for image in "${!path[@]}"; do
+    docker build -t ${REPO}/$image:${TAG} -f ${path[$image]} .
+  done
+}
+
+
+function push {
+  for image in "${!path[@]}"; do
+    docker push ${REPO}/$image:${TAG}
+  done
+}
+
+function usage {
+  echo "This script must be run from a runnable distribution of Apache Spark."
+  echo "Usage: ./sbin/build-push-docker-images.sh -r <repo> -t <tag> build"
+  echo "       ./sbin/build-push-docker-images.sh -r <repo> -t <tag> push"
+  echo "for example: ./sbin/build-push-docker-images.sh -r docker.io/myrepo -t 
v2.3.0 push"
+}
+
+if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then
+  usage
+  exit 0
+fi
+
+while getopts r:t: option
+do
+ case "${option}"
+ in
+ r) REPO=${OPTARG};;
+ t) TAG=${OPTARG};;
+ esac
+done
+
+if [ -z "$REPO" ] || [ -z "$TAG" ]; then
+    usage
+else
+  case "${@: -1}" in
+    build) build;;
+    push) push;;
+    *) usage;;
+  esac
+fi


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to