[ 
https://issues.apache.org/jira/browse/SPARK-21819?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16138302#comment-16138302
 ] 

Sean Owen commented on SPARK-21819:
-----------------------------------

Would it suffice to add this configuration somehow after Spark initializes but 
before it is required?

>  UserGroupInformation initialization in SparkHadoopUtilwill overwrite user 
> config
> ---------------------------------------------------------------------------------
>
>                 Key: SPARK-21819
>                 URL: https://issues.apache.org/jira/browse/SPARK-21819
>             Project: Spark
>          Issue Type: Bug
>          Components: Java API, YARN
>    Affects Versions: 2.1.0, 2.1.1
>         Environment: Ubuntu14.04
> Spark2.10/2.11 (I checked the github of 2.20 , it exist there as well)
> Cluster mode: Yarn client 
>            Reporter: Keith Sun
>
> When  submit job in Java or Scala code to ,the initialization of 
> SparkHadoopUtil will trigger the configuration overwritten in UGI which may 
> not be expected if the UGI has already been initialized by customized xmls 
> which are not on the classpath (like the cfg4j , which could set conf from 
> github code, a database etc). 
> {code:java}
> //it will overwrite the UGI conf which has already been initialized
> class SparkHadoopUtil extends Logging {
>   private val sparkConf = new SparkConf(false).loadFromSystemProperties(true)
>   val conf: Configuration = newConfiguration(sparkConf)
>   UserGroupInformation.setConfiguration(conf)
> {code}
> My scenario : My yarn cluster is kerberized, my configuration is set to use 
> kerberos for hadoop security. While, after the initialzation of 
> SparkHadoopUtil , the authentiationMethod in UGI is updated to "simple"(my 
> xmls not on the classpath), which lead to the failure like below :
> {code:java}
> 933453 [main] INFO  org.apache.spark.SparkContext  - Successfully stopped 
> SparkContext
> Exception in thread "main" org.apache.hadoop.security.AccessControlException: 
> SIMPLE authentication is not enabled.  Available:[TOKEN, KERBEROS]
>       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.hadoop.yarn.ipc.RPCUtil.instantiateException(RPCUtil.java:53)
>       at 
> org.apache.hadoop.yarn.ipc.RPCUtil.unwrapAndThrowException(RPCUtil.java:104)
>       at 
> org.apache.hadoop.yarn.api.impl.pb.client.ApplicationClientProtocolPBClientImpl.getClusterMetrics(ApplicationClientProtocolPBClientImpl.java:209)
>       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.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:256)
>       at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:104)
>       at com.sun.proxy.$Proxy16.getClusterMetrics(Unknown Source)
>       at 
> org.apache.hadoop.yarn.client.api.impl.YarnClientImpl.getYarnClusterMetrics(YarnClientImpl.java:501)
>       at 
> org.apache.spark.deploy.yarn.Client$$anonfun$submitApplication$1.apply(Client.scala:154)
>       at 
> org.apache.spark.deploy.yarn.Client$$anonfun$submitApplication$1.apply(Client.scala:154)
>       at org.apache.spark.internal.Logging$class.logInfo(Logging.scala:54)
>       at org.apache.spark.deploy.yarn.Client.logInfo(Client.scala:60)
>       at 
> org.apache.spark.deploy.yarn.Client.submitApplication(Client.scala:153)
>       at 
> org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend.start(YarnClientSchedulerBackend.scala:56)
>       at 
> org.apache.spark.scheduler.TaskSchedulerImpl.start(TaskSchedulerImpl.scala:149)
>       at org.apache.spark.SparkContext.<init>(SparkContext.scala:497)
>       at org.apache.spark.SparkContext$.getOrCreate(SparkContext.scala:2258)
>       at 
> org.apache.spark.sql.SparkSession$Builder$$anonfun$8.apply(SparkSession.scala:831)
>       at 
> org.apache.spark.sql.SparkSession$Builder$$anonfun$8.apply(SparkSession.scala:823)
>       at scala.Option.getOrElse(Option.scala:121)
>       at 
> org.apache.spark.sql.SparkSession$Builder.getOrCreate(SparkSession.scala:823)
>       at SparkTest.SparkEAZDebug.main(SparkEAZDebug.java:84)
> Caused by: 
> org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.AccessControlException):
>  SIMPLE authentication is not enabled.  Available:[TOKEN, KERBEROS]
>       at org.apache.hadoop.ipc.Client.call(Client.java:1426)
>       at org.apache.hadoop.ipc.Client.call(Client.java:1363)
>       at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
>       at com.sun.proxy.$Proxy15.getClusterMetrics(Unknown Source)
>       at 
> org.apache.hadoop.yarn.api.impl.pb.client.ApplicationClientProtocolPBClientImpl.getClusterMetrics(ApplicationClientProtocolPBClientImpl.java:206)
>       ... 22 more
> {code}
> Sample code :
> {code:java}
>               Configuration hc = new  Configuration(false);
>               String yarnxml=String.format("%s/%s", 
> ConfigLocation,"yarn-site.xml");
>               String corexml=String.format("%s/%s", 
> ConfigLocation,"core-site.xml");
>               String hdfsxml=String.format("%s/%s", 
> ConfigLocation,"hdfs-site.xml");
>               String hivexml=String.format("%s/%s", 
> ConfigLocation,"hive-site.xml");
>               hc.addResource(yarnxml);
>               hc.addResource(corexml);
>               hc.addResource(hdfsxml);
>               hc.addResource(hivexml);
>               SparkConf sc = new SparkConf(true);
>               hc.forEach(entry-> {                                    
>                               if(entry.getKey().startsWith("hive")) {
>                                       sc.set(entry.getKey(), 
> entry.getValue());
>                               }else {
>                                       sc.set("spark.hadoop."+entry.getKey(), 
> entry.getValue());
>                               }
>                       });
>           UserGroupInformation.setConfiguration(hc);
>           UserGroupInformation.loginUserFromKeytab(Principal, Keytab);        
>         
>               SparkSession sparkSessesion= SparkSession
>                               .builder()
>                               .master("yarn-client") //"yarn-client", "local"
>                               .config(sc)
>                               .appName(SparkEAZDebug.class.getName())
>                               .enableHiveSupport() 
>                               .getOrCreate();
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to