Proxy user has normally kerberos credentials and with that it's possible to
fetch new HDFS tokens.
When you can keep the proxy user kerberos credentials up-to-date (which is
not an easy task) then the YARN side can potentially work.

G


On Wed, Aug 14, 2024 at 5:42 PM dpp <pengpeng.d...@foxmail.com> wrote:

> Thanks for the quick reply, good buddy.
> Based on the current situation, it seems that the Flink on Yarn mode has
> some flaws in the Kerberos scenario, and the Yarn team doesn't seem to be
> paying much attention to this issue, which is quite troublesome.
>
> I noticed there is a Yarn parameter called
> `yarn.resourcemanager.proxy-user-privileges.enabled` that seems to be
> suitable for long-running applications. Are you familiar with this
> parameter? Can it be used to solve Flink's issue?
>
>
> ------------------------------
>
>
>
>
> ------------------ 原始邮件 ------------------
> *发件人:* "Gabor Somogyi" <gabor.g.somo...@gmail.com>;
> *发送时间:* 2024年8月14日(星期三) 晚上11:17
> *收件人:* "user"<user@flink.apache.org>;
> *主题:* Re: Flink jobs failed in Kerberos delegation token
>
> Hi,
>
> Now I see the situation. In short from Fink point of view there is no
> possibility to refresh AM Container tokens since it's YARN responsibility
> so this is a known issue.
> I've reported and discussed this issue with the YARN guys back in the days
> when we've added token framework to Spark but since then nothing happened.
>
> All in all the scenario related the YARN log aggregation what I've shared
> is something which is not what Flink can solve.
> I can imagine the following options depending on your possibilities:
> * Not advised because it increases the attack surface but cheap: Increase
> the token max lifetime
> * Implement a custom HadoopFSDelegationTokenReceiver which is somehow
> update AM tokens in the outside YARN world (this is a hacky but viable)
> * Try to turn off all YARN features which are depending on tokens, like
> log aggregation etc...
> * Switch to k8s
>
> BR,
> G
>
>
> On Wed, Aug 14, 2024 at 4:55 PM dpp <pengpeng.d...@foxmail.com> wrote:
>
>> Hi,
>> I have reviewed your response and retested using the officially released
>> version 1.17.2 in the Kerberos scenario, but the same issue still occurred.
>>
>> In Yarn mode, I simulated the stop of a NodeManager node where the
>> JobManager is located, triggering the high availability feature of Flink
>> jobs. I set the Flink job to restart 3 times. After 3 retries, the Flink
>> JobManager's Yarn container failed to start, with the following exception
>> reported:
>>
>> *Diagnostics Info: *
>> *AM Container for appattempt_1723616701630_0003_000003 exited with
>> exitCode: -1000*
>> *Failing this attempt.Diagnostics: [2024-08-14 16:12:49.183]token (token
>> for mr: HDFS_DELEGATION_TOKEN owner=xxx/x...@xxx.com, renewer=, realUser=,
>> issueDate=1723621278534, maxDate=1723621878534, sequenceNumber=130,
>> masterKeyId=24) is expired, current time: 2024-08-14 16:12:49,152+0800
>> expected renewal time: 2024-08-14 15:46:18,534+0800*
>> *org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken):
>> token (token for mr: HDFS_DELEGATION_TOKEN owner=xxx/x...@xxx.com, renewer=,
>> realUser=, issueDate=1723621278534, maxDate=1723621878534,
>> sequenceNumber=130, masterKeyId=24) is expired, current time: 2024-08-14
>> 16:12:49,152+0800 expected renewal time: 2024-08-14 15:46:18,534+0800*
>>
>> Although I observed the HDFS delegation token being periodically
>> refreshed in the logs of Flink's JobManager and TaskManager, the AM
>> Container did not use the latest delegation token to start after the job
>> failed and a retry was attempted.
>>
>> So, it seems to me that the new delegation framework of Flink has not
>> solved the problem of using an expired delegation token when the container
>> starts.
>>
>>
>> I think my situation is somewhat similar to the description in the link
>> below, and I am not sure if our new framework has addressed this scenario:
>>
>> *One tricky detail is important. Even if the YARN client sets delegation
>> tokens at the initial stage AM must re-obtain tokens at startup because
>> otherwise AM restart may fail (we’ve struggled with this in Spark). Please
>> consider the following situation:*
>>
>>    - *YARN client obtains delegation tokens and sets them on the
>>    container*
>>    - *AM starts and uses HDFS token for log aggregation*
>>    - *AM dies for whatever reason after HDFS token max lifetime
>>    (typically 7 days)*
>>    - *AM restarts with the old token*
>>    - *Log aggregation fails immediately because of expired token*
>>
>> *(*FLIP-211: Kerberos delegation token framework - Apache Flink - Apache
>> Software Foundation
>> <https://cwiki.apache.org/confluence/display/FLINK/FLIP-211%3A+Kerberos+delegation+token+framework>
>> *)*
>>
>>
>>  You can verify this Kerberos scenario, thank you.
>>
>>
>> ------------------ 原始邮件 ------------------
>> *发件人:* "Gabor Somogyi" <gabor.g.somo...@gmail.com>;
>> *发送时间:* 2024年8月13日(星期二) 晚上11:25
>> *收件人:* "user"<user@flink.apache.org>;
>> *主题:* Re: Flink jobs failed in Kerberos delegation token
>>
>> Hi,
>>
>> As a general suggestion please use the official releases since we're not
>> able to analyze any kind of custom code with cherry-picks with potential
>> hand made conflict resolution.
>>
>> When you state "needs to be restarted due to an exception" then what kind
>> of exception you're are referring to?
>> I mean what kind of file operation is happening? Full stack trace can be
>> useful too...
>>
>> The reason why I'm asking is because there are features which are planned
>> not working, like YARN log aggregation
>> but Flink data processing must work after the TM registered itself at the
>> JM. When the mentioned registration happens then
>> the TM receives a set of fresh tokens which must be used for data
>> processing.
>>
>> BR,
>> G
>>
>>
>>> From: dpp <pengpeng.d...@foxmail.com>
>>> Date: Sat, Aug 10, 2024 at 6:42 AM
>>> Subject: Flink jobs failed in Kerberos delegation token
>>> To: user <user@flink.apache.org>
>>>
>>>
>>> Hello, I am currently using Flink version 1.15.2 and have encountered an
>>> issue with the HDFS delegation token expiring after 7 days in a Kerberos
>>> scenario.
>>> I have seen a new delegation token framework (
>>> https://issues.apache.org/jira/browse/FLINK-21232)  and I have merged
>>> the code commits from 1 to 12 (Sub-Tasks 1-12) in the link into my Flink
>>> version 1.15.2.
>>> Now, it is possible to refresh the delegation token periodically.
>>> However, after 7 days, if the JobManager or TaskManager needs to be
>>> restarted due to an exception, I found that the Yarn container used to
>>> start JM/TM still uses the HDFS_DELEGATION_KIND that was generated the
>>> first time the job was submitted.And it also reports an error similar
>>> to 'token (HDFS_DELEGATION_TOKEN token 31615466 for xx) can't be found in
>>> cache'.
>>> So,the new delegation token framework did not take effect. I'm using the
>>> default method of Flink and delegation tokens are not managed elsewhere.
>>> Could anyone help me with this issue? Thank you very much.
>>>
>>

Reply via email to