Uploaded image for project: 'Hadoop HDFS'
  1. Hadoop HDFS
  2. HDFS-14013

Skip any credentials stored in HDFS when starting ZKFC

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 3.1.1
    • 3.3.1, 3.4.0, 3.2.3
    • hdfs
    • Reviewed

    Description

      HADOOP-15157 added the ability to use a jceks credential provider to store the Zookeeper credentials needed by the Failover Controller to connect to Zookeeper.

      By default, if any provider is specified in hadoop.security.credential.provider.path it will be checked to see if it holds the required information, otherwise the traditional way of getting the the login will be used.

      hadoop.security.credential.provider.path can hold a list of credential providers and if there is an error reading any of them, the exception bubbles up and causes the ZKFC to fail. The intent of HADOOP-15157 is to have a local jceks file for the FC credentials, but if there is another provider stored in HDFS (eg S3A credentials), then it will fail to be read and cause the FC to fail.

      Other components which use credential providers (eg S3A, ABFS etc) explicitly disallow storing the credentials in the same type of filesystem. Ie, S3A cannot use providers stored in S3. To avoid this sort of circular dependency, any such credentials are removed from the list before they are used.

      The Failover Controller should do the same, and ensure it does not try to read any credentials stored in HDFS, as it will never be able to do so until HDFS is full started.

      For reference, the stack logged when the FC meets this problem is:
        

      2018-10-22 08:17:09,251 FATAL tools.DFSZKFailoverController (DFSZKFailoverController.java:main(197)) - DFSZKFailOverController exiting due to earlier exception java.io.IOException: Configuration problem with provider path. 2018-10-22 08:17:09,252 DEBUG util.ExitUtil (ExitUtil.java:terminate(209)) - Exiting with status 1: java.io.IOException: Configuration problem with provider path. 1: java.io.IOException: Configuration problem with provider path.     at org.apache.hadoop.util.ExitUtil.terminate(ExitUtil.java:265)     at org.apache.hadoop.hdfs.tools.DFSZKFailoverController.main(DFSZKFailoverController.java:199) Caused by: java.io.IOException: Configuration problem with provider path.     at org.apache.hadoop.conf.Configuration.getPasswordFromCredentialProviders(Configuration.java:2363)     at org.apache.hadoop.conf.Configuration.getPassword(Configuration.java:2282)     at org.apache.hadoop.security.SecurityUtil.getZKAuthInfos(SecurityUtil.java:732)     at org.apache.hadoop.ha.ZKFailoverController.initZK(ZKFailoverController.java:343)     at org.apache.hadoop.ha.ZKFailoverController.doRun(ZKFailoverController.java:194)     at org.apache.hadoop.ha.ZKFailoverController.access$000(ZKFailoverController.java:60)     at org.apache.hadoop.ha.ZKFailoverController$1.run(ZKFailoverController.java:175)     at org.apache.hadoop.ha.ZKFailoverController$1.run(ZKFailoverController.java:171)     at java.security.AccessController.doPrivileged(Native Method)     at javax.security.auth.Subject.doAs(Subject.java:360)     at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1710)     at org.apache.hadoop.security.SecurityUtil.doAsLoginUserOrFatal(SecurityUtil.java:480)     at org.apache.hadoop.ha.ZKFailoverController.run(ZKFailoverController.java:171)     at org.apache.hadoop.hdfs.tools.DFSZKFailoverController.main(DFSZKFailoverController.java:195) Caused by: org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.ipc.StandbyException): Operation category READ is not supported in state standby. Visit https://s.apache.org/sbnn-error     at org.apache.hadoop.hdfs.server.namenode.ha.StandbyState.checkOperation(StandbyState.java:88)     at org.apache.hadoop.hdfs.server.namenode.NameNode$NameNodeHAContext.checkOperation(NameNode.java:1951)     at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkOperation(FSNamesystem.java:1427)     at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getFileInfo(FSNamesystem.java:3100)     at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getFileInfo(NameNodeRpcServer.java:1154)     at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getFileInfo(ClientNamenodeProtocolServerSideTranslatorPB.java:966)     at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)     at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:524)     at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1025)     at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:876)     at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:822)     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:1730)     at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2682) 
           at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1497)     at org.apache.hadoop.ipc.Client.call(Client.java:1443)     at org.apache.hadoop.ipc.Client.call(Client.java:1353)     at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:228)     at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:116)     at com.sun.proxy.$Proxy9.getFileInfo(Unknown Source)     at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:900)     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.$Proxy10.getFileInfo(Unknown Source)     at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1654)     at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1583)     at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1580)     at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)     at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1595)     at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1734)     at org.apache.hadoop.security.alias.JavaKeyStoreProvider.keystoreExists(JavaKeyStoreProvider.java:65)     at org.apache.hadoop.security.alias.AbstractJavaKeyStoreProvider.locateKeystore(AbstractJavaKeyStoreProvider.java:319)     at org.apache.hadoop.security.alias.AbstractJavaKeyStoreProvider.<init>(AbstractJavaKeyStoreProvider.java:86)     at org.apache.hadoop.security.alias.JavaKeyStoreProvider.<init>(JavaKeyStoreProvider.java:49)     at org.apache.hadoop.security.alias.JavaKeyStoreProvider.<init>(JavaKeyStoreProvider.java:41)     at org.apache.hadoop.security.alias.JavaKeyStoreProvider$Factory.createProvider(JavaKeyStoreProvider.java:100)     at org.apache.hadoop.security.alias.CredentialProviderFactory.getProviders(CredentialProviderFactory.java:73)     at org.apache.hadoop.conf.Configuration.getPasswordFromCredentialProviders(Configuration.java:2344)     ... 13 more

       

      Attachments

        1. hadoop-hdfs-zkfc-server1.log
          63 kB
          Krzysztof Adamski
        2. HDFS-14013.001.patch
          8 kB
          Stephen O'Donnell

        Activity

          People

            sodonnell Stephen O'Donnell
            krisss Krzysztof Adamski
            Votes:
            1 Vote for this issue
            Watchers:
            10 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: