Z1Wu opened a new issue, #5625: URL: https://github.com/apache/kyuubi/issues/5625
### Code of Conduct - [X] I agree to follow this project's [Code of Conduct](https://www.apache.org/foundation/policies/conduct) ### Search before asking - [X] I have searched in the [issues](https://github.com/apache/kyuubi/issues?q=is%3Aissue) and found no similar issues. ### Describe the bug # Problem When submiting Spark Engine in cluster mode with k8s, SparkSQLEngine running in k8s pod throws execption as follow( detailed log see Engine Log Section) ``` Caused by: java.lang.RuntimeException: java.io.IOException: kyuubi.ha.zookeeper.auth.keytab: /home/hadoop/keytab/hive.keytab does not exists ``` Zookeeper keytab file is stored in kyuubi server host. When submit spark application, this keytab file will added to spark runtime using `--files`. https://github.com/apache/kyuubi/blob/fdacd239647a37c3ce993c517575b3c254a33681/kyuubi-server/src/main/scala/org/apache/kyuubi/engine/spark/SparkProcessBuilder.scala#L193-L205 But when creating zookeeper client in SparkSQLEngine, current implementation try to find zookeeper keytab file from `current work directory` and `full path`. - `Full path` will not work due to that is the path of kyuubi server host, while - `Current work directory` will not work either. In k8s cluster mode, driver pod will run with `SPARK_LOCAL_DIRS` by default setting to a tmp directory and file added by `--files` will be fetched into that direcotory. https://github.com/apache/kyuubi/blob/fdacd239647a37c3ce993c517575b3c254a33681/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/zookeeper/ZookeeperClientProvider.scala#L146-L151 IMO, `ZookeeperClientProvider` should read files uploaded by spark-submit using `SparkFiles` utils. However this method will introduce spark dependency to `kyuubi-ha` module. Currently, my work-aournd looks like blow: ``` scala // rewrite HA_ZK_AUTH_KEYTAB with ture path obtain from `SparkFiles` when running spark with k8s-cluster mode // org.apache.kyuubi.engine.spark.SparkSQLEngine#createSpark(): val zkKeytabPath = kyuubiConf.get(HA_ZK_AUTH_KEYTAB) // if submit in cluster mode with k8s master rewrite SparkFile related settings if (zkKeytabPath.isDefined && session.sparkContext.master.contains("k8s") && _sparkConf.getBoolean("spark.kubernetes.submitInDriver", defaultValue = false)) { info("rewrite zk auth path when in k8s-cluster mode") val relativeFileName = new File(zkKeytabPath.get).getName val uploadedFilePath = SparkFiles.get(relativeFileName) if (new File(uploadedFilePath).exists()) { kyuubiConf.set(HA_ZK_AUTH_KEYTAB.key, uploadedFilePath) } else { error( s"unable to get zk_auth_keytab from uploaded files, " + s"when setting (${HA_ZK_AUTH_KEYTAB.key} = ${zkKeytabPath})" + s"and uploaded file path ${uploadedFilePath}") } } ``` ### Affects Version(s) master ### Kyuubi Server Log Output ```logtalk 2023-11-06 09:53:34.103 INFO KyuubiSessionManager-exec-pool: Thread-111 org.apache.kyuubi.engine.EngineRef: Launching engine: /home/hadoop/spark-3.2.2-for-k8s/bin/spark-submit \ --class org.apache.kyuubi.engine.spark.SparkSQLEngine \ --conf spark.hive.server2.proxy.user=tmp_user \ --conf spark.kyuubi.backend.engine.exec.pool.size=600 \ --conf spark.kyuubi.backend.engine.exec.pool.wait.queue.size=200 \ --conf spark.kyuubi.backend.server.exec.pool.size=600 \ --conf spark.kyuubi.backend.server.exec.pool.wait.queue.size=200 \ --conf spark.kyuubi.client.ipAddress=10.90.132.92 \ --conf spark.kyuubi.engine.credentials=* \ --conf spark.kyuubi.engine.engineLog.path=* \ --conf spark.kyuubi.engine.event.loggers=CUSTOM,SPARK \ --conf spark.kyuubi.engine.operation.log.dir.root=* \ --conf spark.kyuubi.engine.submit.time=1699235614058 \ --conf spark.kyuubi.frontend.thrift.max.worker.threads=5000 \ --conf spark.kyuubi.frontend.thrift.min.worker.threads=200 \ --conf spark.kyuubi.ha.addresses=fenghuangzk.zk01.gdc.x.netease.com,fenghuangzk.zk02.gdc.x.netease.com,fenghuangzk.zk03.gdc.x.netease.com \ --conf spark.kyuubi.ha.enabled=true \ --conf spark.kyuubi.ha.engine.ref.id=0de43d7e-0ba5-4546-9062-6ae934c223b6 \ --conf spark.kyuubi.ha.namespace=* \ --conf spark.kyuubi.ha.zookeeper.auth.keytab=/home/hadoop/keytab/hive.keytab \ --conf spark.kyuubi.ha.zookeeper.auth.principal=hive/[email protected] \ --conf spark.kyuubi.ha.zookeeper.auth.type=KERBEROS \ --conf spark.kyuubi.ha.zookeeper.client.port=2181 \ --conf spark.kyuubi.ha.zookeeper.engine.auth.type=KERBEROS \ --conf spark.kyuubi.ha.zookeeper.session.timeout=600000 \ --conf spark.kyuubi.metrics.prometheus.port=46031 \ --conf spark.kyuubi.metrics.reporters=PROMETHEUS \ --conf spark.kyuubi.server.ipAddress=10.191.55.182 \ --conf spark.kyuubi.session.connection.url=gdc-sparkserver09-fenghuang.i.nease.net:36031 \ --conf spark.kyuubi.session.engine.idle.timeout=PT30M \ --conf spark.kyuubi.session.engine.initialize.timeout=PT4M \ --conf spark.kyuubi.session.engine.spark.showProgress=true \ --conf spark.kyuubi.session.idle.timeout=PT60M \ --conf spark.kyuubi.session.real.user=hadoop \ --conf spark.app.name=* \ --conf spark.driver.extraClassPath=* \ --conf spark.driver.extraJavaOptions=-Djava.library.path=/home/hadoop/hadoop/lib/native -Djavax.security.auth.useSubjectCredsOnly=false \ --conf spark.executor.extraJavaOptions=-Djava.library.path=/home/hadoop/hadoop/lib/native \ --conf spark.files=/home/hadoop/keytab/hive.keytab \ --conf spark.kubernetes.driver.label.kyuubi-unique-tag=0de43d7e-0ba5-4546-9062-6ae934c223b6 \ --conf spark.kubernetes.driver.pod.name=kyuubi-kyuubi-user-spark-sql-dep214-sdc-default-0de43d7e-0ba5-4546-9062-6ae934c223b6-0de43d7e-0ba5-4546-9062-6ae934c223b6-driver \ --conf spark.kubernetes.executor.podNamePrefix=kyuubi-kyuubi-user-spark-sql-dep214-sdc-default-0de43d7e-0ba5-4546-9062-6ae934c223b6-0de43d7e-0ba5-4546-9062-6ae934c223b6 \ --conf spark.kubernetes.driverEnv.SPARK_USER_NAME=tmp_user \ --conf spark.executorEnv.SPARK_USER_NAME=tmp_user \ --proxy-user tmp_user /home/hadoop/tmp1/apache-kyuubi-1.9.0-SNAPSHOT-bin/externals/engines/spark/kyuubi-spark-sql-engine_2.12-1.9.0-SNAPSHOT.jar ``` ### Kyuubi Engine Log Output ```logtalk 23/11/06 01:54:27 ERROR SparkSQLEngine: Spark application name: kyuubi_USER_SPARK_SQL_tmp_user_default_0de43d7e-0ba5-4546-9062-6ae934c223b6 application ID: spark-99c1a00e83ac4f54844a9b641e9c535e application tags: application web UI: http://spark-9460878ba255da35-driver-svc.spark.svc:4040 master: k8s://https://*:6443 version: 3.2.2_GDC_V1.0.9-SNAPSHOT driver: [cpu: 1, mem: 10g] executor: [cpu: 4, mem: 6g, maxNum: 1] Start time: Mon Nov 06 01:53:53 UTC 2023 End time: Mon Nov 06 01:54:27 UTC 2023 User: tmp_user (shared mode: USER) State: LATENT Diagnostic: Failed to initialize SparkSQLEngine: org.apache.kyuubi.ha.client.zookeeper.ZookeeperDiscoveryClient must extend of org.apache.kyuubi.ha.client.DiscoveryClient org.apache.kyuubi.KyuubiException: Failed to initialize SparkSQLEngine: org.apache.kyuubi.ha.client.zookeeper.ZookeeperDiscoveryClient must extend of org.apache.kyuubi.ha.client.DiscoveryClient at org.apache.kyuubi.engine.spark.SparkSQLEngine$.$anonfun$startEngine$1(SparkSQLEngine.scala:313) at org.apache.kyuubi.engine.spark.SparkSQLEngine$.$anonfun$startEngine$1$adapted(SparkSQLEngine.scala:298) at scala.Option.foreach(Option.scala:407) at org.apache.kyuubi.engine.spark.SparkSQLEngine$.startEngine(SparkSQLEngine.scala:298) at org.apache.kyuubi.engine.spark.SparkSQLEngine$.main(SparkSQLEngine.scala:360) at org.apache.kyuubi.engine.spark.SparkSQLEngine.main(SparkSQLEngine.scala) 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.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52) at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:983) at org.apache.spark.deploy.SparkSubmit.$anonfun$submit$2(SparkSubmit.scala:185) at org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:62) at org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:61) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1857) at org.apache.spark.deploy.SparkHadoopUtil.runAsSparkUser(SparkHadoopUtil.scala:61) at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:185) at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:231) at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90) at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1071) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1080) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) Caused by: org.apache.kyuubi.KyuubiException: org.apache.kyuubi.ha.client.zookeeper.ZookeeperDiscoveryClient must extend of org.apache.kyuubi.ha.client.DiscoveryClient at org.apache.kyuubi.util.ClassUtils$.createInstance(ClassUtils.scala:44) at org.apache.kyuubi.ha.client.DiscoveryClientProvider$.createDiscoveryClient(DiscoveryClientProvider.scala:52) at org.apache.kyuubi.ha.client.ServiceDiscovery.initialize(ServiceDiscovery.scala:54) at org.apache.kyuubi.service.CompositeService.$anonfun$initialize$1(CompositeService.scala:40) at org.apache.kyuubi.service.CompositeService.$anonfun$initialize$1$adapted(CompositeService.scala:40) at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) at org.apache.kyuubi.service.CompositeService.initialize(CompositeService.scala:40) at org.apache.kyuubi.service.AbstractFrontendService.initialize(AbstractFrontendService.scala:41) at org.apache.kyuubi.service.TBinaryFrontendService.initialize(TBinaryFrontendService.scala:129) at org.apache.kyuubi.service.CompositeService.$anonfun$initialize$1(CompositeService.scala:40) at org.apache.kyuubi.service.CompositeService.$anonfun$initialize$1$adapted(CompositeService.scala:40) at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) at org.apache.kyuubi.service.CompositeService.initialize(CompositeService.scala:40) at org.apache.kyuubi.service.Serverable.initialize(Serverable.scala:46) at org.apache.kyuubi.engine.spark.SparkSQLEngine.initialize(SparkSQLEngine.scala:67) at org.apache.kyuubi.engine.spark.SparkSQLEngine$.$anonfun$startEngine$1(SparkSQLEngine.scala:309) ... 24 more Caused by: java.lang.RuntimeException: java.io.IOException: kyuubi.ha.zookeeper.auth.keytab: /home/hadoop/keytab/hive.keytab does not exists at org.apache.kyuubi.util.reflect.DynConstructors$Ctor.newInstance(DynConstructors.java:76) at org.apache.kyuubi.util.ClassUtils$.createInstance(ClassUtils.scala:41) ... 43 more Caused by: java.io.IOException: kyuubi.ha.zookeeper.auth.keytab: /home/hadoop/keytab/hive.keytab does not exists at org.apache.kyuubi.ha.client.zookeeper.ZookeeperClientProvider$.setupZkAuth$1(ZookeeperClientProvider.scala:108) at org.apache.kyuubi.ha.client.zookeeper.ZookeeperClientProvider$.setUpZooKeeperAuth(ZookeeperClientProvider.scala:141) at org.apache.kyuubi.ha.client.zookeeper.ZookeeperClientProvider$.buildZookeeperClient(ZookeeperClientProvider.scala:46) at org.apache.kyuubi.ha.client.zookeeper.ZookeeperDiscoveryClient.<init>(ZookeeperDiscoveryClient.scala:51) at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at org.apache.kyuubi.util.reflect.DynConstructors$Ctor.newInstanceChecked(DynConstructors.java:54) at org.apache.kyuubi.util.reflect.DynConstructors$Ctor.newInstance(DynConstructors.java:71) ... 44 more ``` ### Kyuubi Server Configurations ```yaml kyuubi.server.name kyuubi_abcsql_def_mid_36031 kyuubi.authentication KERBEROS kyuubi.kinit.principal hive/hiveserver@X kyuubi.kinit.keytab /home/hadoop/keytab/hive.keytab kyuubi.kinit.interval PT24H kyuubi.ha.addresses clusterzk.zk01.abc.x.netease.com,clusterzk.zk02.abc.x.netease.com,clusterzk.zk03.abc.x.netease.com kyuubi.ha.enabled true kyuubi.ha.zookeeper.client.port 2181 kyuubi.ha.zookeeper.session.timeout 600000 kyuubi.ha.namespace abc-sparkserver09-cluster-kyuubi_abcsql_def_mid_36031 kyuubi.ha.zookeeper.auth.type KERBEROS kyuubi.ha.zookeeper.engine.auth.type KERBEROS kyuubi.ha.zookeeper.auth.keytab /home/hadoop/keytab/hive.keytab kyuubi.ha.zookeeper.auth.principal hive/hiveserver@X kyuubi.frontend.thrift.binary.bind.port 36031 kyuubi.frontend.thrift.max.worker.threads 5000 kyuubi.frontend.thrift.min.worker.threads 200 kyuubi.engine.event.loggers CUSTOM,SPARK kyuubi.engine.operation.log.dir.root /home/hadoop/logs/kyuubi/kyuubi_abcsql_def_mid_36031_engineop kyuubi.session.engine.initialize.timeout 180000 kyuubi.session.engine.idle.timeout PT30M kyuubi.session.engine.initialize.timeout PT4M kyuubi.session.engine.spark.showProgress true kyuubi.session.idle.timeout PT60M kyuubi.operation.log.dir.root /home/hadoop/logs/kyuubi/kyuubi_abcsql_def_mid_36031_op kyuubi.metrics.reporters PROMETHEUS kyuubi.metrics.prometheus.port 46031 spark.driver.extraClassPath /home/hadoop/abcconf/hive/cluster/def_wd kyuubi.backend.engine.exec.pool.size 600 kyuubi.backend.engine.exec.pool.wait.queue.size 200 kyuubi.backend.server.exec.pool.size 600 kyuubi.backend.server.exec.pool.wait.queue.size 200 # spark driver jvm option spark.driver.extraJavaOptions -Djava.library.path=/home/hadoop/hadoop/lib/native -Djavax.security.auth.useSubjectCredsOnly=false # spark executor jvm option spark.executor.extraJavaOptions -Djava.library.path=/home/hadoop/hadoop/lib/native -Dabc_KYUUBI_USER=hadoop ``` ### Kyuubi Engine Configurations ```yaml spark.eventLog.enabled true spark.history.kerberos.enabled true spark.history.kerberos.principal hadoop/sparkhistoryserver@X spark.eventLog.dir hdfs://cluster/tmp/spark-events spark.history.fs.logDirectory hdfs://cluster/tmp/spark-events spark.yarn.historyServer.address gdc-rm02-cluster.i.nease.net:18080 spark.driver.memory 10g spark.master k8s://https://*:6443 # FOR K8S related spark.submit.deployMode cluster spark.kubernetes.file.upload.path hdfs://cluster/tmp/spark_on_k8s spark.kubernetes.authenticate.driver.serviceAccountName spark spark.kubernetes.namespace spark spark.kubernetes.container.image apache/spark:v3.3.2 spark.kubernetes.driver.podTemplateFile /home/hadoop/spark_k8s_conf/spark_driver_template.yaml spark.kubernetes.driverEnv.HADOOP_HOME /home/hadoop/hadoop330 spark.kubernetes.executor.deleteOnTermination false # spark.kubernetes.driverEnv.SPARK_SUBMIT_OPTS -agentlib:jdwp=transport=dt_socket,suspend=y,server=y,address=5005 spark.executor.memory 4g spark.executor.cores 2 spark.executor.instances 1 ``` ### Additional context _No response_ ### Are you willing to submit PR? - [X] Yes. I would be willing to submit a PR with guidance from the Kyuubi community to fix. - [ ] No. I cannot submit a PR at this time. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
