Skip to content

Conversation

@ifilonenko
Copy link
Contributor

@ifilonenko ifilonenko commented Jun 29, 2018

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
This initiative can be broken down into 4 Stages

STAGE 1

  • 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

  • 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

  • Driver

STAGE 4

  • Executor

How was this patch tested?

Locally tested on a single-noded, pseudo-distributed Kerberized Hadoop Cluster

Docs and Error Handling?

  • Docs
  • Error Handling

Contribution Credit

@kimoonkim @skonto

@SparkQA
Copy link

SparkQA commented Jun 29, 2018

@SparkQA
Copy link

SparkQA commented Jun 29, 2018

Kubernetes integration test status success
URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/583/

@SparkQA
Copy link

SparkQA commented Jun 29, 2018

Test build #92471 has finished for PR 21669 at commit 73f2777.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@felixcheung
Copy link
Member

btw, have you sent out this + doc to dev@spark.apache.org?

@SparkQA
Copy link

SparkQA commented Jul 1, 2018

@SparkQA
Copy link

SparkQA commented Jul 1, 2018

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/609/

@SparkQA
Copy link

SparkQA commented Jul 1, 2018

Test build #92512 has finished for PR 21669 at commit 6069be5.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jul 2, 2018

@SparkQA
Copy link

SparkQA commented Jul 2, 2018

Kubernetes integration test status failure
URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/613/

@SparkQA
Copy link

SparkQA commented Jul 2, 2018

Test build #92520 has finished for PR 21669 at commit 000120f.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.


// assure a keytab is available from any place in a JVM
if (clusterManager == YARN || clusterManager == LOCAL || isMesosClient) {
if (clusterManager == YARN || clusterManager == LOCAL || isMesosClient || isKubernetesCluster) {
Copy link
Contributor

@skonto skonto Jul 3, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This check has been restrictive for customers in the past. There are cases where spark submit should not have the file locally and keytab should be mounted as a secret within the cluster for example.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This check can be removed, but I included it since I believed that the keytab shouldn't be stored as a secret for security reasons and should instead be only accessible from the JVM.


val KUBERNETES_KERBEROS_SUPPORT =
ConfigBuilder("spark.kubernetes.kerberos.enabled")
.doc("Specify whether your job is a job that will require a Delegation Token to access HDFS")
Copy link
Contributor

@skonto skonto Jul 3, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think kerberos goes beyond DTs so it shouldnt be specific to that. Also I think you dont need the user to pass that. You just need to call: UserGroupInformation.isSecurityEnabled instead of getting that property from config.

SPARK_HADOOP_PREFIX + "hadoop.security.authentication"

// Kerberos Token-Refresh Server
val KERBEROS_REFRESH_LABEL_KEY = "refresh-hadoop-tokens"
Copy link
Contributor

@skonto skonto Jul 3, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I left a comment in the design doc. Can we also provide the option for using an existing renewal service like when integrating with an external hadoop cluster where people already have that. AFAIK hadoop libs do the renewal behind the scenes by talking to the appropriate service.
This is the current implementation with mesos for integrating with existing clusters.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Because our original architecture had the opinion that the renewal service pod will exist as a separate micro-service, that option could be handled by that renewal service. We used this label to detect that this specific secret was to be renewed. But if we wished to use another renewal service via some existing service, we might be able to just grab an Array[Byte] from some DTManager that may exist in their external Hadoop clusters, and store in a secret. Thank you for this note in the design doc.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It would be good to have that option.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agreed but that would be out of the scope of this PR as the renewal service is a separate micro-service (instead of a running thread), and thereby that logic would be housed in a separate PR governing the opinion of the renewal service pods "DT retrieving" protocol

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this label actually being used for anything?

}
}

def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = {
Copy link
Contributor

@skonto skonto Jul 3, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

requireNandDefined is spelling ok? Also that function is not used, should be removed or will be used later when PR will be updated?

val ENV_CLASSPATH = "SPARK_CLASSPATH"
val ENV_DRIVER_BIND_ADDRESS = "SPARK_DRIVER_BIND_ADDRESS"
val ENV_SPARK_CONF_DIR = "SPARK_CONF_DIR"
val ENV_SPARK_USER = "SPARK_USER"
Copy link
Contributor

@skonto skonto Jul 3, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I guess this is for setting the correct user. But I think hadoop libs should pick the correct user like in SparkContext where Utils.getCurrentUserName() is used. In addition I think we should allow the pods to run with any username (eg. with customized images) and we should have a security context per container like: SecurityContextBuilder().withRunAsUser(). In the latter scenario hadoop libraries will pick up that logged in user as well. A common scenario would be integrating the container with host PAM: jupyterhub/jupyterhub#535 https://medium.com/@pawitp/syncing-host-and-container-users-in-docker-39337eff0094 or with LDAP etc.
Btw if you change the user with which the container user needs also to exist on the image when basic authorization is used (unix groups).

@SparkQA
Copy link

SparkQA commented Jul 7, 2018

@SparkQA
Copy link

SparkQA commented Jul 7, 2018

Kubernetes integration test status success
URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/743/

@SparkQA
Copy link

SparkQA commented Jul 7, 2018

@SparkQA
Copy link

SparkQA commented Jul 7, 2018

Kubernetes integration test status success
URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/746/

@SparkQA
Copy link

SparkQA commented Jul 7, 2018

Test build #92705 has finished for PR 21669 at commit 13b3adc.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jul 18, 2018

Test build #93211 has finished for PR 21669 at commit 13b3adc.

  • This patch fails Spark unit tests.
  • This patch does not merge cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jul 30, 2018

@SparkQA
Copy link

SparkQA commented Jul 30, 2018

Kubernetes integration test status success
URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/1501/

@SparkQA
Copy link

SparkQA commented Jul 31, 2018

Test build #93807 has finished for PR 21669 at commit 0939738.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 9, 2018

@SparkQA
Copy link

SparkQA commented Oct 9, 2018

Kubernetes integration test status success
URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/3833/

val initialTokenDataKeyName = s"$KERBEROS_SECRET_KEY_PREFIX-$currentTime"
val newSecretName =
s"$kubernetesResourceNamePrefix-$KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME.$currentTime"
val initialTokenDataKeyName = KERBEROS_SECRET_KEY_PREFIX
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: you don't need this variable and you can rename KERBEROS_SECRET_KEY_PREFIX to KERBEROS_SECRET_KEY.

@ifilonenko
Copy link
Contributor Author

thank you @liyinan926 for your review! @vanzin, ill leave it to you to give the final LGTM

Copy link
Contributor

@vanzin vanzin left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Man, it's really tiring to go through the code and keep seeing the same style issues I've pointed out many times already. Sorry if I sound frustrated, but there's so much of that I can do before I start to give up on the review.

Please go through your code and fix all of those.

</td>
</tr>
<tr>
<td><code>spark.kubernetes.kerberos.krb5.location</code></td>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Kinda let for this one, but since this is a local path, I think krb5.path makes more sense. Or maybe krb5.confFile to contrast with the config map alternative.

<td><code>spark.kubernetes.kerberos.krb5.configMapName</code></td>
<td><code>(none)</code></td>
<td>
Specify the name of the ConfigMap, containing the krb5 file, to be mounted on the driver and executors
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

"krb5.conf file"

<td><code>spark.kubernetes.kerberos.tokenSecret.name</code></td>
<td><code>(none)</code></td>
<td>
Specify the name of the secret where your existing delegation token is stored. This removes the need for the job user
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

"tokens are"

<td><code>spark.kubernetes.kerberos.tokenSecret.itemKey</code></td>
<td><code>(none)</code></td>
<td>
Specify the item key of the data where your existing delegation token is stored. This removes the need for the job user
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

"tokens are"

docs/security.md Outdated
`spark.kubernetes.hadoop.configMapName` as well as either
`spark.kubernetes.kerberos.krb5.location` or `spark.kubernetes.kerberos.krb5.configMapName`.

It also important to note that the KDC needs to be visible from inside the containers if the user uses a local
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not just "if the user uses a local krb5 file".

hFiles <- hadoopConfigurationFiles
} yield {
HadoopBootstrapUtil.buildHadoopConfigMap(
hName,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

all args fit in the original line

}

// Kerberos DT Secret
val kerberosDTSecret = for {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

really, please: kerberosConfSpec.flatMap(_.dtSecret).

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To my knowledge and after a quick test in the shell, you can't use flatMap as this would cause a: dtSecret is not a member of Nothing error for one of the input cases:
i.e. None.flatMap(_.dtSecret) is the cause of the failure.
:( I understand that it isn't preferred but, I think, this might be the cleanest way to handle the cases:

  • None,
  • Some(KerberosConfSpec(dtSecret: None,...)),
  • Some(KerberosConfSpec(dtSecret: Some(Secret),...)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the type of kerberosConfSpec is known, no? Then it should work.

scala> case class Foo(bar: Option[String])
defined class Foo

scala> 

scala> Some(Foo(Some("bar"))).flatMap(_.bar)
res1: Option[String] = Some(bar)

scala> Some(Foo(None)).flatMap(_.bar)
res2: Option[String] = None

scala> Option[Foo](null).flatMap(_.bar)
res3: Option[String] = None

A raw "None" would not work, but you don't have a raw "None" here; you have a val kerberosConfSpec, of type Option[KerberosConfigSpec], and value None, so you have all the type information you need.

* This step is responsible for mounting the DT secret for the executors
*/
private[spark] class KerberosConfExecutorFeatureStep(
kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf])
Copy link
Contributor

@vanzin vanzin Oct 9, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

arg indentation. this is another class that is all off with 3-space indentation.

*/
private[spark] class KerberosConfExecutorFeatureStep(
kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf])
extends KubernetesFeatureConfigStep with Logging{
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

space before {

* @return a modified SparkPod
*/
def bootstrapKerberosPod(
dtSecretName: String,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

arg indentation

@SparkQA
Copy link

SparkQA commented Oct 9, 2018

@SparkQA
Copy link

SparkQA commented Oct 9, 2018

Kubernetes integration test status success
URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/3835/

@SparkQA
Copy link

SparkQA commented Oct 10, 2018

@SparkQA
Copy link

SparkQA commented Oct 10, 2018

Kubernetes integration test status success
URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/3839/

@SparkQA
Copy link

SparkQA commented Oct 10, 2018

Test build #97175 has finished for PR 21669 at commit e2f8063.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 10, 2018

Test build #97183 has finished for PR 21669 at commit a958920.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

HadoopBootstrapUtil.bootstrapHadoopConfDir(None, None, hadoopConfDirCMapName, pod)
}

override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No need to address it here but it feels like these methods should have default implementations, given that lots of classes just don't do anything with them.


override def configurePod(pod: SparkPod): SparkPod = {
val sparkUserName = kubernetesConf.sparkConf.get(KERBEROS_SPARK_USER_NAME)
HadoopBootstrapUtil.bootstrapSparkUserPod(sparkUserName, pod)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

indentation

*/
private[spark] class KerberosConfDriverFeatureStep(
kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf])
extends KubernetesFeatureConfigStep with Logging {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ping.

docs/security.md Outdated

In all cases you must define the environment variable: `HADOOP_CONF_DIR` or
`spark.kubernetes.hadoop.configMapName` as well as either
`spark.kubernetes.kerberos.krb5.path` or `spark.kubernetes.kerberos.krb5.configMapName`.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This needs a small update after the recent changes.

}

override def getAdditionalKubernetesResources(): Seq[HasMetadata] = {
// HADOOP_CONF_DIR ConfigMap
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Unnecessary comment.

fileLocation)
}

// Kerberos DT Secret
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Unnecessary comment.


private[spark] object HadoopBootstrapUtil extends Logging {

/**
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

indentation is wrong

SparkPod(pod.pod, envModifiedContainer)
}

/**
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

indentation is wrong

* @return a modified SparkPod
*/
def bootstrapSparkUserPod(sparkUserName: String, pod: SparkPod): SparkPod = {
val envModifiedContainer = new ContainerBuilder(pod.container)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

indentation is wrong

import org.apache.spark.deploy.k8s.Constants._
import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager

/**
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

indentation is wrong.

Really, when I asked you to go through all your code, I meant all your code, not just the code I commented on. Please do that.

Copy link
Contributor Author

@ifilonenko ifilonenko Oct 10, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I apologize, these slip through the cracks. Definitely gonna add an additional linter / style-checker as the Jenkin's scalastyle checks seem to be quite primitive and does not complain. Thank you for patience in your review! I really appreciate it.

@SparkQA
Copy link

SparkQA commented Oct 10, 2018

@SparkQA
Copy link

SparkQA commented Oct 10, 2018

Kubernetes integration test status success
URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/3860/

@SparkQA
Copy link

SparkQA commented Oct 11, 2018

Test build #97220 has finished for PR 21669 at commit dd95fca.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@vanzin
Copy link
Contributor

vanzin commented Oct 15, 2018

retest this please

@vanzin
Copy link
Contributor

vanzin commented Oct 15, 2018

I went through the code again, and the main thing for me is that following this whole "steps" thing in the k8s backend still seems really hard to me. It feels like related code gets scattered around into a bunch of different places for no reason that I can detect.

e.g. adding the "HadoopConfigSpec" thing to "KubernetesConf" just seems to require a bunch of changes in unrelated places instead of just reading that information from the underlying configuration where it's actually needed.

But that's not going to be fixed here. When/if I have time I might try to figure this stuff out.

I still noticed some remaining style issues but at this point those don't matter. The functionality looks ok so, pending the new test run, I'll merge this.

@SparkQA
Copy link

SparkQA commented Oct 15, 2018

@SparkQA
Copy link

SparkQA commented Oct 15, 2018

Kubernetes integration test status success
URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/4000/

@SparkQA
Copy link

SparkQA commented Oct 15, 2018

Test build #97408 has finished for PR 21669 at commit dd95fca.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@vanzin
Copy link
Contributor

vanzin commented Oct 15, 2018

Merging to master.

@asfgit asfgit closed this in 6c9c84f Oct 15, 2018
@hustcat
Copy link
Contributor

hustcat commented Feb 15, 2019

Why closed? abandoned?

jackylee-ch pushed a commit to jackylee-ch/spark that referenced this pull request Feb 18, 2019
## 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 apache#22608
- [ ] Unit tests

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

## Contribution Credit
kimoonkim skonto

Closes apache#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>
@rvesse
Copy link
Member

rvesse commented Feb 19, 2019

@hustcat If you looked at the history above you would see that this was merged as part of
6c9c84f and is present in current master

The Closed status is an unfortunate artefact of how ASF infrastructure does mirroring between there and GitHub

markterm pushed a commit to CodecLondon/spark that referenced this pull request Jul 4, 2019
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

Locally tested on a single-noded, pseudo-distributed Kerberized Hadoop Cluster
- [x] E2E Integration tests apache#22608
- [ ] Unit tests

- [x] Docs
- [x] Error Handling

kimoonkim skonto

Closes apache#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>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.