Skip to content

Conversation

vanzin
Copy link
Contributor

@vanzin vanzin commented Oct 31, 2018

This change hooks up the k8s backed to the updated HadoopDelegationTokenManager,
so that delegation tokens are also available in client mode, and keytab-based token
renewal is enabled.

The change re-works the k8s feature steps related to kerberos so
that the driver does all the credential management and provides all
the needed information to executors - so nothing needs to be added
to executor pods. This also makes cluster mode behave a lot more
similarly to client mode, since no driver-related config steps are run
in the latter case.

The main two things that don't need to happen in executors anymore are:

  • adding the Hadoop config to the executor pods: this is not needed
    since the Spark driver will serialize the Hadoop config and send
    it to executors when running tasks.

  • mounting the kerberos config file in the executor pods: this is
    not needed once you remove the above. The Hadoop conf sent by
    the driver with the tasks is already resolved (i.e. has all the
    kerberos names properly defined), so executors do not need access
    to the kerberos realm information anymore.

The change also avoids creating delegation tokens unnecessarily.
This means that they'll only be created if a secret with tokens
was not provided, and if a keytab is not provided. In either of
those cases, the driver code will handle delegation tokens: in
cluster mode by creating a secret and stashing them, in client
mode by using existing mechanisms to send DTs to executors.

One last feature: the change also allows defining a keytab with
a "local:" URI. This is supported in client mode (although that's
the same as not saying "local:"), and in k8s cluster mode. This
allows the keytab to be mounted onto the image from a pre-existing
secret, for example.

Finally, the new code always sets SPARK_USER in the driver and
executor pods. This is in line with how other resource managers
behave: the submitting user reflects which user will access
Hadoop services in the app. (With kerberos, that's overridden
by the logged in user.) That user is unrelated to the OS user
the app is running as inside the containers.

Tested:

  • client and cluster mode with kinit
  • cluster mode with keytab
  • cluster mode with local: keytab
  • YARN cluster with keytab (to make sure it isn't broken)

… renewal.

This change hooks up the k8s backed to the updated HadoopDelegationTokenManager,
so that delegation tokens are also available in client mode.

It also closes the remaining gap in kerberos support which is to support
the current keytab-based token renewal supported by YARN (and partially by
Mesos). That is done by stashing the keytab in a secret on the driver pod,
and then providing it to the HadoopDelegationTokenManager when the
driver starts.

Tested in cluster mode with and without a keytab, and also in client
mode.
@SparkQA
Copy link

SparkQA commented Oct 31, 2018

Test build #98333 has finished for PR 22911 at commit 7f86e23.

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

@SparkQA
Copy link

SparkQA commented Oct 31, 2018

@SparkQA
Copy link

SparkQA commented Oct 31, 2018

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

@mccheah
Copy link
Contributor

mccheah commented Oct 31, 2018

@ifilonenko @skonto

@ifilonenko
Copy link
Contributor

ifilonenko commented Oct 31, 2018

Just a note, this PR has some overlaps with #22915. But just after a brief skim, I am a bit hesitant on storing keytabs in secrets, but it might have to be done for cluster client mode support.

@vanzin
Copy link
Contributor Author

vanzin commented Oct 31, 2018

It is actually not needed for client mode because only the driver needs the keytab.

But whether to store it in secrets is not a question. You either store it in a secret or you don't support the keytab/principal feature in Spark at all, and we can delete a bunch of code here.

}

override protected def createTokenManager(): Option[HadoopDelegationTokenManager] = {
Some(new HadoopDelegationTokenManager(conf, sc.hadoopConfiguration))
Copy link
Contributor

@ifilonenko ifilonenko Nov 5, 2018

Choose a reason for hiding this comment

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

If we are introducing this change, I think it is important that we talk about the future of secret creation upon using --keytab + principle. Right now, secrets are created when a keytab is used by the kubernetes client or for client-mode by the driver; this was used primarily for testing (on my end) but also because this logic wasn't previously generalized for all cluster-managers. Should we create an option for the user to create a secret or get rid of it as a whole, as delegation token logic is handled via the UpdateDelegationToken message passing framework. In essence, if we leave the ability to create a secret we are twice obtaining a DT which is extraneous. And if we are removing it, it is sensible to refactor the KerberosConfig logic to account for this removal. I was planning to do this in my token renewal PR where I was also introducing this change, but it seems that this will probably get merged in before mine, as such, here would be a better place to refactor. Or maybe a seperate PR that introduces this line and does the refactor, and then this and my PR could be introduced subsequently.

thoughts, @vanzin ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I'm not sure I follow your train of thought here so I'll comment on what I understand.

First, the code that creates the secret is in KerberosConfDriverFeatureStep. As far as I know, that class is not used in client mode. In client mode the keytab stays in the client machine, with the driver, and the driver just sends DTs to executors. So the whole discussion about secrets is irrelevant in that case.

In cluster mode, you need the driver to have access to the keytab for this feature to work. There are a few ways to achieve that:

  • the current YARN mode, which is the keytab lives in the submission host, and is distributed with the application. In k8s this would amount to what I have here: the submission code creates a secret for the driver pod and stashes the keytab in it.

  • add the ability to store the keytab in an external place (like HDFS or an HTTP server). That has drawbacks (e.g. people probably wouldn't like that, and there's a chicken & egg problem in HDFS, so you'd still need a kerberos TGT to bootstrap things).

  • add a k8s-specific feature of mounting a pre-defined secret in the driver pod. I believe this is what you're suggesting?

I think supporting the first is easy as this change shows, and keeps feature parity with what's already supported in YARN. Unless there's a glaring issue with using secrets that I'm not aware of, I don't see a reason for not doing it.

The third option (pre-defined secret) could also be added. My hope is that you could do it with pre-existing configs (spark.kubernetes.driver.secrets. & company), but I don't know how you'd set the spark.kerberos.keytab and spark.kerberos.principal configs just in the driver - and not in the submission client. So it seems we'd need at least a little bit of code here to support that scenario.

Copy link
Contributor

Choose a reason for hiding this comment

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

Oh, I was referencing the creation of Delegation Token secret when a --keytab is specified. I believe that you are right in that in client-mode you would not need to worry about running this step. But I think the 3rd option would be good to include here. I think that with the introduction of HadoopDelegationTokenManager we should remove the creation of the dtSecret, and that should be included in this PR if you are introducing this. Therefore, I think it is sensible to refactor the KerberosConfigSpec to have a generic secret, secretName, secretKey, that would either contain a DelegationToken or a keytab. Such that the code block:

  private val kerberosConfSpec: Option[KerberosConfigSpec] = (for {
    secretName <- existingSecretName
    secretItemKey <- existingSecretItemKey
  } yield {
    KerberosConfigSpec(
      secret = None,
      secretName = secretName,
      secretItemKey = secretItemKey,
      jobUserName = kubeTokenManager.getCurrentUser.getShortUserName)
  }).orElse(
    if (isKerberosEnabled) {
      keytab.map { ..... }
    } else {
      None
    }

would return a kerberosConfSpec that would account for either case. Erm, that would also mean that you could delete the HadoopKerberosLogin method.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ah, ok I get it now. I can do that. I'll try to include support for (3) but it depends on how much I have to touch other parts of the code. Hopefully not much.

Copy link
Contributor

Choose a reason for hiding this comment

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

Yeah, I can always throw up a follow-up for that. No worries

@ifilonenko
Copy link
Contributor

ifilonenko commented Nov 5, 2018

But whether to store it in secrets is not a question. You either store it in a secret or you don't support the keytab/principal feature in Spark at all, and we can delete a bunch of code here.

But can we, for a specific mode, client or cluster, tune whether the keytab is put into a secret? Right now, if a keytab is specified, regardless, a secret is created. I do wish to support keytab/principal so I think that storage in secrets should be fine.

@@ -126,20 +134,53 @@ private[spark] class KerberosConfDriverFeatureStep(
HadoopBootstrapUtil.bootstrapSparkUserPod(
kubeTokenManager.getCurrentUser.getShortUserName,
hadoopBasedSparkPod))

if (keytab.isDefined) {
val podWitKeytab = new PodBuilder(kerberizedPod.pod)
Copy link
Member

Choose a reason for hiding this comment

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

Typo - Wit -> With

Marcelo Vanzin added 5 commits November 7, 2018 15:26
This change re-works the k8s feature steps related to kerberos so
that the driver and executor steps are more clearly defined, and
less things need to be done in executor pods.

The main driver behind a lot of these changes is that, in client
mode, no driver-related config steps are run. So all executor-related
steps must be able to be applied even when no driver steps have
previously run and potentially modified the pod or Spark configuration.

The second is that based on the above changes, a bunch of things that
were being done on executor pods are actually not needed. These are:

- adding the Hadoop config to the executor pods: this is not needed
  since the Spark driver will serialize the Hadoop config and send
  it to executors when running tasks.

- mounting the kerberos config file in the executor pods: this is
  not needed once you remove the above. The Hadoop conf sent by
  the driver with the tasks is already resolved (i.e. has all the
  kerberos names properly defined), so executors do not need access
  to the kerberos realm information anymore.

So there is a single thing that executor pods need: mounting the
delegation token secret when one is provided. Everything else is
handled in the driver, regardless of whether it's client or cluster
mode.

The change also avoids creating delegation tokens unnecessarily.
This means that they'll only be created if a secret with tokens
was not provided, and if a keytab is not provided. In either of
those cases, the driver code will handle delegation tokens: in
cluster mode by creating a secret and stashing them, in client
mode by using existing mechanisms to send DTs to executors.

One last feature: the change also allows defining a keytab with
a "local:" URI. This is supported in client mode (although that's
the same as not saying "local:"), and in k8s cluster mode. This
allows the keytab to be mounted onto the image from a pre-existing
secret, for example.

Finally, the new code always sets SPARK_USER in the driver and
executor pods. This is in line with how other resource managers
behave: the submitting user reflects which user will access
Hadoop services in the app. (With kerberos, that's overridden
by the logged in user.) That user is unrelated to the OS user
the app is running as inside the containers.

Tested:
- client and cluster mode with kinit
- cluster mode with keytab
- cluster mode with local: keytab
- YARN cluster with keytab (to make sure it isn't broken)
@vanzin
Copy link
Contributor Author

vanzin commented Nov 14, 2018

To address the comments I had to change a whole bunch of things; mainly because I noticed that the client and cluster paths were really different, when they should for the most part be similar. Check the updated PR description for details.

* it can do matching without needing to exhaust all the possibilities. If the function
* is not applied, then the original pod will be kept.
*/
def transform(fn: PartialFunction[SparkPod, SparkPod]): SparkPod = fn.lift(this).getOrElse(this)
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I added this because I started to get tired of code like the following:

val someIntermediateName = someOption.map { blah =>
   // create the updated pod
}.getOrElse(previousPodName)

// lather, rinse, repeat

To me that's hard to follow and brittle, and this pattern makes things clearer IMO.

Copy link
Contributor

Choose a reason for hiding this comment

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

I would think that this change is out of the scope of this PR, but I do love the use of a PartialFunction here. Thanks for this!

@SparkQA
Copy link

SparkQA commented Nov 14, 2018

Test build #98832 has finished for PR 22911 at commit a129314.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 14, 2018

@SparkQA
Copy link

SparkQA commented Nov 14, 2018

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

@SparkQA
Copy link

SparkQA commented Nov 14, 2018

@SparkQA
Copy link

SparkQA commented Nov 14, 2018

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

* When preparing pods, this step will mount the delegation token secret (either pre-defined,
* or generated by this step when preparing the driver).
*/
private[spark] class DelegationTokenFeatureStep(conf: KubernetesConf[_], isDriver: Boolean)
Copy link
Contributor Author

Choose a reason for hiding this comment

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

During the recent discussion in the sig meeting I was thinking about this guy. I think it may not be necessary to mount this secret in executors, and let the driver propagate the tokens to executors through its normal means.

e.g. later if/when adding code to monitor delegation tokens for updates, that would mean only the driver has to do it.

I'll git this a try; if it works, it might be feasible to re-merge this code with the kerberos step.

Copy link
Contributor

Choose a reason for hiding this comment

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

This is what I meant above when I said that the HadoopKerberosLogin logic could be deleted. The assumption here is that the secret should not be created as the keytab will use the HadoopDelegationTokenManager logic. The only secret that should be created would be the keytab. However, I personally thought that we should point to a secretName that is either the delegationToken or the keytab. Hence why I suggested that the secretName and secretItemKey remain.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think you're missing one use case there. There are 3 use cases:

  • keytab: keytab is provided to driver, driver handles kerberos login, creates tokens, distributes tokens to executors
  • pre-defined token secret: secret is mounted in the driver, env variable is set, driver loads them and distributes to executors
  • default kerberos case: submission client generates delegation tokens, creates a secret for them, then this behaves like the bullet above

The third use case is actually the most common. In your reply above you're only covering the other two. My code covers all three.

I'm just saying that this code actually doesn't need to do anything on the executor side, because the driver takes care of everything when the credentials are provided.

@SparkQA
Copy link

SparkQA commented Nov 14, 2018

Test build #98833 has finished for PR 22911 at commit d8416cd.

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

@@ -144,6 +144,10 @@ private[spark] class BasicExecutorFeatureStep(
.addToLimits("memory", executorMemoryQuantity)
.addToRequests("cpu", executorCpuQuantity)
.endResources()
.addNewEnv()
.withName(ENV_SPARK_USER)
Copy link
Contributor

Choose a reason for hiding this comment

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

I see that you noted that this is always done across resource managers. What is the reason for that, just wondering? as I introduced it exclusively in the HadoopSteps

Copy link
Contributor Author

Choose a reason for hiding this comment

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

If you don't do this, whatever is the OS user in the container will become the identity used to talk to Hadoop services (when kerberos is not on).

In YARN, for example, that would be the "yarn" user.

In k8s, with the current image, that would be "root".

You probably don't want that by default. We're talking about non-secured Hadoop here, so users can easily override this stuff, but by default let's at least try to identify the user correctly.

kubernetesConf.krbConfigMapName,
fileLocation)
// Visible for testing.
def createDelegationTokens(): Array[Byte] = {
Copy link
Contributor

@ifilonenko ifilonenko Nov 14, 2018

Choose a reason for hiding this comment

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

nit: Why isn't this logic done in a KubernetesDelegationTokenManager or some Util class? It seems we are over-running this step with a lot of functionality? no? You are also just mocking via
when(step).createDelegationToken(), I personally don't agree with the coverage brought up by this method of unit-testing but this is an extension of the discussion on a separate PR. i.e. I would mock the tokenManager and test to ensure that the obtainDelegationTokens take in a particular creds as an argument (also, so that one could test the if statement below).

Copy link
Contributor Author

@vanzin vanzin Nov 14, 2018

Choose a reason for hiding this comment

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

Why isn't this logic done in a KubernetesDelegationTokenManager or some Util class?

Because that's not needed. What exactly would doing that achieve?

I would mock the tokenManager and test to ensure that the obtainDelegationTokens take in a particular creds

That's not what the corresponding test is testing. It is testing that when delegation tokens are created, they are stashed in a secret. For that I'm mocking the creation of delegation tokens, because without any services to talk to, delegation tokens will not be created.

(BTW, that's what mocks should be used for.)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I'm mocking the creation of delegation tokens

BTW there are other ways to achieve that, e.g. with UserGroupInformation.createTestUser() and doAs, but I think the current test is easier to understand.

Copy link
Contributor

@ifilonenko ifilonenko Nov 14, 2018

Choose a reason for hiding this comment

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

Because that's not needed. What exactly would doing that achieve?

I mean that is the function of a DelegationTokenManager, no? To handle nuances related to DT management, both for creation and renewal. shrug

This code is testing that when delegation tokens are created, they are stashed in a secret

Well there is logic being introduced here beyond just tokenManager.obtainDelegationToken(), right? And if the purpose of the tests is to take the results from delegation token creation and put into a secret, then it would make sense to mock a KubernetesDelegtionTokenManager and just call kdtm.obtainTokens(), where this logic would be housed, or to leave it within this step. But regardless, I'd prefer that all parts of the logic are mocked.

Once again, a matter of opinion, that differs between unit-testing strategy. I'll loop in @mccheah

I think the current test is easier to understand.

I agree, but is it comprehensive is what I am talking about here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

that is the function of a DelegationTokenManager. To handle nuances related to DT management

Yes. And the test here is not interested in whether DT creation works or not. It's interested in that if they are created, then they are stashed in a secret.

then it would make sense to mock a KubernetesDelegtionTokenManager and just call kdtm.obtainTokens()

Would it make you happier if instead of having instead of the current method, have this:

def createDelegationTokens(creds: Credentials)

The only extra thing you'd really be testing is the call to SparkHadoopUtil.get.serialize. I don't really see a lot of gains there.

Copy link
Contributor

Choose a reason for hiding this comment

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

Yes. And the test here is not interested in whether DT creation works or not.

Hmm, I somewhat disagree, but maybe I am just overly opinionated. I think that the function definition should be:
def createDelegationTokens(creds: Credentials, tm: HadoopDelegationTokenManager) so that you can, most importantly imo, mock the tm to ensure that the result coming from that method results in the necessary tokens (and if it is necessary to test that: if it returns nothing that no secret is created)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hmm, I somewhat disagree, but maybe I am just overly opinionated.

The code that creates delegation tokens is not in this class. If delegation token creation is to be tested, it needs to be tested in the test code for HadoopDelegationTokenManager.

All this class does is take the tokens that other class creates and stash them in a secret.

I'll change this to not need a mock at all, mostly to make a point.

hFiles <- hadoopConfigurationFiles
} yield {
HadoopBootstrapUtil.buildHadoopConfigMap(hName, hFiles)
Seq[HasMetadata]() ++ {
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: is there a particular advantage gained by this style of Seq() building? I prefer the .toSeq approach as I personally don't like if {...} else { Nil }. Once again, why was the Util class removed? Isn't this a lot of logic to throw into this one step?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

re: the style, same reasoning as the transform method I added. It avoids having to create a bunch of local variables and then concatenate them at the end.

re: the Util class, it existed because the methods were being called from multiple feature steps. Now there's only one step that does that stuff, so the util class became unnecessary.

Copy link
Contributor

Choose a reason for hiding this comment

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

the style, same reasoning as the transform method I added. It avoids having to create a bunch of local variables and then concatenate them at the end.

The transform I like because it gets rid of:

  val someIntermediateName = someOption.map { blah =>
     // create the updated pod
  }.getOrElse(previousPodName)

But that is not what is happening here exactly. Personally, I thought that the previous approach (with local variables + concatonation) was quite clear and didn't really require a change. But its a matter of opinion.

Now there's only one step that does that stuff, so the util class became unnecessary.

Okay I guess from a re-usability perspective that is true, but imo seems like a lot of logic going into this step. but once again, just a matter of opinion

Copy link
Contributor Author

Choose a reason for hiding this comment

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

seems like a lot of logic going into this step

The logic itself isn't really different from before. The only difference is that the "what is done" for each combination of configs now also lives here.

It could be broken down into separate steps, but then you'd have 3 related steps that need to agree on how to treat the configuration of the driver; e.g. if the step that adds they keytab runs, then the step that creates delegation tokens shouldn't run. And that makes the code harder to follow, because you have to look at multiple places to understand how this works.

val maybeHadoopConfigStep =
kubernetesConf.hadoopConfSpec.map { _ =>
provideHadoopGlobalStep(kubernetesConf)}
val otherSteps = Seq(
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: otherStep isn't very descriptive to the contents of the steps. I prefer hadoopSteps if anything

Copy link
Contributor Author

Choose a reason for hiding this comment

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

All this code will go away when I get to SPARK-25877, so I'm not super concerned with the names here.

@rvesse
Copy link
Member

rvesse commented Dec 14, 2018

This might be a bug, I've complained in the past that HDFS stuff doesn't seem to work unless you set HADOOP_CONF_DIR in the submission clients environments

@vanzin
Copy link
Contributor Author

vanzin commented Dec 14, 2018

This one might have been just a bad merge on my part. Without integration tests this is hard to catch (and I was having trouble with minikube yesterday).

I'll try out my tests on minikube tomorrow.

@SparkQA
Copy link

SparkQA commented Dec 14, 2018

@SparkQA
Copy link

SparkQA commented Dec 14, 2018

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

@erictcgs
Copy link

Great, that commit looks promising - I'll test in the morning.

I did have HADOOP_CONF_DIR set in the submit environment, had only tried the configmap after it didn't seem to work.

Do you suspect this may fix the executors not using the delegation token as well? I couldn't find documentation on how they get signaled to use that for hdfs access, or how to check whether they're trying to - if the dt is supposed to be mapped in as a configmap/secret and there's a command argument/config/env var signaling them to use it, or if there's another channel from the driver to pass the dt and have them use it?

@SparkQA
Copy link

SparkQA commented Dec 14, 2018

Test build #100118 has finished for PR 22911 at commit ccb3956.

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

@erictcgs
Copy link

Had a chance to try it - the driver pod is now getting the keytab mounted and is able to generate delegation token, so that's looking good. I'm still having an issue where the executors seem to not be using the dt and so are getting denied access - are they also supposed to have hadoop_conf_dir set?

@erictcgs
Copy link

@vanzin - how are you compiling spark? Specifically, what version of hadoop are you testing with? I'm working with v2.7.7, wondering if there are changes in later versions to the ipc.client and usergroupinformation classes that may handle delegation tokens differently

@vanzin
Copy link
Contributor Author

vanzin commented Dec 14, 2018

I've tried this again in all configurations I know (and also pyspark) and it's working for me:

  • cluster mode: keytab, local: keytab, no keytab
  • client mode: keytab / no keytab, client in my laptop, client in k8s pod
  • spark-shell and pyspark shell with tgt

I'm building Spark normally. Make sure your images are updated in docker (remember that the pyspark image is not built by default anymore).

You should see a log message like this in the executors:

2018-12-14 18:52:00 INFO  SparkHadoopUtil:58 - Updating delegation tokens for current user.

@erictcgs
Copy link

Continuing to dig on this - I do have that "Updating delegation tokens" line, but later find that the UserGroupInformation is attempting to access HDFS using auth:SIMPLE.

It looks like the executor is running everything in the context of runAsSparkUser, which ends up creating a new UserGroupInformation user with createRemoteUser, which defaults to auth:simple

I see how addCurrentUserCredentials in Spark is calling addCredentials in hadoop, but I don't see anywhere that's changing the authMethod of the sparkuser from SIMPLE to TOKEN - shouldn't this be somewhere?

My working hypothesis is that my executor is trying to run with sparkuser credentials, which have the delegation token present, but authmethod set to simple. Currently modifying SparkHadoopUtil.addCurrentUserCredentials to call setAuthenticationMethod(TOKEN) when auth method was SIMPLE (will post with results), but would love to hear if this is known how SIMPLE is supposed to change to TOKEN

@vanzin
Copy link
Contributor Author

vanzin commented Dec 14, 2018

The code you're mentioning really should not have any effect on what's happening when reading from HDFS.

The config used to connect to HDFS comes from the driver; when you run a job, the driver will serialize its Hadoop config and broadcast it to executors. And that config should have all the needed auth configuration correctly set.

Maybe you have another set of conflicting Hadoop configs somewhere (my executors have zero Hadoop configuration anywhere). Or perhaps you should be sharing the code you have, to explain exactly what you're doing.

@erictcgs
Copy link

Update - my setAuthenticationMethod change didn't work - after calling that I see that the executor reports it's trying to use TOKEN method, but it's still getting rejected with an error that it can't login via TOKEN, KERBEROS:

18/12/14 23:26:01 DEBUG UserGroupInformation: PrivilegedActionException as:user (auth:TOKEN) cause:org.apache.hadoop.security.AccessControlException: Client cannot authenticate via:[TOKEN, KERBEROS]
18/12/14 23:26:01 DEBUG UserGroupInformation: PrivilegedAction as:user (auth:TOKEN) from:org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:651)
18/12/14 23:26:01 WARN Client: Exception encountered while connecting to the server : org.apache.hadoop.security.AccessControlException: Client cannot authenticate via:[TOKEN, KERBEROS]

As a test I tried mounting in my keytab and hadoop_conf_dir into the executors and setting KRB5CCNAME env var (via pod template) - this results in the main user being kerberized, then I commented out the "runassparkuser" section of the coarsescheduler, and now the executors are able to successfully login to HDFS and grab the parquet data.

It looks like there's some incompatibility in my environment with how the delegation tokens are being used - I think this shouldn't block this merge if it's working for others - is there any other venue people would suggest I continue trying to debug?

@vanzin
Copy link
Contributor Author

vanzin commented Dec 14, 2018

I suggest filing a bug or starting a mailing list thread. But really I doubt anyone will be able to help if you only provide the errors, without explaining your environment and, especially, the code you're running.

@vanzin
Copy link
Contributor Author

vanzin commented Dec 17, 2018

retest this please

@SparkQA
Copy link

SparkQA commented Dec 17, 2018

@SparkQA
Copy link

SparkQA commented Dec 17, 2018

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

@SparkQA
Copy link

SparkQA commented Dec 18, 2018

Test build #100257 has finished for PR 22911 at commit ccb3956.

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

@vanzin
Copy link
Contributor Author

vanzin commented Dec 18, 2018

Alright, given the lack of further comments, merging to master.

@asfgit asfgit closed this in 4b3fe3a Dec 18, 2018
@vanzin vanzin deleted the SPARK-25815 branch December 18, 2018 22:06
holdenk pushed a commit to holdenk/spark that referenced this pull request Jan 5, 2019
…n renewal.

This change hooks up the k8s backed to the updated HadoopDelegationTokenManager,
so that delegation tokens are also available in client mode, and keytab-based token
renewal is enabled.

The change re-works the k8s feature steps related to kerberos so
that the driver does all the credential management and provides all
the needed information to executors - so nothing needs to be added
to executor pods. This also makes cluster mode behave a lot more
similarly to client mode, since no driver-related config steps are run
in the latter case.

The main two things that don't need to happen in executors anymore are:

- adding the Hadoop config to the executor pods: this is not needed
  since the Spark driver will serialize the Hadoop config and send
  it to executors when running tasks.

- mounting the kerberos config file in the executor pods: this is
  not needed once you remove the above. The Hadoop conf sent by
  the driver with the tasks is already resolved (i.e. has all the
  kerberos names properly defined), so executors do not need access
  to the kerberos realm information anymore.

The change also avoids creating delegation tokens unnecessarily.
This means that they'll only be created if a secret with tokens
was not provided, and if a keytab is not provided. In either of
those cases, the driver code will handle delegation tokens: in
cluster mode by creating a secret and stashing them, in client
mode by using existing mechanisms to send DTs to executors.

One last feature: the change also allows defining a keytab with
a "local:" URI. This is supported in client mode (although that's
the same as not saying "local:"), and in k8s cluster mode. This
allows the keytab to be mounted onto the image from a pre-existing
secret, for example.

Finally, the new code always sets SPARK_USER in the driver and
executor pods. This is in line with how other resource managers
behave: the submitting user reflects which user will access
Hadoop services in the app. (With kerberos, that's overridden
by the logged in user.) That user is unrelated to the OS user
the app is running as inside the containers.

Tested:
- client and cluster mode with kinit
- cluster mode with keytab
- cluster mode with local: keytab
- YARN cluster with keytab (to make sure it isn't broken)

Closes apache#22911 from vanzin/SPARK-25815.

Authored-by: Marcelo Vanzin <[email protected]>
Signed-off-by: Marcelo Vanzin <[email protected]>
jackylee-ch pushed a commit to jackylee-ch/spark that referenced this pull request Feb 18, 2019
…n renewal.

This change hooks up the k8s backed to the updated HadoopDelegationTokenManager,
so that delegation tokens are also available in client mode, and keytab-based token
renewal is enabled.

The change re-works the k8s feature steps related to kerberos so
that the driver does all the credential management and provides all
the needed information to executors - so nothing needs to be added
to executor pods. This also makes cluster mode behave a lot more
similarly to client mode, since no driver-related config steps are run
in the latter case.

The main two things that don't need to happen in executors anymore are:

- adding the Hadoop config to the executor pods: this is not needed
  since the Spark driver will serialize the Hadoop config and send
  it to executors when running tasks.

- mounting the kerberos config file in the executor pods: this is
  not needed once you remove the above. The Hadoop conf sent by
  the driver with the tasks is already resolved (i.e. has all the
  kerberos names properly defined), so executors do not need access
  to the kerberos realm information anymore.

The change also avoids creating delegation tokens unnecessarily.
This means that they'll only be created if a secret with tokens
was not provided, and if a keytab is not provided. In either of
those cases, the driver code will handle delegation tokens: in
cluster mode by creating a secret and stashing them, in client
mode by using existing mechanisms to send DTs to executors.

One last feature: the change also allows defining a keytab with
a "local:" URI. This is supported in client mode (although that's
the same as not saying "local:"), and in k8s cluster mode. This
allows the keytab to be mounted onto the image from a pre-existing
secret, for example.

Finally, the new code always sets SPARK_USER in the driver and
executor pods. This is in line with how other resource managers
behave: the submitting user reflects which user will access
Hadoop services in the app. (With kerberos, that's overridden
by the logged in user.) That user is unrelated to the OS user
the app is running as inside the containers.

Tested:
- client and cluster mode with kinit
- cluster mode with keytab
- cluster mode with local: keytab
- YARN cluster with keytab (to make sure it isn't broken)

Closes apache#22911 from vanzin/SPARK-25815.

Authored-by: Marcelo Vanzin <[email protected]>
Signed-off-by: Marcelo Vanzin <[email protected]>
srowen pushed a commit that referenced this pull request Apr 22, 2022
…riverFeatureStep`

### What changes were proposed in this pull request?

This PR removes a variable `hadoopConf` from `KerberosConfDriverFeatureStep`.

### Why are the changes needed?

#22959 added a variable `hadoopConf` to generate `tokenManager`. And, #22911 removed `tokenManager` and `buildKerberosSpec`, so `hadoopConf` is no-use.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass the GA.

Closes #36283 from dcoliversun/SPARK-38968.

Authored-by: Qian.Sun <[email protected]>
Signed-off-by: Sean Owen <[email protected]>
@turboFei
Copy link
Member

turboFei commented May 15, 2023

The main two things that don't need to happen in executors anymore are:
adding the Hadoop config to the executor pods: this is not needed
since the Spark driver will serialize the Hadoop config and send
it to executors when running tasks.

gentle ping @vanzin
seems the executor still need hadoop config.
FYI:
for driver, it works well to resolve hdfs schema: hdfs://zeus,
but failed on executor side.
image

dongjoon-hyun pushed a commit that referenced this pull request Jun 1, 2023
### What changes were proposed in this pull request?

In this pr, for spark on k8s, the hadoop config map will be mounted in executor side as well.
Before, the  hadoop config map is only mounted in driver side.
### Why are the changes needed?

Since [SPARK-25815](https://issues.apache.org/jira/browse/SPARK-25815) [,](#22911,) the hadoop config map will not be mounted in executor side.

Per the  #22911 description:

> The main two things that don't need to happen in executors anymore are:
> 1. adding the Hadoop config to the executor pods: this is not needed
> since the Spark driver will serialize the Hadoop config and send
> it to executors when running tasks.

But in fact, the executor still need the hadoop configuration.

![image](https://github.com/apache/spark/assets/6757692/ff6374c9-7ebd-4472-a85c-99c75a737e2a)

As shown in above picture, the driver can resolve `hdfs://zeus`, but the executor can not.

so we still need to mount the hadoop config map in executor side.

### Does this PR introduce _any_ user-facing change?

Yes, users do not need to take workarounds to make executors load the hadoop configuration.
Such as:
- including hadoop conf in executor image
- placing hadoop conf files under `SPARK_CONF_DIR`.
### How was this patch tested?

UT.

Closes #41181 from turboFei/exec_hadoop_conf.

Authored-by: fwang12 <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
czxm pushed a commit to czxm/spark that referenced this pull request Jun 12, 2023
### What changes were proposed in this pull request?

In this pr, for spark on k8s, the hadoop config map will be mounted in executor side as well.
Before, the  hadoop config map is only mounted in driver side.
### Why are the changes needed?

Since [SPARK-25815](https://issues.apache.org/jira/browse/SPARK-25815) [,](apache#22911,) the hadoop config map will not be mounted in executor side.

Per the  apache#22911 description:

> The main two things that don't need to happen in executors anymore are:
> 1. adding the Hadoop config to the executor pods: this is not needed
> since the Spark driver will serialize the Hadoop config and send
> it to executors when running tasks.

But in fact, the executor still need the hadoop configuration.

![image](https://github.com/apache/spark/assets/6757692/ff6374c9-7ebd-4472-a85c-99c75a737e2a)

As shown in above picture, the driver can resolve `hdfs://zeus`, but the executor can not.

so we still need to mount the hadoop config map in executor side.

### Does this PR introduce _any_ user-facing change?

Yes, users do not need to take workarounds to make executors load the hadoop configuration.
Such as:
- including hadoop conf in executor image
- placing hadoop conf files under `SPARK_CONF_DIR`.
### How was this patch tested?

UT.

Closes apache#41181 from turboFei/exec_hadoop_conf.

Authored-by: fwang12 <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
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.

9 participants