-
Notifications
You must be signed in to change notification settings - Fork 13.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-31109][yarn] Support Hadoop proxy user when delegation token f… #22009
Conversation
Please review. cc @MartijnVisser @gaborgsomogyi @becketqin |
Looking into the test failure, somehow it worked locally but failing in the pipeline. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for fixing it! Some minors found but when fixed then it's mergeble.
...rc/test/java/org/apache/flink/runtime/security/token/hadoop/KerberosLoginProviderITCase.java
Outdated
Show resolved
Hide resolved
flink-runtime/src/main/java/org/apache/flink/runtime/security/modules/HadoopModule.java
Show resolved
Hide resolved
...time/src/main/java/org/apache/flink/runtime/security/token/hadoop/KerberosLoginProvider.java
Outdated
Show resolved
Hide resolved
e2e tests are still running so most probably it's going to pass. |
@gaborgsomogyi So all there's left are the minor review comments and then it's ready to merge from your perspective? |
@MartijnVisser when then mentioned suggestions are fixed then it can go. Planning to track the changes and approve it when I think it's flawless. |
@MartijnVisser everything is green so good to go from.my side. |
@venkata91 Can you also create a back port towards |
…etch is disabled. This closes apache#22009 * [FLINK-31109][yarn] Support Hadoop proxy user when delegation token fetch is disabled (cherry picked from commit 7813613)
…etch is disabled. This closes #22009 * [FLINK-31109][yarn] Support Hadoop proxy user when delegation token fetch is disabled (cherry picked from commit 7813613)
@MartijnVisser Looks like you already backported it. I was about to get to it. Thanks for backporting it to release-1.17 as well. |
I've tried specifying proxyUser using Flink seems to use @venkata91 @gaborgsomogyi Could you please take a look? |
Not sure what you mean here. If you update the tokens externally then it's the external code responsibility to do its job properly, right?
In your own distribution you can do anything you can imagine. There is no configuration infra for this so you need to change the source (namely In general, what is the use-case what you would like to achieve here? If I understand correctly the goal is to have 2 users in a single cluster (proxy user + the other one who is creating tokens)? |
@gaborgsomogyi Thank you for your reply. I want to implement impersonation mode for flink engine in Apahce Kyuubi. Flink proxy user mode relies on external services to maintain delegation tokens, and kyuubi server can do this well. Kyuubi server uses super user proxy to sessionUser to obtain delegation tokens, and brings them to flink jobmanger when starting the flink engine. It will regularly update delegation tokens and send them to jobmanager, and at the same time I want to update taskmanager tokens (but I can't find a suitable way). Related issues: |
Is this improvement reasonable? Can we introduce it upstream? |
Just to give some historical background Flink (just like Spark) can't really guarantee that an externally given proxy user TGT will be renewed. This may end-up in stopped streaming workloads which will be bad UX for users. As a conclusion we decided to say plz manage it externally. If some internal Flink constructs can be re-used to do that then it's fine but that was not an explicit aim.
Adding a config which manager implementation should be used is no brainer but the default manager has some requirements in its constructor: Lines 101 to 105 in 5f5722f
When we would say that the manager can be changed that would mean this constructor param set must be a user facing API which I wouldn't do. These however are subject to change. As a conclusion I can imagine the following:
Flink and Spark session cluster scalable multi-user handling is something which can be enhanced but it's pending. |
Yes, I want to update UserGroupInformation directly, but this update is difficult to pass to taskmanagers since we turned off delegation token manager. If there are any exposed interfaces, that would be great. flink/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java Lines 1578 to 1597 in 5f5722f
|
The framework is so flexible that one can do anything with it 🙂
|
Sounds great, thanks, I will try this way. |
Thanks @gaborgsomogyi , this is an effective way, I've tried it in kyuubi and it mostly works well. Just one point, since we disabled flink/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java Lines 1374 to 1378 in 4b342da
Shall we use |
…etch is disabled
What is the purpose of the change
FLINK-28330 removed old delegation token framework code as part of it removed the existing support for delegation tokens that are managed outside of Flink
Brief change log
YarnClusterDescriptor#setTokensFor
load the available delegation tokens in the client machine and then load the tokens obtained throughDelegationTokenManager
ifsecurity.delegation.tokens.enabled
is set to true.HadoopModule
should throw exception if theUserGroupInformation.currentUser
is a hadoop proxy user and alsosecurity.delegation.tokens.enabled
is set to true.Verifying this change
HadoopModuleTest
and inKerberosLoginProviderITCase
.Does this pull request potentially affect one of the following parts:
@Public(Evolving)
: (yes / no)Documentation