Skip to content

Commit 514ac19

Browse files
committed
documentation
1 parent 62354eb commit 514ac19

23 files changed

+164
-38
lines changed

docs/running-on-kubernetes.md

Lines changed: 47 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -768,6 +768,53 @@ from the other deployment modes. See the [configuration page](configuration.html
768768
<code>myIdentifier</code>. Multiple node selector keys can be added by setting multiple configurations with this prefix.
769769
</td>
770770
</tr>
771+
<tr>
772+
<td><code>spark.kubernetes.kerberos</code></td>
773+
<td>false</td>
774+
<td>
775+
Specify whether your job is a job that will require a Delegation Token to access HDFS. By default, we
776+
will assume that you will not require secure HDFS access.
777+
</td>
778+
</tr>
779+
<tr>
780+
<td><code>spark.kubernetes.kerberos.keytab</code></td>
781+
<td>(none)</td>
782+
<td>
783+
Assuming you have set <code>spark.kubernetes.kerberos</code> to be true. This will let you specify
784+
the location of your Kerberos keytab to be used in order to access Secure HDFS. This is optional as you
785+
may login by running <code>kinit -kt</code> before running the spark-submit, and the submission client
786+
will look within your local TGT cache to resolve this.
787+
</td>
788+
</tr>
789+
<tr>
790+
<td><code>spark.kubernetes.kerberos.principal</code></td>
791+
<td>(none)</td>
792+
<td>
793+
Assuming you have set <code>spark.kubernetes.kerberos</code> to be true. This will let you specify
794+
your Kerberos principal that you wish to use to access Secure HDFS. This is optional as you
795+
may login by running <code>kinit -kt</code> before running the spark-submit, and the submission client
796+
will look within your local TGT cache to resolve this.
797+
</td>
798+
</tr>
799+
<tr>
800+
<td><code>spark.kubernetes.kerberos.tokensecret.name</code></td>
801+
<td>(none)</td>
802+
<td>
803+
Assuming you have set <code>spark.kubernetes.kerberos</code> to be true. This will let you specify
804+
the name of the secret where your existing delegation token data is stored. You must also specify the
805+
label <code>spark.kubernetes.kerberos.tokensecret.name</code> where your data is stored on the secret.
806+
</td>
807+
</tr>
808+
<tr>
809+
<td><code>spark.kubernetes.kerberos.tokensecret.label</code></td>
810+
<td>spark.kubernetes.kerberos.dt.label</td>
811+
<td>
812+
Assuming you have set <code>spark.kubernetes.kerberos</code> to be true. This will let you specify
813+
the label within the pre-specified secret where the data of your existing delegation token data is stored.
814+
We have a default value of <code>spark.kubernetes.kerberos.dt.label</code> should you not include it. But
815+
you should always include this if you are proposing a pre-existing secret contain the delegation token data.
816+
</td>
817+
</tr>
771818
</table>
772819

773820

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopConfBootstrap.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,7 @@ import org.apache.spark.internal.Logging
2828

2929
/**
3030
* This is separated out from the HadoopConf steps API because this component can be reused to
31-
* set up the hadoop-conf for executors as well.
31+
* set up the Hadoop Configuration for executors as well.
3232
*/
3333
private[spark] trait HadoopConfBootstrap {
3434
/**

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosConfBootstrap.scala

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,12 +19,18 @@ package org.apache.spark.deploy.kubernetes
1919
import io.fabric8.kubernetes.api.model.ContainerBuilder
2020

2121
import org.apache.spark.deploy.kubernetes.constants._
22-
22+
/**
23+
* This is separated out from hadoopsteps because this component can be reused to
24+
* set up the Kerberos logic for executors as well.
25+
*/
2326
private[spark] trait KerberosConfBootstrap {
27+
/**
28+
* Bootstraps a main container with an ENV variable
29+
* pointing to the data storing the DT in the secret
30+
*/
2431
def bootstrapMainContainerAndVolumes(originalPodWithMainContainer: PodWithMainContainer)
2532
: PodWithMainContainer
2633
}
27-
2834
private[spark] class KerberosConfBootstrapImpl(
2935
delegationTokenLabelName: String) extends KerberosConfBootstrap{
3036
override def bootstrapMainContainerAndVolumes(

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithMainContainer.scala

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,11 @@ package org.apache.spark.deploy.kubernetes
1818

1919
import io.fabric8.kubernetes.api.model.{Container, Pod}
2020

21+
/**
22+
* The purpose of this case class is so that we can package together
23+
* the driver pod with its container so we can bootstrap and modify
24+
* the class instead of each component seperately
25+
*/
2126
private[spark] case class PodWithMainContainer(
2227
pod: Pod,
2328
mainContainer: Container)

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala

Lines changed: 12 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -510,6 +510,7 @@ package object config extends Logging {
510510
.createOptional
511511

512512
private[spark] val KUBERNETES_NODE_SELECTOR_PREFIX = "spark.kubernetes.node.selector."
513+
513514
private[spark] val KUBERNETES_KERBEROS_SUPPORT =
514515
ConfigBuilder("spark.kubernetes.kerberos")
515516
.doc("Specify whether your job is a job that will require a Delegation Token to access HDFS")
@@ -530,14 +531,22 @@ package object config extends Logging {
530531
.stringConf
531532
.createOptional
532533

533-
private[spark] val KUBERNETES_KERBEROS_DT_SECRET =
534-
ConfigBuilder("spark.kubernetes.kerberos.tokensecret")
535-
.doc("Specify the label of the secret where " +
534+
private[spark] val KUBERNETES_KERBEROS_DT_SECRET_NAME =
535+
ConfigBuilder("spark.kubernetes.kerberos.tokensecret.name")
536+
.doc("Specify the name of the secret where " +
536537
" your existing delegation token is stored. This removes the need" +
537538
" for the job user to provide any keytab for launching a job")
538539
.stringConf
539540
.createOptional
540541

542+
private[spark] val KUBERNETES_KERBEROS_DT_SECRET_LABEL =
543+
ConfigBuilder("spark.kubernetes.kerberos.tokensecret.label")
544+
.doc("Specify the label of the data where " +
545+
" your existing delegation token is stored. This removes the need" +
546+
" for the job user to provide any keytab for launching a job")
547+
.stringConf
548+
.createWithDefault("spark.kubernetes.kerberos.dt.label")
549+
541550
private[spark] def resolveK8sMaster(rawMasterString: String): String = {
542551
if (!rawMasterString.startsWith("k8s://")) {
543552
throw new IllegalArgumentException("Master URL should start with k8s:// in Kubernetes mode.")

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -99,12 +99,15 @@ package object constants {
9999
private[spark] val DEFAULT_SHUFFLE_MOUNT_NAME = "shuffle"
100100
private[spark] val INIT_CONTAINER_SECRET_VOLUME_NAME = "spark-init-secret"
101101

102+
// Hadoop Configuration
102103
private[spark] val HADOOP_FILE_VOLUME = "hadoop-properties"
103104
private[spark] val HADOOP_FILE_DIR = "/etc/hadoop"
104105
private[spark] val HADOOP_CONF_DIR = "HADOOP_CONF_DIR"
105106
private[spark] val HADOOP_CONF_DIR_LOC = "spark.kubernetes.hadoop.conf.dir"
106107
private[spark] val HADOOP_CONFIG_MAP_SPARK_CONF_NAME =
107108
"spark.kubernetes.hadoop.executor.hadoopconfigmapname"
109+
110+
// Kerberos Configuration
108111
private[spark] val HADOOP_KERBEROS_SECRET_NAME =
109112
"spark.kubernetes.kerberos.dt"
110113
private[spark] val KERBEROS_SPARK_CONF_NAME =

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -198,6 +198,8 @@ private[spark] object Client {
198198
def main(args: Array[String]): Unit = {
199199
val parsedArguments = ClientArguments.fromCommandLineArgs(args)
200200
val sparkConf = new SparkConf()
201+
// hadoopConfDir is passed into Client#run() to allow for us to
202+
// test this env variable within the integration test environment
201203
val hadoopConfDir = sys.env.get("HADOOP_CONF_DIR")
202204
run(sparkConf, parsedArguments, hadoopConfDir)
203205
}

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStep.scala

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,9 +23,11 @@ import io.fabric8.kubernetes.api.model.ConfigMapBuilder
2323
import org.apache.spark.deploy.kubernetes.constants._
2424
import org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps.{HadoopConfigSpec, HadoopConfigurationStep}
2525

26-
2726
/**
28-
* Configures the driverSpec that bootstraps dependencies into the driver pod.
27+
* This class configures the driverSpec with hadoop configuration logic which includes
28+
* volume mounts, config maps, and environment variable manipulation. The steps are
29+
* resolved with the orchestrator and they are run modifying the HadoopSpec with each
30+
* step. The final HadoopSpec's contents will be appended to the driverSpec.
2931
*/
3032
private[spark] class HadoopConfigBootstrapStep(
3133
hadoopConfigurationSteps: Seq[HadoopConfigurationStep],

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HDFSDelegationToken.scala

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -16,4 +16,8 @@
1616
*/
1717
package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps
1818

19+
/**
20+
* This case class contain the information that is important to be stored for
21+
* delegation token logic
22+
*/
1923
private[spark] case class HDFSDelegationToken(bytes: Array[Byte], renewal: Long)

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,11 @@ import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrap, PodWithMainConta
2424
import org.apache.spark.deploy.kubernetes.constants._
2525

2626
/**
27-
* Step that configures the ConfigMap + Volumes for the driver
27+
* This step is responsible for taking the contents from each file in
28+
* HADOOP_CONF_DIR, grabbing its contents as a string and storing each of them
29+
* as a key-value pair in a configmap. Each key-value pair will be stored
30+
* as a file, via Volume Mounts, later. The HADOOP_CONF_DIR_LOC is passed into the
31+
* SchedulerBackend via sparkConf.
2832
*/
2933
private[spark] class HadoopConfMounterStep(
3034
hadoopConfigMapName: String,

0 commit comments

Comments
 (0)