[
https://issues.apache.org/jira/browse/HADOOP-12563?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16635508#comment-16635508
]
Steve Loughran commented on HADOOP-12563:
-----------------------------------------
Following up on this, I'm not sure the API works that well, with a specific
example: using it to ask for filesystem delegation tokens.
# the matching is per-schema, so you would need to provide a different service
for every single FS schema
# {{isTokenRequired()}} doesn't take the URL where tokens are requested, so it
can't actually give a valid answer if the outcome is specific to a specific
endpoint, but only on system/process state (UGI.isSecurityEnabled()). An
implementation cannot currently return true and decide in
{{addDelegationTokens()}} what to do, as {{DtFileOperations.getTokenFile()}}
can't currently handle {{addDelegationTokens()}} not returning a token.
(Implementation issue: Fixable)
# in DtFileOperations, the call to addDelegationTokens(} hands in a URL which
has already had http/https schemas stripped. That may not be the desired
outcome.
To add FS token fetching, it seems to me that I'd need to
# have an abstract DtFetcher with an implementation for every FS known to
return DTs
# have it return true for all {{isTokenRequired()}} calls
# and use addDelegationTokens for the logic about whether an FS exists, can be
instantaited, supports DTs, etc.
# while patching DtFileOperations to handle the case "no token returned"
Is this correct
Alternatively: add a special {{getfs}} command which instantiates the FS and
asks for it directly. That's closest to what
{{org.apache.hadoop.hdfs.tools.DelegationTokenFetcher}} does, but it woiuld be
broader than just hdfs and support the new file format
> Updated utility to create/modify token files
> --------------------------------------------
>
> Key: HADOOP-12563
> URL: https://issues.apache.org/jira/browse/HADOOP-12563
> Project: Hadoop Common
> Issue Type: New Feature
> Affects Versions: 3.0.0-alpha1
> Reporter: Allen Wittenauer
> Assignee: Matthew Paduano
> Priority: Major
> Fix For: 3.0.0-alpha1
>
> Attachments: HADOOP-12563.01.patch, HADOOP-12563.02.patch,
> HADOOP-12563.03.patch, HADOOP-12563.04.patch, HADOOP-12563.05.patch,
> HADOOP-12563.06.patch, HADOOP-12563.07.patch, HADOOP-12563.07.patch,
> HADOOP-12563.08.patch, HADOOP-12563.09.patch, HADOOP-12563.10.patch,
> HADOOP-12563.11.patch, HADOOP-12563.12.patch, HADOOP-12563.13.patch,
> HADOOP-12563.14.patch, HADOOP-12563.15.patch, HADOOP-12563.16.patch,
> dtutil-test-out, example_dtutil_commands_and_output.txt,
> generalized_token_case.pdf
>
>
> hdfs fetchdt is missing some critical features and is geared almost
> exclusively towards HDFS operations. Additionally, the token files that are
> created use Java serializations which are hard/impossible to deal with in
> other languages. It should be replaced with a better utility in common that
> can read/write protobuf-based token files, has enough flexibility to be used
> with other services, and offers key functionality such as append and rename.
> The old version file format should still be supported for backward
> compatibility, but will be effectively deprecated.
> A follow-on JIRA will deprecrate fetchdt.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]