hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Zsolt Venczel (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HDFS-13697) DFSClient should instantiate and cache KMSClientProvider using UGI at creation time for consistent UGI handling
Date Mon, 06 Aug 2018 12:54:00 GMT

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

Zsolt Venczel commented on HDFS-13697:
--------------------------------------

Thanks a lot [~xiaochen] for the support on this task. It's a challenging one indeed :)

Please find my answers below:
{quote}Ideally we want to do the same as DFSClient, where a ugi of {{UGI#getCurrentUser}} is
just cached at construction time, and used for later auths. I tried that but it caused test
failures in TestKMS with the {{doWebHDFSProxyUserTest}} tests and {{testTGTRenewal}} -
for the sake of compatibility I think we can do something like this to allow the tests to
pass.
{code:java}
// in KMSCP ctor
    ugi = UserGroupInformation.getCurrentUser().getRealUser() == null ?
 UserGroupInformation.getCurrentUser() : 
 UserGroupInformation.getCurrentUser().getRealUser();
{code}
[~daryn] [~xyao] [~jnp] what do you think?
{quote}
The tests are failing because with the above approach we are not supporting the scenario when
the user component provides new entitlements for KMS interactions through a doAs call (eg.
calls the 'createConnection' function implicitly having a proxy user provided in a doAs context).
If we do want to be compatible, caching at construction time the UGI is not enough.
{quote}
We don't need cachedProxyUgi, and getDoAsUser can figure things out from the ugi cached if
we do the above
{quote}
I was trying to introduce some clean code here by defining explicitly under what circumstances
can we have a cachedProxyUgi and by this I also moved one computation to the constructor level
instead of having many on the getDoAsUser level. Does this make sense?
{quote}
ugiToUse doesn't seem necessary
{quote}
I was trying to make the code more meaningful and also to support the above mentioned, proxy
scenario we still need to check whether the current call (currentUgi) introduces any proxy
ugi.
{quote}
Could you explain why the setLoginUser lines were removed in TestKMS? I'd like to make sure
existing tests pass as-is, if possible.
{quote}
I've reverted HADOOP-13749 and these lines were introduced by it. I'm not sure if it makes
sense to set the login user even after the revert. What do you think?
{quote}
the new com.google imports should be placed next to other existing imports of that module.
{quote}
Thanks for checking, I've fixed it in my latest patch.
{quote}
I would not call the KeyProvider variable testKeyProvider - it's used for all purposes. Just
the VisibleForTesting annotation on setKeyProvider would be enough, which you already have.
{quote}
Yes, it makes sense, I've fixed it in my latest patch. On a long run I might refactor these
test cases to use Mockito to reduce production code complexity.
{quote}
The new patch's KeyProviderSupplier#isKeyProviderCreated doesn't seem necessary. We can't
prevent the caller calling getKeyProvider after calling close here from that check. (We probably
can add a guard in DFSClient to prevent all API calls after close, but that's separate from
this jira.)
{quote}
KeyProviderSupplier#isKeyProviderCreated is the only way to know for sure whether KeyProvider
got instantiated or not. If we call keyProviderCache.get() in the close method we might end
up with an unnecessary creation of a KeyProvider.
I agree that we should take care of any post closure calls separately.
{quote}
Although callers seem to have check about nullity of the provider, if DFSClient failed to
create a key provider, it's preferred to throw immediately.
{quote}
I was trying to reproduce the already available behavior present in the KeyProviderCache that
had returned a null and had emitted warn level log messages. Should we change that?

> 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
>
>
> 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-unsubscribe@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-help@hadoop.apache.org


Mime
View raw message