[ https://issues.apache.org/jira/browse/SPARK-10181?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967643#comment-14967643 ]
Yu Gao commented on SPARK-10181: -------------------------------- I reproduced this issue. Some hive metastore client threads try to read local ticket cache credentials instead of the specified principal and keytab. If there is no valid ticket cache for the unix user account running the code, following exception will be thrown - ERROR transport.TSaslTransport: SASL negotiation failure javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)] at com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:212) at org.apache.thrift.transport.TSaslClientTransport.handleSaslStartMessage(TSaslClientTransport.java:94) at org.apache.thrift.transport.TSaslTransport.open(TSaslTransport.java:271) at org.apache.thrift.transport.TSaslClientTransport.open(TSaslClientTransport.java:37) at org.apache.hadoop.hive.thrift.client.TUGIAssumingTransport$1.run(TUGIAssumingTransport.java:52) at org.apache.hadoop.hive.thrift.client.TUGIAssumingTransport$1.run(TUGIAssumingTransport.java:49) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:415) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628) at org.apache.hadoop.hive.thrift.client.TUGIAssumingTransport.open(TUGIAssumingTransport.java:49) at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.open(HiveMetaStoreClient.java:420) at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.<init>(HiveMetaStoreClient.java:236) at org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient.<init>(SessionHiveMetaStoreClient.java:74) at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:526) at org.apache.hadoop.hive.metastore.MetaStoreUtils.newInstance(MetaStoreUtils.java:1521) at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.<init>(RetryingMetaStoreClient.java:86) at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:132) at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:104) at org.apache.hadoop.hive.ql.metadata.Hive.createMetaStoreClient(Hive.java:3005) at org.apache.hadoop.hive.ql.metadata.Hive.getMSC(Hive.java:3024) at org.apache.hadoop.hive.ql.metadata.Hive.getAllDatabases(Hive.java:1234) at org.apache.hadoop.hive.ql.metadata.Hive.reloadFunctions(Hive.java:174) at org.apache.hadoop.hive.ql.metadata.Hive.<clinit>(Hive.java:166) at org.apache.hadoop.hive.ql.session.SessionState.start(SessionState.java:503) at org.apache.spark.sql.hive.client.ClientWrapper.<init>(ClientWrapper.scala:171) at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:526) at org.apache.spark.sql.hive.client.IsolatedClientLoader.liftedTree1$1(IsolatedClientLoader.scala:183) at org.apache.spark.sql.hive.client.IsolatedClientLoader.<init>(IsolatedClientLoader.scala:179) at org.apache.spark.sql.hive.HiveContext.metadataHive$lzycompute(HiveContext.scala:227) at org.apache.spark.sql.hive.HiveContext.metadataHive(HiveContext.scala:186) at org.apache.spark.sql.hive.HiveContext.setConf(HiveContext.scala:393) at org.apache.spark.sql.hive.HiveContext.defaultOverrides(HiveContext.scala:175) at org.apache.spark.sql.hive.HiveContext.<init>(HiveContext.scala:178) at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:526) at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:234) at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:379) at py4j.Gateway.invoke(Gateway.java:214) at py4j.commands.ConstructorCommand.invokeConstructor(ConstructorCommand.java:79) at py4j.commands.ConstructorCommand.execute(ConstructorCommand.java:68) at py4j.GatewayConnection.run(GatewayConnection.java:207) at java.lang.Thread.run(Thread.java:745) If there happens to have a valid ticket cache, the credentials there will be used. In my env, I ran the spark-submit command as unix user root, and specified ambari-qa as the kerberos principal, and put a different user's kerberos credentials in root's ticket cache. Also set hdfs umask to 027. [root@bdvs1401 spark-yu]# klist Ticket cache: FILE:/tmp/krb5cc_0 Default principal: user4/@BIADS.SVL.IBM.COM Valid starting Expires Service principal 10/21/15 11:08:13 10/21/15 21:08:36 krbtgt/biads.svl.ibm....@biads.svl.ibm.com renew until 10/27/15 13:41:52 [root@bdvs1401 spark-yu]# spark-submit --num-executors 1 --driver-class-path lib/datanucleus-api-jdo-3.2.6.jar:lib/datanucleus-core-3.2.10.jar:lib/datanucleus-rdbms-3.2.9.jar:conf/hive-site.xml --files conf/hive-site.xml --master yarn --principal ambari-qa --keytab /etc/security/keytabs/ambari-qa.keytab --conf spark.eventLog.enabled=true ~/test.py Then ran into the exact file permission error: org.apache.hadoop.hive.ql.metadata.HiveException: checkPaths: filesystem error in check phase. Permission denied: user=user4, access=READ_EXECUTE, inode="/tmp/test_gl/.hive-staging_hive_2015-10-14_23-10-57_135_5312925588384959525-1/-ext-10000":ambari-qa:hdfs:drwxr-x--- ... at org.apache.hadoop.hive.ql.metadata.Hive.checkPaths(Hive.java:2504) at org.apache.hadoop.hive.ql.metadata.Hive.replaceFiles(Hive.java:2840) at org.apache.hadoop.hive.ql.metadata.Hive.loadTable(Hive.java:1640) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at org.apache.spark.sql.hive.client.Shim_v0_14.loadTable(HiveShim.scala:441) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$loadTable$1.apply$mcV$sp(ClientWrapper.scala:489) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$loadTable$1.apply(ClientWrapper.scala:489) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$loadTable$1.apply(ClientWrapper.scala:489) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$withHiveState$1.apply(ClientWrapper.scala:256) at org.apache.spark.sql.hive.client.ClientWrapper.retryLocked(ClientWrapper.scala:211) at org.apache.spark.sql.hive.client.ClientWrapper.withHiveState(ClientWrapper.scala:248) at org.apache.spark.sql.hive.client.ClientWrapper.loadTable(ClientWrapper.scala:488) at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.sideEffectResult$lzycompute(InsertIntoHiveTable.scala:243) at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.sideEffectResult(InsertIntoHiveTable.scala:127) at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.doExecute(InsertIntoHiveTable.scala:263) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:140) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:138) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:147) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:138) at org.apache.spark.sql.SQLContext$QueryExecution.toRdd$lzycompute(SQLContext.scala:927) at org.apache.spark.sql.SQLContext$QueryExecution.toRdd(SQLContext.scala:927) at org.apache.spark.sql.hive.execution.CreateTableAsSelect.run(CreateTableAsSelect.scala:89) at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:57) at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:57) at org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:69) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:140) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:138) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:147) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:138) at org.apache.spark.sql.SQLContext$QueryExecution.toRdd$lzycompute(SQLContext.scala:927) at org.apache.spark.sql.SQLContext$QueryExecution.toRdd(SQLContext.scala:927) at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:144) at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:129) at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:51) at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:719) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:231) at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:379) at py4j.Gateway.invoke(Gateway.java:259) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:133) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:207) at java.lang.Thread.run(Thread.java:745) ... > HiveContext is not used with keytab principal but with user principal/unix > username > ----------------------------------------------------------------------------------- > > Key: SPARK-10181 > URL: https://issues.apache.org/jira/browse/SPARK-10181 > Project: Spark > Issue Type: Bug > Components: SQL > Affects Versions: 1.5.0 > Environment: kerberos > Reporter: Bolke de Bruin > Labels: hive, hivecontext, kerberos > > `bin/spark-submit --num-executors 1 --executor-cores 5 --executor-memory 5G > --driver-java-options -XX:MaxPermSize=4G --driver-class-path > lib/datanucleus-api-jdo-3.2.6.jar:lib/datanucleus-core-3.2.10.jar:lib/datanucleus-rdbms-3.2.9.jar:conf/hive-site.xml > --files conf/hive-site.xml --master yarn --principal sparkjob --keytab > /etc/security/keytabs/sparkjob.keytab --conf > spark.yarn.executor.memoryOverhead=18000 --conf > "spark.executor.extraJavaOptions=-XX:MaxPermSize=4G" --conf > spark.eventLog.enabled=false ~/test.py` > With: > #!/usr/bin/python > from pyspark import SparkContext > from pyspark.sql import HiveContext > sc = SparkContext() > sqlContext = HiveContext(sc) > query = """ SELECT * FROM fm.sk_cluster """ > rdd = sqlContext.sql(query) > rdd.registerTempTable("test") > sqlContext.sql("CREATE TABLE wcs.test LOCATION '/tmp/test_gl' AS SELECT * > FROM test") > Ends up with: > Caused by: > org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.AccessControlException): > Permission denie > d: user=ua80tl, access=READ_EXECUTE, > inode="/tmp/test_gl/.hive-staging_hive_2015-08-24_10-43-09_157_78057390024057878 > 34-1/-ext-10000":sparkjob:hdfs:drwxr-x--- > (Our umask denies read access to other by default) -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org