Uploaded image for project: 'Oozie'
  1. Oozie
  2. OOZIE-3128

Oozie Shell Action configuration to disable Parent Process Token

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 4.3.0
    • None
    • action
    • None

    Description

      When executing pig / spark commands from Oozie Shell on secure cluster it fails as it picks the HADOOP_TOKEN_FILE_LOCATION of
      parent process Oozie Launcher's MapTask and gets affected. Below are some of the issues where pig / spark command runs fine when we execute separate
      but with Oozie Shell action fails.

      Most of the users expect the same command which ran fine separate to run fine on Oozie Shell Action. And this issue will be
      difficult to debug. Better to have a separate configuration for Oozie Shell Action
      to unset the Token of parent process by default and if needed we can enable it. This way everyone will be aware of the Parent
      process token and it's impact to their shell script.

      1. Oozie Shell with Spark Command accessing Secure Hive Metastore

      kinit command
      spark-submit --class SparkHiveExample --master yarn-cluster --files /etc/spark/conf/hive-site.xml --jars /tmp/datanucleus-core-3.2.10.jar,/tmp/datanucleus-rdbms-3.2.9.jar,/tmp/datanucleus-api-jdo-3.2.6.jar spark_hive.jar

      Spark Job fails with

      17/05/09 12:24:29 ERROR 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:211)
              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:422)
              at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1724)
              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:62)
              at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
              at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
              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:204)
              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.spark.sql.hive.client.IsolatedClientLoader.createClient(IsolatedClientLoader.scala:249)
              at org.apache.spark.sql.hive.HiveContext.metadataHive$lzycompute(HiveContext.scala:345)
              at org.apache.spark.sql.hive.HiveContext.metadataHive(HiveContext.scala:255)
              at org.apache.spark.sql.hive.HiveContext.setConf(HiveContext.scala:459)
              at org.apache.spark.sql.SQLContext$$anonfun$4.apply(SQLContext.scala:272)
              at org.apache.spark.sql.SQLContext$$anonfun$4.apply(SQLContext.scala:271)
              at scala.collection.Iterator$class.foreach(Iterator.scala:727)
              at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
              at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
              at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
              at org.apache.spark.sql.SQLContext.<init>(SQLContext.scala:271)
              at org.apache.spark.sql.hive.HiveContext.<init>(HiveContext.scala:90)
              at org.apache.spark.sql.hive.HiveContext.<init>(HiveContext.scala:101)
      

      Analysis:

      Running direct Spark-Submit, Spark Client won't have any credentials before and so creates a fresh one as indicated by below logs to access secure Hive. 
      
      >>>>> 
      17/05/12 13:01:42 INFO YarnSparkHadoopUtil: getting token for namenode: hdfs://kerberos-1.openstacklocal:8020/user/prabhu/.sparkStaging/application_1494402843427_0026 
      17/05/12 13:01:42 INFO DFSClient: Created HDFS_DELEGATION_TOKEN token 806 for prabhu on 172.26.71.118:8020 
      >>>>> 
      
      Whereas on Oozie - the shell script is a child of Mapper task. The Mapper task is already having a credentials as indicated by below env HADOOP_TOKEN_FILE_LOCATION 
      
      >>>>> 
      HADOOP_TOKEN_FILE_LOCATION=/hadoop/yarn/local/usercache/prabhu/appcache/application_1494402843427_0014/container_e63_1494402843427_0014_01_000002/container_tokens 
      >>>>> 
      
      So the Spark Client tries to use the previous credentials which won;t have access to Secure Hive and so fails with GSSException. 
      
      >>>> 
      17/05/12 13:08:53 INFO Client: Using credentials supplied in environment.
      >>>> 
      
      
      Workaround:  Adding below in the script
      ********** 
      unset HADOOP_TOKEN_FILE_LOCATION
      
      

      2. Oozie Shell Action with pig on tez

      kinit command
      pig -x tez -useHCatalog pigscript

      fails with "Delegation Token can be issued only with kerberos or web authentication". Adding "unset HADOOP_TOKEN_FILE_LOCATION" into the shell script will fix.

      2017-09-22 12:38:20,765 [PigTezLauncher-0] ERROR org.apache.pig.backend.hadoop.executionengine.tez.TezSessionManager - Exception while waiting for Tez client to be ready
      org.apache.hadoop.ipc.RemoteException(java.io.IOException): Delegation Token can be issued only with kerberos or web authentication
      	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getDelegationToken(FSNamesystem.java:7087)
      	at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getDelegationToken(NameNodeRpcServer.java:676)
      	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getDelegationToken(ClientNamenodeProtocolServerSideTranslatorPB.java:998)
      	at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
      	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:640)
      	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:982)
      	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2351)
      	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2347)
      	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:1866)
      	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2345)
      	at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1554)
      	at org.apache.hadoop.ipc.Client.call(Client.java:1498)
      	at org.apache.hadoop.ipc.Client.call(Client.java:1398)
      	at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:233)
      	at com.sun.proxy.$Proxy10.getDelegationToken(Unknown Source)
      	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getDelegationToken(ClientNamenodeProtocolTranslatorPB.java:980)
      	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:291)
      	at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:203)
      	at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:185)
      	at com.sun.proxy.$Proxy11.getDelegationToken(Unknown Source)
      	at org.apache.hadoop.hdfs.DFSClient.getDelegationToken(DFSClient.java:1041)
      	at org.apache.hadoop.hdfs.DistributedFileSystem.getDelegationToken(DistributedFileSystem.java:1688)
      	at org.apache.hadoop.fs.FileSystem.collectDelegationTokens(FileSystem.java:549)
      	at org.apache.hadoop.fs.FileSystem.addDelegationTokens(FileSystem.java:527)
      	at org.apache.hadoop.hdfs.DistributedFileSystem.addDelegationTokens(DistributedFileSystem.java:2400)
      	at org.apache.tez.common.security.TokenCache.obtainTokensForFileSystemsInternal(TokenCache.java:119)
      	at org.apache.tez.common.security.TokenCache.obtainTokensForFileSystemsInternal(TokenCache.java:98)
      	at org.apache.tez.common.security.TokenCache.obtainTokensForFileSystems(TokenCache.java:76)
      	at org.apache.tez.client.TezClientUtils.setupTezJarsLocalResources(TezClientUtils.java:198)
      	at org.apache.tez.client.TezClient.getTezJarResources(TezClient.java:831)
      	at org.apache.tez.client.TezClient.start(TezClient.java:355)
      	at org.apache.pig.backend.hadoop.executionengine.tez.TezSessionManager.createSession(TezSessionManager.java:102)
      	at org.apache.pig.backend.hadoop.executionengine.tez.TezSessionManager.getClient(TezSessionManager.java:234)
      	at org.apache.pig.backend.hadoop.executionengine.tez.TezJob.run(TezJob.java:203)
      	at org.apache.pig.backend.hadoop.executionengine.tez.TezLauncher$1.run(TezLauncher.java:210)
      	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
      	at java.lang.Thread.run(Thread.java:745)
      

      Attachments

        Activity

          People

            Unassigned Unassigned
            prabhujoseph Prabhu Joseph
            Votes:
            1 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated: