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

DFSClient should instantiate and cache KMSClientProvider using UGI at creation time for consistent UGI handling

    Details

    • Type: Bug
    • Status: Patch Available
    • Priority: Major
    • Resolution: Unresolved
    • Affects Version/s: None
    • Fix Version/s: None
    • Component/s: None
    • Labels:
      None

      Description

      While calling KeyProviderCryptoExtension decryptEncryptedKey the call stack might not have doAs privileged execution call (in the DFSClient for example). This results in loosing the proxy user from UGI as UGI.getCurrentUser finds no AccessControllerContext and does a re-login for the login user only.

      This can cause the following for example: if we have set up the oozie user to be entitled to perform actions on behalf of example_user but oozie is forbidden to decrypt any EDEK (for security reasons), due to the above issue, example_user entitlements are lost from UGI and the following error is reported:

      [0] 
      SERVER[xxx] USER[example_user] GROUP[-] TOKEN[] APP[Test_EAR] JOB[0020905-180313191552532-oozie-oozi-W] ACTION[0020905-180313191552532-oozie-oozi-W@polling_dir_path] Error starting action [polling_dir_path]. ErrorType [ERROR], ErrorCode [FS014], Message [FS014: User [oozie] is not authorized to perform [DECRYPT_EEK] on key with ACL name [encrypted_key]!!]
      org.apache.oozie.action.ActionExecutorException: FS014: User [oozie] is not authorized to perform [DECRYPT_EEK] on key with ACL name [encrypted_key]!!
       at org.apache.oozie.action.ActionExecutor.convertExceptionHelper(ActionExecutor.java:463)
       at org.apache.oozie.action.ActionExecutor.convertException(ActionExecutor.java:441)
       at org.apache.oozie.action.hadoop.FsActionExecutor.touchz(FsActionExecutor.java:523)
       at org.apache.oozie.action.hadoop.FsActionExecutor.doOperations(FsActionExecutor.java:199)
       at org.apache.oozie.action.hadoop.FsActionExecutor.start(FsActionExecutor.java:563)
       at org.apache.oozie.command.wf.ActionStartXCommand.execute(ActionStartXCommand.java:232)
       at org.apache.oozie.command.wf.ActionStartXCommand.execute(ActionStartXCommand.java:63)
       at org.apache.oozie.command.XCommand.call(XCommand.java:286)
       at org.apache.oozie.service.CallableQueueService$CompositeCallable.call(CallableQueueService.java:332)
       at org.apache.oozie.service.CallableQueueService$CompositeCallable.call(CallableQueueService.java:261)
       at java.util.concurrent.FutureTask.run(FutureTask.java:262)
       at org.apache.oozie.service.CallableQueueService$CallableWrapper.run(CallableQueueService.java:179)
       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: org.apache.hadoop.security.authorize.AuthorizationException: User [oozie] is not authorized to perform [DECRYPT_EEK] on key with ACL name [encrypted_key]!!
       at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
       at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57)
       at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
       at java.lang.reflect.Constructor.newInstance(Constructor.java:526)
       at org.apache.hadoop.util.HttpExceptionUtils.validateResponse(HttpExceptionUtils.java:157)
       at org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:607)
       at org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:565)
       at org.apache.hadoop.crypto.key.kms.KMSClientProvider.decryptEncryptedKey(KMSClientProvider.java:832)
       at org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider$5.call(LoadBalancingKMSClientProvider.java:209)
       at org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider$5.call(LoadBalancingKMSClientProvider.java:205)
       at org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.doOp(LoadBalancingKMSClientProvider.java:94)
       at org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.decryptEncryptedKey(LoadBalancingKMSClientProvider.java:205)
       at org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.decryptEncryptedKey(KeyProviderCryptoExtension.java:388)
       at org.apache.hadoop.hdfs.DFSClient.decryptEncryptedDataEncryptionKey(DFSClient.java:1440)
       at org.apache.hadoop.hdfs.DFSClient.createWrappedOutputStream(DFSClient.java:1542)
       at org.apache.hadoop.hdfs.DFSClient.createWrappedOutputStream(DFSClient.java:1527)
       at org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:408)
       at org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:401)
       at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
       at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:401)
       at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:344)
       at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:923)
       at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:904)
       at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:801)
       at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:790)
       at org.apache.oozie.action.hadoop.FsActionExecutor.touchz(FsActionExecutor.java:519)

      The operation should have succeeded as [example_user] is the owner of the [encrypted_key]

        Attachments

        1. HDFS-13697.01.patch
          13 kB
          Zsolt Venczel
        2. HDFS-13697.02.patch
          13 kB
          Zsolt Venczel
        3. HDFS-13697.03.patch
          14 kB
          Zsolt Venczel
        4. HDFS-13697.04.patch
          35 kB
          Zsolt Venczel
        5. HDFS-13697.05.patch
          37 kB
          Zsolt Venczel
        6. HDFS-13697.06.patch
          33 kB
          Zsolt Venczel
        7. HDFS-13697.07.patch
          33 kB
          Zsolt Venczel
        8. HDFS-13697.08.patch
          53 kB
          Zsolt Venczel
        9. HDFS-13697.09.patch
          109 kB
          Zsolt Venczel
        10. HDFS-13697.10.patch
          109 kB
          Zsolt Venczel
        11. HDFS-13697.11.patch
          109 kB
          Zsolt Venczel
        12. HDFS-13697.12.patch
          110 kB
          Zsolt Venczel
        13. HDFS-13697.prelim.patch
          42 kB
          Xiao Chen

          Activity

            People

            • Assignee:
              zvenczel Zsolt Venczel
              Reporter:
              zvenczel Zsolt Venczel
            • Votes:
              0 Vote for this issue
              Watchers:
              13 Start watching this issue

              Dates

              • Created:
                Updated: