Uploaded image for project: 'Hadoop HDFS'
  1. Hadoop HDFS
  2. HDFS-15747 RBF: Rename across sub-namespaces.
  3. HDFS-15923

RBF: Authentication failed when rename accross sub clusters

VotersWatch issueWatchersLinkCloneUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    Description

      Rename accross subcluster with RBF and Kerberos environment. Will encounter the following two errors:

      1. Save Object to journal.
      2. Precheck try to get src file status

      So, we need use Router Login UGI doAs create DistcpProcedure and TrashProcedure and submit Job.

       

      Beside, we should check user permission for src and dst path in router side before do rename internal. (HDFS-15973)

      First: Save Object to journal.

      // code placeholder
      2021-03-23 14:01:16,233 WARN org.apache.hadoop.ipc.Client: Exception encountered while connecting to the server 
      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.hadoop.security.SaslRpcClient.saslConnect(SaslRpcClient.java:408)
              at org.apache.hadoop.ipc.Client$Connection.setupSaslConnection(Client.java:622)
              at org.apache.hadoop.ipc.Client$Connection.access$2300(Client.java:413)
              at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:822)
              at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:818)
              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:1762)
              at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:818)
              at org.apache.hadoop.ipc.Client$Connection.access$3800(Client.java:413)
              at org.apache.hadoop.ipc.Client.getConnection(Client.java:1636)
              at org.apache.hadoop.ipc.Client.call(Client.java:1452)
              at org.apache.hadoop.ipc.Client.call(Client.java:1405)
              at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:233)
              at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:118)
              at com.sun.proxy.$Proxy11.create(Unknown Source)
              at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.create(ClientNamenodeProtocolTranslatorPB.java:376)
              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:422)
              at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:165)
              at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:157)
              at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:95)
              at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:359)
              at com.sun.proxy.$Proxy12.create(Unknown Source)
              at org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:277)
              at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1240)
              at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1219)
              at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1201)
              at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1139)
              at org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:533)
              at org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:530)
              at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
              at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:544)
              at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:471)
              at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1125)
              at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1105)
              at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:994)
              at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:982)
              at org.apache.hadoop.tools.fedbalance.procedure.BalanceJournalInfoHDFS.saveJob(BalanceJournalInfoHDFS.java:89)
              at org.apache.hadoop.tools.fedbalance.procedure.BalanceProcedureScheduler.submit(BalanceProcedureScheduler.java:134)
              at org.apache.hadoop.hdfs.server.federation.router.RouterFederationRename.routerFedRename(RouterFederationRename.java:105)
              at org.apache.hadoop.hdfs.server.federation.router.RouterClientProtocol.rename(RouterClientProtocol.java:596)
              at org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer.rename(RouterRpcServer.java:950)
              at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.rename(ClientNamenodeProtocolServerSideTranslatorPB.java:661)
              at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
              at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:528)
              at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1086)
              at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:1029)
              at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:957)
              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:1762)
              at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2957)
      Caused by: GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)
              at sun.security.jgss.krb5.Krb5InitCredential.getInstance(Krb5InitCredential.java:147)
              at sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:122)
              at sun.security.jgss.krb5.Krb5MechFactory.getMechanismContext(Krb5MechFactory.java:187)
              at sun.security.jgss.GSSManagerImpl.getMechanismContext(GSSManagerImpl.java:224)
              at sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:212)
              at sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:179)
              at com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:192)
              ... 56 more
      

      Second: Precheck try to get src file status

       

       

      // code placeholder
      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.hadoop.security.SaslRpcClient.saslConnect(SaslRpcClient.java:408)
              at org.apache.hadoop.ipc.Client$Connection.setupSaslConnection(Client.java:622)
              at org.apache.hadoop.ipc.Client$Connection.access$2300(Client.java:413)
              at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:822)
              at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:818)
              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:1762)
              at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:818)
              at org.apache.hadoop.ipc.Client$Connection.access$3800(Client.java:413)
              at org.apache.hadoop.ipc.Client.getConnection(Client.java:1636)
              at org.apache.hadoop.ipc.Client.call(Client.java:1452)
              at org.apache.hadoop.ipc.Client.call(Client.java:1405)
              at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:233)
              at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:118)
              at com.sun.proxy.$Proxy11.getFileInfo(Unknown Source)
              at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:914)
              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:422)
              at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:165)
              at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:157)
              at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:95)
              at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:359)
              at com.sun.proxy.$Proxy12.getFileInfo(Unknown Source)
              at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1689)
              at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1608)
              at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1605)
              at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
              at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1620)
              at org.apache.hadoop.tools.fedbalance.DistCpProcedure.preCheck(DistCpProcedure.java:183)
              at org.apache.hadoop.tools.fedbalance.DistCpProcedure.execute(DistCpProcedure.java:157)
              at org.apache.hadoop.tools.fedbalance.procedure.BalanceJob.execute(BalanceJob.java:132)
              at org.apache.hadoop.tools.fedbalance.procedure.BalanceProcedureScheduler$Reader.lambda$run$0(BalanceProcedureScheduler.java:339)
              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:748)
      Caused by: GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)
              at sun.security.jgss.krb5.Krb5InitCredential.getInstance(Krb5InitCredential.java:147)
              at sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:122)
              at sun.security.jgss.krb5.Krb5MechFactory.getMechanismContext(Krb5MechFactory.java:187)
              at sun.security.jgss.GSSManagerImpl.getMechanismContext(GSSManagerImpl.java:224)
              at sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:212)
              at sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:179)
              at com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:192)
              ... 41 more
      

       

       

      Attachments

        1. HDFS-15923.001.patch
          17 kB
          zhuobin zheng
        2. HDFS-15923.002.patch
          17 kB
          zhuobin zheng
        3. HDFS-15923.003.patch
          16 kB
          zhuobin zheng
        4. HDFS-15923.stack-trace
          5 kB
          Jinglun
        5. hdfs-15923-fix-security-issue.patch
          3 kB
          Jinglun

        Issue Links

        Activity

          This comment will be Viewable by All Users Viewable by All Users
          Cancel

          People

            zhengzhuobinzzb zhuobin zheng
            zhengzhuobinzzb zhuobin zheng
            Votes:
            0 Vote for this issue
            Watchers:
            6 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Time Tracking

                Estimated:
                Original Estimate - Not Specified
                Not Specified
                Remaining:
                Remaining Estimate - 0h
                0h
                Logged:
                Time Spent - 1.5h
                1.5h

                Slack

                  Issue deployment