Skip to content

Conversation

@turboFei
Copy link
Member

@turboFei turboFei commented May 16, 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 , 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

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.

@turboFei turboFei changed the title [SPARK-43504] Mount hadoop config map in executor side [SPARK-43504][K8S] Mount hadoop config map in executor side May 16, 2023
@turboFei turboFei force-pushed the exec_hadoop_conf branch from 2945219 to c599d24 Compare May 16, 2023 04:08
@turboFei turboFei changed the title [SPARK-43504][K8S] Mount hadoop config map in executor side [SPARK-43504][K8S] Mount hadoop config map on the executor pod May 16, 2023
@turboFei turboFei changed the title [SPARK-43504][K8S] Mount hadoop config map on the executor pod [SPARK-43504][K8S] Mounts the hadoop config map on the executor pod May 16, 2023
@turboFei turboFei force-pushed the exec_hadoop_conf branch from 2b052b1 to 6689b74 Compare May 16, 2023 04:22
@pan3793
Copy link
Member

pan3793 commented May 16, 2023

I encountered the same issues w/ Spark 3.3.1. This sounds like a regression (I suppose it works before SPARK-25815, I don't have experience w/ running such an old version of Spark on K8s).

The key point is that the executor needs to download artifacts during the bootstrap phase, so the assumption in SPARK-25815 is not always true.

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.

Given the executor use SparkHadoopUtil.get.newConfiguration(conf) to construct Hadoop conf, we can put the related hdfs/s3 configurations into spark-defaults.conf w/ spark.hadoop. prefix as a workaround.

private[executor] def updateDependencies(
newFiles: Map[String, Long],
newJars: Map[String, Long],
newArchives: Map[String, Long],
testStartLatch: Option[CountDownLatch] = None,
testEndLatch: Option[CountDownLatch] = None): Unit = {
lazy val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf)

This PR definitely fixes some use cases, @turboFei would you mind updating "Does this PR introduce any user-facing change?"

@turboFei
Copy link
Member Author

would you mind updating "Does this PR introduce any user-facing change?"

updated

@turboFei
Copy link
Member Author

seems the k8s integration testing is stuck, will check this pr in our dev hadoop cluster tomorrow.

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

Thank you for making a PR, @turboFei .

However, this PR might cause a outage because the number of configMap is controlled by quota.

$ kubectl describe quota | grep configmaps
count/configmaps                                                  4     150

To avoid the production outage, this should be under a new configuration with false by default at least.

@turboFei
Copy link
Member Author

turboFei commented May 17, 2023

thanks for the comments, I will check it

@advancedxy
Copy link
Contributor

advancedxy commented May 17, 2023

Thank you for making a PR, @turboFei .

However, this PR might cause a outage because the number of configMap is controlled by quota.

$ kubectl describe quota | grep configmaps
count/configmaps                                                  4     150

To avoid the production outage, this should be under a new configuration with false by default at least.

150 is a bit small for serious production usage, we may add this note in the running_on_k8s.md documentation.

And BTW, this PR doesn't create new ConfigMaps, it either uses a user pre-set config map (no creation) or just reuse the config map created by driver which is created if necessary.

@turboFei
Copy link
Member Author

this PR doesn't create new ConfigMaps, it either uses a user pre-set config map (no creation) or just reuse the config map created by driver which is created if necessary.

yes, this PR doesn't create new ConfigMap.

@dongjoon-hyun
Copy link
Member

Oh, got it. Thank you for correcting me.

@turboFei
Copy link
Member Author

the UT has passed, gentle ping @dongjoon-hyun

Copy link
Contributor

@advancedxy advancedxy left a comment

Choose a reason for hiding this comment

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

lgtm.
left one minor comment, I'm ok with merging this as it is.

@turboFei
Copy link
Member Author

gentle ping @dongjoon-hyun would you like to review again? thanks

@yaooqinn
Copy link
Member

The Hadoop configurations can be propagated after #27735. And putting and locating extra configuration files in SPARK_HOME/conf is also a suggested way from our docs, so is this step necessary?

Alternatively, if both exist, what is the precedence between them? Is it idempotent?

@turboFei
Copy link
Member Author

turboFei commented May 22, 2023

And putting and locating extra configuration files in SPARK_HOME/conf is also a suggested way from our docs, so is this step necessary?

I think it is necessary.

Hadoop and spark are different components, it is better to maintain them separately.

In our company, we have conf version for hadoop conf, so we do not put hadoop config files under SPARK_HOME/conf, we use soft link to manage the hadoop conf.

Alternatively, if both exist, what is the precedence between them? Is it idempotent?

In this pr, it just mounts the hadoop config map in the executor side(mounts HADOOP_CONF_DIR env) and the hadoop conf mounted is absolute same with that in driver pod.

As shown below, the SPARK_CONF_DIR has higher precedence. I think it is idempotent.

if ! [ -z ${HADOOP_CONF_DIR+x} ]; then
SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH";
fi
if ! [ -z ${SPARK_CONF_DIR+x} ]; then
SPARK_CLASSPATH="$SPARK_CONF_DIR:$SPARK_CLASSPATH";
elif ! [ -z ${SPARK_HOME+x} ]; then
SPARK_CLASSPATH="$SPARK_HOME/conf:$SPARK_CLASSPATH";
fi

@turboFei
Copy link
Member Author

gentle ping @yaooqinn @dongjoon-hyun

@yaooqinn
Copy link
Member

Hadoop and spark are different components, it is better to maintain them separately.

I do not fully agree. In the early days, Hadoop may be special. We have a specific code path to read HADOOP_CONF_DIR. But now Hadoop is an option, as we have other options for storage and scheduling, especially on the cloud or Kubernetes.

Maybe we shall treat it like hive configurations or other third-party components to reduce the maintenance burden and complexity of the deployment.

@turboFei
Copy link
Member Author

turboFei commented May 23, 2023

Maybe we shall treat it like hive configurations or other third-party components to reduce the maintenance burden and complexity of the deployment.

I believe different companies treat hadoop conf differently.

For ebay, we add conf version for hadoop conf, because it is used by

  • public hadoop client nodes
  • private hadoop client nodes
  • hadoop service nodes(nn, rm, hms, kyuubi)
  • hadoop slave nodes(nm, dn)

and between different conf versions, there might be incompatibilities.

image

and we have an RESTful service to download the hdaoop conf and we use soft link to manage them locally.

Recently, we are making spark migration, from spark3 + hadoop2 to spark3 + hadoop3.

For hadoop2 and hadoop3, the hadoop confs are even different.

So to manage the hadoop conf well and due to the current situation, in ebay, we do not want to put the hadoop conf files and spark conf files together.

treat it like hive configurations or other third-party components to reduce the maintenance burden and complexity of the deployment.

yes, I agree, it makes it easy.

@infoankitp
Copy link
Contributor

I can also help with a use case for this, usually the submission client is on a single environment (Lets say we have it on cloud), and with spark on k8s, we can easily run jobs in different envs like in private Cloud Clusters being submitted from public Cloud. Where we would need diff properties to be passed for the submission client as well as for drivers and executors. This is also a use case where mounting the hadoopConfMap in executors would help in making the task easy to maintain the configs.

@turboFei
Copy link
Member Author

I can also help with a use case for this, usually the submission client is on a single environment (Lets say we have it on cloud), and with spark on k8s, we can easily run jobs in different envs like in private Cloud Clusters being submitted from public Cloud. Where we would need diff properties to be passed for the submission client as well as for drivers and executors. This is also a use case where mounting the hadoopConfMap in executors would help in making the task easy to maintain the configs.

Yes, I think this pr is general for hadoop conf use case, and it does not create more resource because it just use the existing config map.

@yaooqinn @dongjoon-hyun could you help to take another look? Appreciated for your help.

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

Although this way is a little controversial like @yaooqinn pointed out, I agree with the intention and use-cases and I believe we can allow this way back additionally without many burden or intervention because spark.kubernetes.executor.hadoopConfigMapName is still reserved in the code.

If there is no strong objection from other people, +1 from my side. Please let us know if you still disagree, @yaooqinn .

@dongjoon-hyun
Copy link
Member

Merged to master for Apache Spark 3.5.0.

@turboFei turboFei deleted the exec_hadoop_conf branch June 1, 2023 04:39
@turboFei
Copy link
Member Author

turboFei commented Jun 1, 2023

thanks all !!!

@yaooqinn
Copy link
Member

yaooqinn commented Jun 5, 2023

thanks, @dongjoon-hyun and @turboFei. Late +1 from my side.

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]>
@maomaodev
Copy link

