Open liyinan926 opened 6 years ago
@liyinan926 I was reading up on this topic, seems like one approach would be to leverage an init container that has access to both conf dirs, copying / merging files to the final destination (SPARK_CONF_DIR).
Does that sound like a viable solution? Given that Spark recently removed support for init containers, is this something we could accomplish with the webhook?
If you're not in a hurry, I'd like to try this one out (maybe after spark 2.4.0 comes out?) as a ramp-up issue.
@aditanase sorry for not responding sooner. I think that's a viable approach. And yes the admission webhook can be used to inject an init-container for doing that. Thanks for being willing to take on this. Looking forward to your implementation!
No worries. What are your thoughts on https://github.com/apache/spark/pull/22146? Together with support for client mode ( https://github.com/apache/spark/commit/571a6f0574e50e53cea403624ec3795cd03aa204#diff-b5527f236b253e0d9f5db5164bdb43e9), it will be quite disruptive for this project. At best, it might negate the need for the webhook. At worst, it might make things like this issue vert hard to implement. I'm planning to spend some time on that PR this week, you should probably check it out if you didn't already.
The pod template support will make the webhook obsolete for users using Spark 3.0 (assuming that's the version in which the PR will get into). For users who use Spark 2.4 and earlier, the webhook is still needed. So I wouldn't say it's disruptive, but it definitely requires some changes to the operator to make use of that.
@aditanase in the available timeline I don't see it disruptive - if webhook wont be needed for some users better it is. But to support earlier versions one would need webhook in the operator.
Hey @aditanase @liyinan926 @mrow4a, wondering what's the plan to resolve this? One use case is to provide hive-site.xml
in /opt/spark/conf
so we can configure hive behavior, right now it seems I can not achieve that. Would love to hear your inputs.
The approach proposed above of using an init-container to copy the files in sparkConfigMap
to $SPARK_CONF_DIR
of the main Spark container seems reasonable. This can be setup automatically by the operator.
Thanks @liyinan926 ! I'll dig a bit and see if I can help.
I setup an init-container and mount spark generated Volume(based on a configmap) and the operator generated Volume(based on a configmap) to it.
kind: Pod
apiVersion: v1
spec:
volumes:
- name: spark-local-dir-1
emptyDir: {}
- name: spark-conf-volume
configMap:
name: spark-pi3-1580169127969-driver-conf-map
defaultMode: 420
- name: spark-configmap-volume
configMap:
name: wweic-test-xml
defaultMode: 420
initContainers:
- name: init-spark-conf-dir
image: 'wweic-spark-operator:latest'
command:
- /usr/bin/setup-spark-configmap.sh
args:
- /opt/spark/src-conf
- /opt/spark/conf
resources: {}
volumeMounts:
- name: spark-configmap-volume
mountPath: /opt/spark/src-conf
- name: spark-conf-volume
mountPath: /opt/spark/conf
imagePullPolicy: IfNotPresent
Then I do a cp
to copy from operator config to spark config.
set -e
SCRIPT=`basename ${BASH_SOURCE[0]}`
echo "Setting up Spark Configmap for App"
SRC=$1
DST=$2
echo "Copy from ${SRC} to ${DST}"
echo "List src"
ls "${SRC}"
echo "List dst"
ls "${DST}"
touch "${SRC}"/helloworld
echo "List src"
ls "${SRC}"
cp "${SRC}"/* "${DST}"
It throws error:
cp: cannot create regular file in '/opt/spark/conf/test.xml': Read-only file system
This is by design according to k8s(https://github.com/kubernetes/kubernetes/issues/62099, https://github.com/kubernetes/kubernetes/pull/58720/files). A volume created from ConfigMap will be read only. So it won't work out continuing this direction.
Another idea is to patch the spark generated ConfigMap with our ConfigMap(I'll need to figure out how), or we use subPath
to mount each files from spark and operator user. What do you think? @liyinan926
@liyinan926 To follow up on the solution, I propose we patch driver/executor Pod's ConfigMaps to mount files defined in sparkConfigMap
in SparkApplicationSpec
or ScheduledSparkApplicationSpec
. Since both spark generated ConfigMaps and operator generated ConfigMaps will be mount to /opt/spark/conf
simultaneously, we can use subPath
to workaround volume mount limitation. To select which keys in sparkConfigMap
to mount, we need to add new key sparkConfigMapKeys
to specify the keys are available in the ConfigMap, then patch.go
can add a volume mount for each key to the Pod. Does this sound good? I can send a PR for review.
@wweic how about patching the internally created ConfigMap to append data stored in the user-specified ConfigMap (defined in sparkConfigMap
)? We can easily find the internally created ConfigMap because it has an ownerReference
pointing to the driver pod.
@liyinan926 Thanks for reply! I'm open to this option as well. Let me double check if we have access to full ConfigMap content in patch.go
.
@liyinan926 I looked at https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/3df703098970fbf7326ed4296470ea4c3688dec8/pkg/webhook/patch.go#L307-L326. We can patch the ConfigMap here.
But we need to call apiserver to get actual spec for 2 ConfigMaps for the following reasons:
pod
, I printed pod
and only the configmap name is there: &ConfigMapVolumeSource{LocalObjectReference:LocalObjectReference{Name:spark-pi2-1581507753326-driver-conf-map,}
app.Spec.SparkConfigMap
. We need to know the keys in the ConfigMap to merge with Spark generated ConfigMap.I looked around in the codebase, seems like we don't have a client obj to apiserver yet? Do you think we should add one? Or for the subpath approach, we don't need to retrieve the actual ConfigMap spec, just need to patch VolumeMounts. here is a quick patch I tried, I now have spark.properties
and hive-site.xml
in /opt/spark/conf
.
Appreciate your advice to move forward.
With the subPath
approach, you are basically changing the mount path of the internal ConfigMap to be at a subPath named after the properties file name, right?
@liyinan926 Yes, currently when we mount a ConfigMap to a directory, every key in the ConfigMap will become a file in the directory automatically, named by the key name. Using subPath, we have to mount each key in ConfigMap manually.
Example subPath spec:
volumeMounts:
- name: spark-local-dir-1
mountPath: /var/data/spark-2ce9a3df-c6c0-4b51-ae50-7c1ead092c44
- name: spark-conf-volume
mountPath: /opt/spark/conf/spark.properties
subPath: spark.properties
- name: spark-configmap-volume
readOnly: true
mountPath: /opt/spark/conf/hive-site.xml
subPath: hive-site.xml
@liyinan926 What do you think about the subPath
approach? It is a lightweight solution to the problem. But do let me know if there is any design considerations that I miss.
Hi, from the GuideDoc SPARK_CONF_DIR by default is set as "/opt/spark/conf" and will be modified to "/etc/spark/conf" when we specify spec.sparkConfigMap. And at the meanwhile "--properties-file /opt/spark/conf/spark.properties" is added automatically. So it seems the internal spark configs and the one under /etc/spark/conf will both be applied but the former one might be override if there are duplicated parameters. This is also what I observed in my experimentations, am I understanding correctly? Thanks!
We are running into an issue that may potentially be solved by the proposed subPath
approach. We need to have a custom hive-site.xml
in the config path to connect to Glue data catalog.
Is this issue being worked on actively or planned to be resolved soon?
@jaesong-mongo I have a quick patch here, pending @liyinan926's comment, I can send a more detailed RFC about it.
Would love to know the status of this issue.
We build spark docker images from scratch for internal use at my company, and include a /etc/spark/conf/spark-defaults.conf
file in docker image to define company-wide defaults. Unfortunately the mounted ConfigMap wipes out the directory including the spark-defaults.conf
file. Has this issue been resolved by upgrading to spark 3.0.0 in the latest release?
Hi all,
I would like to know the status of this. My use case is also same as #786 where I want to have my Spark application talk to AWS glue metastore. Can someone please guide me whether we can set the spark-hive-site using "SparkConf" or any other way?
@dfarr @batCoder95 We have a simple implementation here that we use internally, based on the changes @wweic suggested, iterating over the configmap keys. We didn't add a PR as we weren't sure it met proper code style / architecture, but it seems to work well for our use case. I'll be happy to PR this if it can help. You can try the image we built here: https://hub.docker.com/r/bbenzikry/spark-eks-operator
@batCoder95 Specifically for glue we use something like this:
apiVersion: v1
data:
hive-site.xml: |
<configuration>
<property>
<name>hive.imetastoreclient.factory.class</name>
<value>com.amazonaws.glue.catalog.metastore.AWSGlueDataCatalogHiveClientFactory</value>
</property>
<property>
<name>aws.region</name>
<value>REGION</value>
</property>
</configuration>
kind: ConfigMap
metadata:
namespace: JOB_NAMESPACE
name: spark-custom-config-map
kind: SparkApplication
metadata:
name: "whatever"
namespace: JOB_NAMESPACE
spec:
sparkConfigMap: spark-custom-config-map
...
Hi @bbenzikry,
This is really really helpful of you. I'll try out the sample implementation that you have mentioned immediately and let you know if I face any issues :)
Thanks a ton for this guidance :)
@dfarr @batCoder95 We have a simple implementation here that we use internally, based on the changes @wweic suggested, iterating over the configmap keys. We didn't add a PR as we weren't sure it met proper code style / architecture, but it seems to work well for our use case. I'll be happy to PR this if it can help. You can try the image we built here: https://hub.docker.com/r/bbenzikry/spark-eks-operator
@bbenzikry I deployed this image, it dosn't support kerberos authentication,is there any solution?Thanks!
Hi @joanjiao2016, the example image is based on the operator version available at that time - it hasn't been updated with any upstream changes ( specifically for kerberos, I saw there's still a PR open, so I really don't know the status )
@liyinan926 As this is getting some traction, do you want me to PR so it can be available upstream? Unfortunately I have limited time to iterate on this, but the latest rebase I did is available here https://github.com/bbenzikry/spark-on-k8s-operator/tree/hive-subpath-rebased - I'll appreciate you taking a look to see if it's suitable for a PR. Thanks
Hi all, Is there any way to solve this problem? MountVolume.SetUp failed for volume "spark-conf-volume" : configmap "spark-45545489ca1f20-driver-conf-map" not found
For what it's worth, we've open sourced a fully contained build and docker image for Spark 3.1.1 (with the kubernetes deps), Hadoop 3.2.0, Hive 2.3.7, and this glue client (largely building on @bbenzikry's work): https://github.com/viaduct-ai/docker-spark-k8s-aws
Hi All,
I am trying to make spark operator work with glue meta store since last 2 days without success.
I am slightly new to this domain. Can someone please provide a bit more detailed steps of any solution/workarounds of this problem?
I tried @jpugliesi image for my sparkapplication but my jobs are still failing to with MetaException
Caused by: MetaException(message:Version information not found in metastore. )
at org.apache.hadoop.hive.metastore.ObjectStore.checkSchema(ObjectStore.java:7810)
at org.apache.hadoop.hive.metastore.ObjectStore.verifySchema(ObjectStore.java:7788)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at org.apache.hadoop.hive.metastore.RawStoreProxy.invoke(RawStoreProxy.java:101)
at com.sun.proxy.$Proxy47.verifySchema(Unknown Source)
at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.getMSForConf(HiveMetaStore.java:595)
at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.getMS(HiveMetaStore.java:588)
at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.createDefaultDB(HiveMetaStore.java:655)
at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.init(HiveMetaStore.java:431)
For what it's worth, we've open sourced a fully contained build and docker image for Spark 3.1.1 (with the kubernetes deps), Hadoop 3.2.0, Hive 2.3.7, and this glue client (largely building on @bbenzikry's work): https://github.com/viaduct-ai/docker-spark-k8s-aws
Hi @jpugliesi - just wanted to say I love this. I changed our own docker builds to use a slightly modified version of your solution, and it has been a great experience. I'll be changing my spark/eks repo to reflect this as my main method. Great job.
Hello Guys,
Any workaround to this?
Thank you
@wweic do we have any traction on this? . Seems I am facing similar issue when I wanted to add few more spark conf properties to spark.properties via the configMap but though the configMap is getting mounted and adding SPARK_CONF_DIR env variable but to /etc/spark/conf folder and the driver startup log shows that its still using /opt/spark/conf/properties file instead of the configMap file path
One more observation, though SPARK_CONF_DIR is set to /etc/spark/conf in the driver pod but due to the following driver's args: which is by default loading /opt/spark/conf/spark.properties hence the configMap mounted file will not be loaded. So is there a way to either overwrite the path in args or add custom /etc/spark/conf/spark.properties file to the args properties-file list
example
args:
- driver
- '--properties-file'
- /opt/spark/conf/spark.properties
- '--class'
- org.apache.spark.examples.SparkPi
- 'local:///opt/spark/examples/jars/spark-examples_2.12-3.0.1.jar'
any directions are highly appreciated. Thanks
@wweic @bbenzikry is it possible that you could start a PR with the changes that you guys have made to get this working? We have been wanting to try this approach but do not want to use a very old version of the operator.
This issue is almost 4 years old. And while several ideas & solutions have been put forward and merged in other repositories, none of them made it to the upstream of this repository. I am in the process of learning go right now and I'd like to put forward a PR to close this issue. @liyinan926 any objections? 😄
@Sudeepam97 @tahesse If the current implementation was not problematic I would assume one of my past iterations would get an approval here, which I've requested a response to several times in the course of this thread.
We have been using our own version of the operator in production since this started - so while I personally consider any PR as constructive, I think most participants of the thread will only benefit from such a PR if it is:
Were I a maintainer I would've tried to push this through quickly, so I'm sorry in advance for everyone who has been tagging me and did not receive a prompt answer.
If anyone does decide to push a proper and extensive PR through, I'll be willing to help with CR for this, as I remember we did encounter some edge cases in our private fork
@bbenzikry Thank you for your offer in regards the CR. I think it is worthwhile to submit a PR even though it doesn't suffice all coding standards of the target repository and might not capture every edge case. With a PR in place, you give everyone the chance to iterate on it (this is my goal and hence your CR offer is very welcome!).
I have read through the PR from apache/spark mentioned in the first post of this issue. I tried to grasp the introduced changes that are incompatible with how to spark-operator on k8s was designed for sparkConfigMap
:
configMap
that populates spark.properties
(https://spark.apache.org/docs/3.2.1/configuration.html#loading-default-configurations anchor does not link anywhere signaling some inconsistencies/transition phase within spark? purely speculative on my side...) (https://github.com/apache/spark/pull/20669/files#diff-d913f075b03cf052530a8f0e76f1288bfa6c5c08332ee88a8c9b8fd3ba53bf6cR124 loads https://github.com/apache/spark/pull/20669/files#diff-f7086eaad627ffccf227afa71e9f4d0a637a53cd079b82803e5a5fcce90b1dbdR72 which points to $SPARK_CONF_DIR/spark.properties
effectively)Issue on k8s-on-spark-operator side:
configMap
name in sparkConfigMap
should load the configuration from the specified configMap
. A k8s configMap
may contain multiple files, e.g.:
apiVersion: v1
kind: ConfigMap
metadata:
name: spark-op-conf
namespace: sparkapps
data:
spark-hadoop.conf: |
spark.hadoop.fs.s3a.endpoint http://minio:9000
spark.hadoop.fs.s3a.aws.credentials.provider org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider
spark.hadoop.fs.s3a.access.key accesskey
spark.hadoop.fs.s3a.secret.key secretkey
spark.hadoop.fs.s3a.path.style.access true
spark.hadoop.fs.s3a.impl org.apache.hadoop.fs.s3a.S3AFileSystem
spark.hadoop.fs.s3a.bucket.all.committer.magic.enabled true
spark-metrics.conf: |
spark.metrics.conf.*.sink.graphite.class org.apache.spark.metrics.sink.GraphiteSink
spark.metrics.conf.*.sink.graphite.host spark-dashboard-influx.spark-operator
spark.metrics.conf.*.sink.graphite.port 1337
spark.metrics.conf.*.source.jvm.class org.apache.spark.metrics.source.JvmSource
spark.metrics.appStatusSource.enabled true
mountPath
. It is unclear how to handle (discover, parse, validate, merge in case of overlapping keys) these files.spark-submit
from https://github.com/apache/spark/pull/20669/files#diff-d913f075b03cf052530a8f0e76f1288bfa6c5c08332ee88a8c9b8fd3ba53bf6cR124 is very explicit about loading spark.properties
from SPARK_CONF_DIR=/opt/spark/conf
.The default Spark properties file is $SPARK_HOME/conf/spark-defaults.conf that could be overriden using spark-submit's --properties-file command-line option.
sparkConfigMap
that contains a spark.properties
, I receive exec /usr/bin/tini -g -- /opt/spark/bin/spark-submit […] --properties-file /opt/spark/conf/spark.properties […]
in the logs.How does spark-on-k8s-operator handle sparkConf
(which is considered the workaround to-date https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/issues/806#issuecomment-585983342) compared to sparkConfigMap
:
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/74ea1c86c9a595e6c1c8a0c2c40a50bf89288ede/pkg/controller/sparkapplication/submission.go#L136-L142
Given all the collected information, I'd like to move forward with the following design:
spark.properties
(working draft filename) from a the sparkConfigMap
(in /var or /tmp as we view this information ephemeral) and patch the existing spark.properties
prior to container submission https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/74ea1c86c9a595e6c1c8a0c2c40a50bf89288ede/pkg/controller/sparkapplication/controller.go#L681$SPARK_CONFI_DIR/spark.properties
(spark builtin) < sparkConfigMap
(k8s spec) < sparkConf
(k8s spec [spark-submit
args]) < SparkConf
(executor code)My reasoning does not consider runtime related order of execution issues, I'm glad for helpful pointers in that regards. So far, I roughly understand that changing the SPARK_CONF_DIR
env var has no impact on https://github.com/apache/spark/pull/20669/files#diff-8b8024410a4c9223ba294eb313d40cd4389447cb601b8e55bd2babf8f56cc87fR116-R136
Based on my findings, I will try to draft a first version of a PR which I will test for my use cases and the edge cases I can think of. As I am new to Go, I'd be happy for hints on how to optimize the development workflow. Links to guides are sufficient. Thank you!
Disclaimer: must not be complete, feel free to add what you think I missed.
@tahesse Thanks for your thoughts - I wouldn't put that much emphasis on my specific point on best practices, I want this to be something that is properly maintained, has the proper follow-ups and the right tests to alert to future issues while accommodating several needs that were raised here and in other relevant issues.
I didn't quite understand how your suggestion aligns with some of what we're trying to achieve besides patching spark.properties specifically. Adding files on the same folder / mount is important for dependencies such as hive-site.xml - which the subpath approach solves inherently and was the path of least resistance when we first encountered it due to the read only way the internal CM is mounted by default and those pesky constants in the spark codebase. Did you take look at the subpath implementation? Sorry if I missed anything that did touch on that.
I will say that for me, from the user / DX perspective, the optimal approach should include:
config.DefaultSparkConfDir
on the operator + required changes in other codebases ) spark.properties
selected for patch propagation sparkConfigMap
and sparkConf
into the same mounted file instead of --conf
appendIf I understand correctly, and please do correct me if I'm wrong, your solution touches on 4,5 and some of 2 Obviously some of my suggestions here require a future spark release and will need to handle any compatibility issues in the operator and spark codebases.
I would love to brainstorm a bit more in a private chat so we can correlate efforts and see if we can create the right PRs as required for overarching goals surrounding the original issue while maintaining compatibility and allowing these items to be gradually implemented ( I'm beni on Keybase )
@bbenzikry Sorry, I don't have keybase but I agree that a private chat for brainstorming makes more sense. Let me see if I can create an account for this very purpose.
- Being able to properly control SPARK_CONF_DIR and have it work as expected ( changing and overriding config.DefaultSparkConfDir on the operator + required changes in other codebases )
I think the current design with the mutating webhook will not work for updating SPARK_CONF_DIR
after spark-submit
has been run by the operator has been run. AFAIU the SPARK_CONF_DIR
for the driver is set initially here on the spark side.
- Allow for multiple files in sparkConfigMap, with spark.properties selected for patch propagation
Out of scope at this point because IMO it is incompatible with the spark semantics and complicates things at this early stage. However, a single sparkConfigMap
configuration has the power to install multiple files to SPARK_CONF_DIR
.
- Allow additional CMs / secrets to be added as files to the proper SPARK_CONF_DIR
Out of scope at this time because the spec sparkConfigMap
should be sufficient to add a custom spark-defaults.conf
and spark.properties
whereas the latter has higher precedence according to the spark docs.
Secrets is a topic, I'd like to tackle in a next iteration. I don't want to put secrets in the CM.
- Be able to properly patch defaults, the file definition within sparkConfigMap and sparkConf into the same mounted file instead of --conf append
Yes!
- Some measure of control over any patch sequence implemented ( even if the default order is implemented as you suggest, advanced users should probably have an option to control precedence )
For a second iteration, yes. I'd drop it in the first iteration to not further complicate things.
- Maybe ( that's probably just a dream of mine ) try and find a way to help future generations in other issues that currently require similar code changes ( at least until a default implementation path is reached ), such as allowing to run arbitrary patches ( similar to Kustomize ) a-la flux for different "hook" points
I will try to document everything I learned and add to the docs/developer-guide.md
.
@yuchaoran2011 You're mentioned as maintainer in the helm chart. Any points from your side regarding @bbenzikry valid concerns?
This issue has been automatically marked as stale because it has not had recent activity. It will be closed if no further activity occurs. Thank you for your contributions.
PR apache/spark#20669 introduced a ConfigMap carrying Spark configuration properties in a file for the driver. The environment variable SPARK_CONF_DIR is set to point to the mount path /opt/spark/conf of the ConfigMap. This is in conflict with what spec.sparkConfigMap is designed to do. We need to find a solution to work with the internally created and mounted ConfigMap.