Uploaded image for project: 'Hadoop HDFS'
  1. Hadoop HDFS
  2. HDFS-5539 NFS gateway secuirty enhancement
  3. HDFS-5804

HDFS NFS Gateway fails to mount and proxy when using Kerberos

    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Closed
    • Major
    • Resolution: Fixed
    • 2.2.0, 3.0.0-alpha1
    • 2.4.0
    • nfs
    • None
    • Incompatible change, Reviewed
    • Fixes NFS on Kerberized cluster.

    Description

      When using HDFS nfs gateway with secure hadoop (hadoop.security.authentication: kerberos), mounting hdfs fails.
      Additionally, there is no mechanism to support proxy user(nfs needs to proxy as the user invoking commands on the hdfs mount).

      Steps to reproduce:
      1) start a hadoop cluster with kerberos enabled.
      2) sudo su -l nfsserver and start an nfs server. This 'nfsserver' account has a an account in kerberos.
      3) Get the keytab for nfsserver, and issue the following mount command: mount -t nfs -o vers=3,proto=tcp,nolock $server:/ $mount_point
      4) You'll see in the nfsserver logs that Kerberos is complaining about not having a TGT for root.
      This is the stacktrace:
      java.io.IOException: Failed on local exception: java.io.IOException: org.apache.hadoop.security.AccessControlException: Client cannot authenticate via:[TOKEN, KERBEROS]; Host Details : local host is: "my-nfs-server-host.com/10.252.4.197"; destination host is: "my-namenode-host.com":8020;
      at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:764)
      at org.apache.hadoop.ipc.Client.call(Client.java:1351)
      at org.apache.hadoop.ipc.Client.call(Client.java:1300)
      at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:206)
      at com.sun.proxy.$Proxy9.getFileLinkInfo(Unknown Source)
      at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source)
      at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      at java.lang.reflect.Method.invoke(Method.java:606)
      at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:186)
      at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
      at com.sun.proxy.$Proxy9.getFileLinkInfo(Unknown Source)
      at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileLinkInfo(ClientNamenodeProtocolTranslatorPB.java:664)
      at org.apache.hadoop.hdfs.DFSClient.getFileLinkInfo(DFSClient.java:1713)
      at org.apache.hadoop.hdfs.nfs.nfs3.Nfs3Utils.getFileStatus(Nfs3Utils.java:58)
      at org.apache.hadoop.hdfs.nfs.nfs3.Nfs3Utils.getFileAttr(Nfs3Utils.java:79)
      at org.apache.hadoop.hdfs.nfs.nfs3.RpcProgramNfs3.fsinfo(RpcProgramNfs3.java:1643)
      at org.apache.hadoop.hdfs.nfs.nfs3.RpcProgramNfs3.handleInternal(RpcProgramNfs3.java:1891)
      at org.apache.hadoop.oncrpc.RpcProgram.messageReceived(RpcProgram.java:143)
      at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70)
      at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:560)
      at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:787)
      at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:281)
      at org.apache.hadoop.oncrpc.RpcUtil$RpcMessageParserStage.messageReceived(RpcUtil.java:132)
      at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70)
      at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:560)
      at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:787)
      at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:296)
      at org.jboss.netty.handler.codec.frame.FrameDecoder.unfoldAndFireMessageReceived(FrameDecoder.java:462)
      at org.jboss.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:443)
      at org.jboss.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.java:303)
      at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70)
      at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:560)
      at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:555)
      at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:268)
      at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:255)
      at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:88)
      at org.jboss.netty.channel.socket.nio.AbstractNioWorker.process(AbstractNioWorker.java:107)
      at org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:312)
      at org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:88)
      at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178)
      at org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
      at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
      at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
      at java.lang.Thread.run(Thread.java:744)
      Caused by: java.io.IOException: org.apache.hadoop.security.AccessControlException: Client cannot authenticate via:[TOKEN, KERBEROS]
      at org.apache.hadoop.ipc.Client$Connection$1.run(Client.java:620)
      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:1491)
      at org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:583)
      at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:667)
      at org.apache.hadoop.ipc.Client$Connection.access$2600(Client.java:314)
      at org.apache.hadoop.ipc.Client.getConnection(Client.java:1399)
      at org.apache.hadoop.ipc.Client.call(Client.java:1318)
      ... 43 more
      Caused by: org.apache.hadoop.security.AccessControlException: Client cannot authenticate via:[TOKEN, KERBEROS]
      at org.apache.hadoop.security.SaslRpcClient.selectSaslClient(SaslRpcClient.java:170)
      at org.apache.hadoop.security.SaslRpcClient.saslConnect(SaslRpcClient.java:387)
      at org.apache.hadoop.ipc.Client$Connection.setupSaslConnection(Client.java:494)
      at org.apache.hadoop.ipc.Client$Connection.access$1700(Client.java:314)
      at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:659)
      at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:655)
      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:1491)
      at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:654)

      Attachments

        1. exception-as-root.log
          179 kB
          Abin Shahab
        2. HDFS-5804.patch
          9 kB
          Abin Shahab
        3. HDFS-5804.patch
          10 kB
          Abin Shahab
        4. HDFS-5804.patch
          10 kB
          Abin Shahab
        5. HDFS-5804.patch
          7 kB
          Abin Shahab
        6. HDFS-5804.patch
          8 kB
          Abin Shahab
        7. HDFS-5804.patch
          8 kB
          Abin Shahab
        8. HDFS-5804.patch
          9 kB
          Abin Shahab
        9. HDFS-5804-documentation.patch
          2 kB
          Abin Shahab
        10. javadoc-after-patch.log
          1.63 MB
          Abin Shahab
        11. javadoc-before-patch.log
          1.64 MB
          Abin Shahab

        Issue Links

          Activity

            People

              ashahab Abin Shahab
              ashahab Abin Shahab
              Votes:
              0 Vote for this issue
              Watchers:
              10 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: