[ 
https://issues.apache.org/jira/browse/HDFS-13697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16580737#comment-16580737
 ] 

Xiao Chen commented on HDFS-13697:
----------------------------------

Thanks for the hard work on this [~zvenczel]!

bq. As these use cases are around for a while I'd expect them to be used widely 
and hard to avoid. What do you think?
This is the part that's really head-scratching. On one hand I really think 
there should be no morphing, but OTOH the existing {{TestAclsEndToEnd}} to 
'work'.

If userA creates the KMSCP, then proxies as userB and calls the KMSCP method, 
our test case is expecting this to be coming from userB. I'm not sure if we 
should deem this as a test issue - anyone knows any downstream usage this way?

[~daryn] [~xyao] what are your thoughts on this? I don't see any other code 
changes that can satisfy these cases without dynamically checking the ugi at 
method invocation time. So it seems there's no way for this to be 'compatible' 
and satisfy the 'just use the ugi at construction time', due to the historical 
morphing nature of the KMSCP...

> DFSClient should instantiate and cache KMSClientProvider using UGI at 
> creation time for consistent UGI handling
> ---------------------------------------------------------------------------------------------------------------
>
>                 Key: HDFS-13697
>                 URL: https://issues.apache.org/jira/browse/HDFS-13697
>             Project: Hadoop HDFS
>          Issue Type: Bug
>            Reporter: Zsolt Venczel
>            Assignee: Zsolt Venczel
>            Priority: Major
>         Attachments: HDFS-13697.01.patch, HDFS-13697.02.patch, 
> HDFS-13697.03.patch, HDFS-13697.04.patch, HDFS-13697.05.patch, 
> HDFS-13697.06.patch, HDFS-13697.07.patch, HDFS-13697.08.patch, 
> HDFS-13697.prelim.patch
>
>
> 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:
> {code}
> [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){code}
> The operation should have succeeded as [example_user] is the owner of the 
> [encrypted_key]



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to