yaooqinn opened a new pull request #25609: [SPARK-28896][K8S] Download hadoop 
configurations from k8s configmap if the client process has files to upload
URL: https://github.com/apache/spark/pull/25609
 
 
   
   
   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: 
https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: 
https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., 
'[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a 
faster review.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section 
is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster 
reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class 
hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other 
DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   Changes in this pr will works in spark-submit client process as the 
following steps,
   
   1. if there are some jars/files to upload
   2. if the spark.kubernetes.file.upload.path is specified
   3. try fetch hadoop configmap from k8s cluster via 
spark.kubernetes.hadoop.configMapName
   4.  apply all hadoop configmap kvs to a new create hadoop configuration 
instance
   5. apply the hadoop configs from cmd on top of it.
   6. reset auth infor with the new hadoop configuration
   7. if is secured now but the original ugi is not, try login in now,(without 
HADOOP_CONF_DIR, the login-in logic  will not actually done)
   8. then it goes to the former upload file logic
   
   Additionally, it used to create hadoop configuration and filesystem per 
file, this pr change these to lazy val to create only once or less.
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   The *BasicDriverFeatureStep* for Spark on Kubernetes will upload the 
files/jars specified by --files/–jars to a hadoop compatible file system 
configured by spark.kubernetes.file.upload.path. While using HADOOP_CONF_DIR, 
the spark-submit process can recognize the file system, but when using 
spark.kubernetes.hadoop.configMapName which only will be mount on the Pods not 
applied back to our client process. 
    
   ```scala
    Kent@KentsMacBookPro  
~/Documents/spark-on-k8s/spark-3.0.0-SNAPSHOT-bin-2.7.3  bin/spark-submit 
--conf spark.kubernetes.file.upload.path=hdfs://hz-cluster10/user/kyuubi/udf 
--jars 
/Users/Kent/Documents/spark-on-k8s/spark-3.0.0-SNAPSHOT-bin-2.7.3/hadoop-lzo-0.4.20-SNAPSHOT.jar
 --conf spark.kerberos.keytab=/Users/Kent/Downloads/kyuubi.keytab --conf 
spark.kerberos.principal=kyuubi/[email protected] --conf  
spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf  --name hehe --deploy-mode 
cluster --class org.apache.spark.examples.HdfsTest   
local:///opt/spark/examples/jars/spark-examples_2.12-3.0.0-SNAPSHOT.jar 
hdfs://hz-cluster10/user/kyuubi/hive_db/kyuubi.db/hive_tbl
   Listening for transport dt_socket at address: 50014
   # spark.master=k8s://https://10.120.238.100:7443
   19/08/27 17:21:06 WARN NativeCodeLoader: Unable to load native-hadoop 
library for your platform... using builtin-java classes where applicable
   Using Spark's default log4j profile: 
org/apache/spark/log4j-defaults.properties
   19/08/27 17:21:07 INFO SparkKubernetesClientFactory: Auto-configuring K8S 
client using current context from users K8S config file
   Listening for transport dt_socket at address: 50014
   Exception in thread "main" org.apache.spark.SparkException: Uploading file 
/Users/Kent/Documents/spark-on-k8s/spark-3.0.0-SNAPSHOT-bin-2.7.3/hadoop-lzo-0.4.20-SNAPSHOT.jar
 failed...
        at 
org.apache.spark.deploy.k8s.KubernetesUtils$.uploadFileUri(KubernetesUtils.scala:287)
        at 
org.apache.spark.deploy.k8s.KubernetesUtils$.$anonfun$uploadAndTransformFileUris$1(KubernetesUtils.scala:246)
        at 
scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:237)
        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 scala.collection.TraversableLike.map(TraversableLike.scala:237)
        at scala.collection.TraversableLike.map$(TraversableLike.scala:230)
        at scala.collection.AbstractTraversable.map(Traversable.scala:108)
        at 
org.apache.spark.deploy.k8s.KubernetesUtils$.uploadAndTransformFileUris(KubernetesUtils.scala:245)
        at 
org.apache.spark.deploy.k8s.features.BasicDriverFeatureStep.$anonfun$getAdditionalPodSystemProperties$1(BasicDriverFeatur#
 spark.master=k8s://https://10.120.238.100:7443
   eStep.scala:165)
        at scala.collection.immutable.List.foreach(List.scala:392)
        at 
org.apache.spark.deploy.k8s.features.BasicDriverFeatureStep.getAdditionalPodSystemProperties(BasicDriverFeatureStep.scala:163)
        at 
org.apache.spark.deploy.k8s.submit.KubernetesDriverBuilder.$anonfun$buildFromFeatures$3(KubernetesDriverBuilder.scala:60)
        at 
scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126)
        at 
scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122)
        at scala.collection.immutable.List.foldLeft(List.scala:89)
        at 
org.apache.spark.deploy.k8s.submit.KubernetesDriverBuilder.buildFromFeatures(KubernetesDriverBuilder.scala:58)
        at 
org.apache.spark.deploy.k8s.submit.Client.run(KubernetesClientApplication.scala:101)
        at 
org.apache.spark.deploy.k8s.submit.KubernetesClientApplication.$anonfun$run$10(KubernetesClientApplication.scala:236)
        at 
org.apache.spark.deploy.k8s.submit.KubernetesClientApplication.$anonfun$run$10$adapted(KubernetesClientApplication.scala:229)
        at org.apache.spark.util.Utils$.tryWithResource(Utils.scala:2567)
        at 
org.apache.spark.deploy.k8s.submit.KubernetesClientApplication.run(KubernetesClientApplication.scala:229)
        at 
org.apache.spark.deploy.k8s.submit.KubernetesClientApplication.start(KubernetesClientApplication.scala:198)
        at 
org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:920)
        at 
org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:179)
        at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:202)
        at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:89)
        at 
org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:999)
        at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1008)
        at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
   Caused by: java.lang.IllegalArgumentException: 
java.net.UnknownHostException: hz-cluster10
        at 
org.apache.hadoop.security.SecurityUtil.buildTokenService(SecurityUtil.java:378)
        at 
org.apache.hadoop.hdfs.NameNodeProxies.createNonHAProxy(NameNodeProxies.java:310)
        at 
org.apache.hadoop.hdfs.NameNodeProxies.createProxy(NameNodeProxies.java:176)
        at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:678)
        at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:619)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:149)
        at 
org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2669)
        at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:94)
        at 
org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2703)
        at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2685)
        at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:373)
        at org.apache.spark.util.Utils$.getHadoopFileSystem(Utils.scala:1881)
        at 
org.apache.spark.deploy.k8s.KubernetesUtils$.uploadFileUri(KubernetesUtils.scala:278)
        ... 30 more
   Caused by: java.net.UnknownHostException: hz-cluster10
        ... 43 more
   ```
   
   Other related spark configurations
   ```
   spark.master=k8s://https://10.120.238.100:7443
   # spark.master=k8s://https://10.120.238.253:7443
   
spark.kubernetes.container.image=harbor-inner.sparkonk8s.netease.com/tenant1-project1/spark:v3.0.0-20190813
   # 
spark.kubernetes.driver.container.image=harbor-inner.sparkonk8s.netease.com/tenant1-project1/spark:v3.0.0-20190813
   # 
spark.kubernetes.executor.container.image=harbor-inner.sparkonk8s.netease.com/tenant1-project1/spark:v3.0.0-20190813
   spark.executor.instanses=5
   spark.kubernetes.namespace=ns1
   spark.kubernetes.container.image.pullSecrets=mysecret
   spark.kubernetes.hadoop.configMapName=hz10-hadoop-dir
   spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf
   spark.kerberos.principal=kyuubi/[email protected]
   spark.kerberos.keytab=/Users/Kent/Downloads/kyuubi.keytab
   ```
   
   
   ### Does this PR introduce any user-facing change?
   <!--
   If yes, please clarify the previous behavior and the change this PR proposes 
- provide the console output, description and/or an example to show the 
behavior difference if possible.
   If no, write 'No'.
   -->
   
   I guess this pr will now use config map of hadoop from k8s cluster to the 
local client process if there are files to upload to hadoop.
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some 
test cases that check the changes thoroughly including negative and positive 
cases if possible.
   If it was tested in a way different from regular unit tests, please clarify 
how you tested step by step, ideally copy and paste-able, so that other 
reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why 
it was difficult to add.
   -->
   
   manually tested with spark + k8s cluster + standalone kerberized hdfs cluster

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to