Uploaded image for project: 'Hadoop Common'
  1. Hadoop Common
  2. HADOOP-11934

Use of JavaKeyStoreProvider in LdapGroupsMapping causes infinite loop

VotersWatch issueWatchersCreate sub-taskLinkCloneUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    • Reviewed

    Description

      I was attempting to use the LdapGroupsMapping code and the JavaKeyStoreProvider at the same time, and hit a really interesting, yet fatal, issue. The code goes into what ought to have been an infinite loop, were it not for it overflowing the stack and Java ending the loop. Here is a snippet of the stack; my annotations are at the bottom.

      	at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:370)
      	at org.apache.hadoop.fs.Path.getFileSystem(Path.java:296)
      	at org.apache.hadoop.security.alias.JavaKeyStoreProvider.<init>(JavaKeyStoreProvider.java:88)
      	at org.apache.hadoop.security.alias.JavaKeyStoreProvider.<init>(JavaKeyStoreProvider.java:65)
      	at org.apache.hadoop.security.alias.JavaKeyStoreProvider$Factory.createProvider(JavaKeyStoreProvider.java:291)
      	at org.apache.hadoop.security.alias.CredentialProviderFactory.getProviders(CredentialProviderFactory.java:58)
      	at org.apache.hadoop.conf.Configuration.getPasswordFromCredentialProviders(Configuration.java:1863)
      	at org.apache.hadoop.conf.Configuration.getPassword(Configuration.java:1843)
      	at org.apache.hadoop.security.LdapGroupsMapping.getPassword(LdapGroupsMapping.java:386)
      	at org.apache.hadoop.security.LdapGroupsMapping.setConf(LdapGroupsMapping.java:349)
      	at org.apache.hadoop.util.ReflectionUtils.setConf(ReflectionUtils.java:73)
      	at org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:133)
      	at org.apache.hadoop.security.Groups.<init>(Groups.java:70)
      	at org.apache.hadoop.security.Groups.<init>(Groups.java:66)
      	at org.apache.hadoop.security.Groups.getUserToGroupsMappingService(Groups.java:280)
      	at org.apache.hadoop.security.UserGroupInformation.initialize(UserGroupInformation.java:283)
      	at org.apache.hadoop.security.UserGroupInformation.ensureInitialized(UserGroupInformation.java:260)
      	at org.apache.hadoop.security.UserGroupInformation.loginUserFromSubject(UserGroupInformation.java:804)
      	at org.apache.hadoop.security.UserGroupInformation.getLoginUser(UserGroupInformation.java:774)
      	at org.apache.hadoop.security.UserGroupInformation.getCurrentUser(UserGroupInformation.java:647)
      	at org.apache.hadoop.fs.FileSystem$Cache$Key.<init>(FileSystem.java:2753)
      	at org.apache.hadoop.fs.FileSystem$Cache$Key.<init>(FileSystem.java:2745)
      	at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2611)
      	at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:370)
      	at org.apache.hadoop.fs.Path.getFileSystem(Path.java:296)
      	at org.apache.hadoop.security.alias.JavaKeyStoreProvider.<init>(JavaKeyStoreProvider.java:88)
      	at org.apache.hadoop.security.alias.JavaKeyStoreProvider.<init>(JavaKeyStoreProvider.java:65)
      	at org.apache.hadoop.security.alias.JavaKeyStoreProvider$Factory.createProvider(JavaKeyStoreProvider.java:291)
      	at org.apache.hadoop.security.alias.CredentialProviderFactory.getProviders(CredentialProviderFactory.java:58)
      	at org.apache.hadoop.conf.Configuration.getPasswordFromCredentialProviders(Configuration.java:1863)
      	at org.apache.hadoop.conf.Configuration.getPassword(Configuration.java:1843)
      	at org.apache.hadoop.security.LdapGroupsMapping.getPassword(LdapGroupsMapping.java:386)
      	at org.apache.hadoop.security.LdapGroupsMapping.setConf(LdapGroupsMapping.java:349)
      	at org.apache.hadoop.util.ReflectionUtils.setConf(ReflectionUtils.java:73)
      	at org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:133)
      	at org.apache.hadoop.security.Groups.<init>(Groups.java:70)
      	at org.apache.hadoop.security.Groups.<init>(Groups.java:66)
      	at org.apache.hadoop.security.Groups.getUserToGroupsMappingService(Groups.java:280)
      	at org.apache.hadoop.security.UserGroupInformation.initialize(UserGroupInformation.java:283)
      	at org.apache.hadoop.security.UserGroupInformation.ensureInitialized(UserGroupInformation.java:260)
      	at org.apache.hadoop.security.UserGroupInformation.loginUserFromSubject(UserGroupInformation.java:804)
      	at org.apache.hadoop.security.UserGroupInformation.getLoginUser(UserGroupInformation.java:774)
      	at org.apache.hadoop.security.UserGroupInformation.getCurrentUser(UserGroupInformation.java:647)
      	at org.apache.hadoop.fs.FileSystem$Cache$Key.<init>(FileSystem.java:2753)
      	at org.apache.hadoop.fs.FileSystem$Cache$Key.<init>(FileSystem.java:2745)
      	at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2611)
      	at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:370)
      	at org.apache.hadoop.fs.Path.getFileSystem(Path.java:296)

      Here's my annotation, going from bottom to top.

      • Somehow we enter Path.getFileSystem()
      • This goes to FileSystem cache stuff, and then it wants the current user
      • So we get to UserGroupInformation.getCurrentUser(), which as you can imagine gets to
      • getUserToGroupsMappingService and thence to LdapGroupsMapping.setConf().
      • That code gets the needed passwords, and we're using the CredentialProvider, so unsurprisingly we get to
      • getPasswordFromCredentialProviders() - which chooses the JavaKeyStoreProvider like I told it to.
      • The JavaKeyStoreProvider, in its constructor, does "fs = path.getFileSystem(conf);"
      • And guess what, we're back in Path.getFileSystem, where we started at the beginning.

      Please let me know if I've somehow configured something incorrectly, but if I have I can't figure out what it is...

      Attachments

        1. HADOOP-11934.001.patch
          33 kB
          Larry McCay
        2. HADOOP-11934.002.patch
          35 kB
          Larry McCay
        3. HADOOP-11934.003.patch
          35 kB
          Larry McCay
        4. HADOOP-11934.004.patch
          35 kB
          Larry McCay
        5. HADOOP-11934.005.patch
          35 kB
          Larry McCay
        6. HADOOP-11934.006.patch
          33 kB
          Larry McCay
        7. HADOOP-11934.007.patch
          33 kB
          Larry McCay
        8. HADOOP-11934.008.patch
          33 kB
          Larry McCay
        9. HADOOP-11934.009.patch
          33 kB
          Larry McCay
        10. HADOOP-11934.010.patch
          33 kB
          Larry McCay
        11. HADOOP-11934-11.patch
          31 kB
          Larry McCay
        12. HADOOP-11934.012.patch
          32 kB
          Larry McCay
        13. HADOOP-11934.013.patch
          32 kB
          Larry McCay
        14. HADOOP-11934-branch.2.6.1.txt
          33 kB
          Vinod Kumar Vavilapalli

        Issue Links

        Activity

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

          People

            lmccay Larry McCay
            yoderme Mike Yoder
            Votes:
            0 Vote for this issue
            Watchers:
            8 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Slack

                Issue deployment