gentle ping @turboFei
In the cluster mode, this merge has caused a new issue in our environment.
After the executor mounts the Hadoop conf file and tries to retrieve user information during startup, it fails to start because the conf file contains Kerberos authentication configuration, but the executor does not mount krb5.conf. Have you tested this scenario?
企业微信20250107-163956@2x

@pan3793
Copy link
Member

pan3793 commented Jan 7, 2025

@maomaodev Generally, the executor is supposed to use the delegation token instead of keytab to access kerberized HDFS. I wonder how do you set up your Kerberos auth for Spark? And what's the behavior before this PR.

@turboFei
Copy link
Member Author

turboFei commented Jan 7, 2025

gentle ping @turboFei In the cluster mode, this merge has caused a new issue in our environment. After the executor mounts the Hadoop conf file and tries to retrieve user information during startup, it fails to start because the conf file contains Kerberos authentication configuration, but the executor does not mount krb5.conf. Have you tested this scenario? 企业微信20250107-163956@2x

How about uploading the krb5.conf?

val KUBERNETES_KERBEROS_KRB5_FILE =
ConfigBuilder("spark.kubernetes.kerberos.krb5.path")
.doc("Specify the local location of the krb5.conf file to be mounted on the driver " +
"and executors for Kerberos. Note: The KDC defined needs to be " +
"visible from inside the containers ")
.version("3.0.0")
.stringConf
.createOptional

@maomaodev
Copy link

@maomaodev Generally, the executor is supposed to use the delegation token instead of keytab to access kerberized HDFS. I wonder how do you set up your Kerberos auth for Spark? And what's the behavior before this PR.

  1. Yes, the executor does use delegation token, but based on the stacktrace provided above, the code is throwing an exception before adding the delegation token.
企业微信20250107-171644@2x
  1. We use Kyuubi for submission, using the parameter spark.kubernetes.kerberos.krb5.path.
企业微信20250107-173420@2x
  1. Prior to this PR, we were using Spark 3.4.2, and the executor did not mount the Hadoop conf during startup. The executor logs showed simple authentication.
企业微信20250107-172445@2x

@maomaodev
Copy link

spark.kubernetes.kerberos.krb5.path

The parameter(spark.kubernetes.kerberos.krb5.path) has been set.

@pan3793
Copy link
Member

pan3793 commented Jan 7, 2025

@maomaodev I see what happens, the official Spark image installs krb5-user which generates a default /etc/krb5.conf with content

$ docker run --rm apache/spark:3.5.4 cat /etc/krb5.conf
[libdefaults]
	default_realm = ATHENA.MIT.EDU
...

so that KerberosUtil.getDefaultRealm works well, this is not the expected behavior but happens to work. I think the root cause is spark.kubernetes.kerberos.krb5.path claims to mount on both driver and executors but actually not.

In short, creating a dummy /etc/krb5.conf in your base image should workaround your issue, and the correct solution is mounting spark.kubernetes.kerberos.krb5.path to executor pod correctly.

@maomaodev
Copy link

@maomaodev I see what happens, the official Spark image installs krb5-user which generates a default /etc/krb5.conf with content

$ docker run --rm apache/spark:3.5.4 cat /etc/krb5.conf
[libdefaults]
	default_realm = ATHENA.MIT.EDU
...

so that KerberosUtil.getDefaultRealm works well, this is not the expected behavior but happens to work. I think the root cause is spark.kubernetes.kerberos.krb5.path claims to mount on both driver and executors but actually not.

In short, creating a dummy /etc/krb5.conf in your base image should workaround your issue, and the correct solution is mounting spark.kubernetes.kerberos.krb5.path to executor pod correctly.

Yes, creating a dummy /etc/krb5.conf in the base image does work. Is the community planning to fix this issue?

@pan3793
Copy link
Member

pan3793 commented Jan 7, 2025

@maomaodev I will prepare a PR soon

@maomaodev
Copy link

@maomaodev I will prepare a PR soon

Thank you, I have created a corresponding jira:https://issues.apache.org/jira/browse/SPARK-50758

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants