Skip to content

Commit 8cb12de

Browse files
committed
[FLINK-21700][security] Set service name as credential alias
1 parent 60812bb commit 8cb12de

File tree

4 files changed

+6
-9
lines changed

4 files changed

+6
-9
lines changed

docs/layouts/shortcodes/generated/security_auth_kerberos_section.html

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -12,7 +12,7 @@
1212
<td><h5>security.kerberos.fetch.delegation-token</h5></td>
1313
<td style="word-wrap: break-word;">true</td>
1414
<td>Boolean</td>
15-
<td>Indicates whether to fetch the delegation tokens for external services the Flink job needs to contact. Only HDFS and HBase are supported. It is used in Yarn deployments. If true, Flink will fetch HDFS and HBase delegation tokens and inject them into Yarn AM containers. If false, Flink will assume that the delegation tokens are managed outside of Flink. As a consequence, it will not fetch delegation tokens for HDFS and HBase. You may need to disable this option, if you rely on submission mechanisms, e.g. Apache Oozie, to handle delegation tokens.</td>
15+
<td>Indicates whether to fetch the delegation tokens for external services the Flink job needs to contact. Only HDFS and HBase are supported. It is used in Yarn deployments. If true, Flink will fetch HDFS and HBase delegation tokens and inject them into Yarn AM containers. If false, Flink will assume that the delegation tokens are managed outside of Flink. As a consequence, it will not fetch delegation tokens for HDFS and HBase. You may need to disable this option, if you rely on submission mechanisms, e.g. Apache Oozie, to handle delegation tokens.</td>
1616
</tr>
1717
<tr>
1818
<td><h5>security.kerberos.login.contexts</h5></td>

docs/layouts/shortcodes/generated/security_configuration.html

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,7 @@
1818
<td><h5>security.kerberos.fetch.delegation-token</h5></td>
1919
<td style="word-wrap: break-word;">true</td>
2020
<td>Boolean</td>
21-
<td>Indicates whether to fetch the delegation tokens for external services the Flink job needs to contact. Only HDFS and HBase are supported. It is used in Yarn deployments. If true, Flink will fetch HDFS and HBase delegation tokens and inject them into Yarn AM containers. If false, Flink will assume that the delegation tokens are managed outside of Flink. As a consequence, it will not fetch delegation tokens for HDFS and HBase. You may need to disable this option, if you rely on submission mechanisms, e.g. Apache Oozie, to handle delegation tokens.</td>
21+
<td>Indicates whether to fetch the delegation tokens for external services the Flink job needs to contact. Only HDFS and HBase are supported. It is used in Yarn deployments. If true, Flink will fetch HDFS and HBase delegation tokens and inject them into Yarn AM containers. If false, Flink will assume that the delegation tokens are managed outside of Flink. As a consequence, it will not fetch delegation tokens for HDFS and HBase. You may need to disable this option, if you rely on submission mechanisms, e.g. Apache Oozie, to handle delegation tokens.</td>
2222
</tr>
2323
<tr>
2424
<td><h5>security.kerberos.krb5-conf.path</h5></td>

flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -116,7 +116,7 @@ public class SecurityOptions {
116116
.withDescription(
117117
"Indicates whether to fetch the delegation tokens for external services the Flink job needs to contact. "
118118
+ "Only HDFS and HBase are supported. It is used in Yarn deployments. "
119-
+ "If true, Flink will fetch HDFS and HBase delegation tokens and inject them into Yarn AM containers. "
119+
+ "If true, Flink will fetch HDFS and HBase delegation tokens and inject them into Yarn AM containers. "
120120
+ "If false, Flink will assume that the delegation tokens are managed outside of Flink. "
121121
+ "As a consequence, it will not fetch delegation tokens for HDFS and HBase. "
122122
+ "You may need to disable this option, if you rely on submission mechanisms, e.g. Apache Oozie, "

flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java

Lines changed: 3 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,6 @@
3434
import org.apache.hadoop.fs.FileSystem;
3535
import org.apache.hadoop.fs.Path;
3636
import org.apache.hadoop.io.DataOutputBuffer;
37-
import org.apache.hadoop.io.Text;
3837
import org.apache.hadoop.mapreduce.security.TokenCache;
3938
import org.apache.hadoop.security.Credentials;
4039
import org.apache.hadoop.security.UserGroupInformation;
@@ -219,9 +218,8 @@ public static void setTokensFor(
219218

220219
Collection<Token<? extends TokenIdentifier>> usrTok = currUsr.getTokens();
221220
for (Token<? extends TokenIdentifier> token : usrTok) {
222-
final Text id = new Text(token.getIdentifier());
223-
LOG.info("Adding user token " + id + " with " + token);
224-
credentials.addToken(id, token);
221+
LOG.info("Adding user token " + token.getService() + " with " + token);
222+
credentials.addToken(token.getService(), token);
225223
}
226224
try (DataOutputBuffer dob = new DataOutputBuffer()) {
227225
credentials.writeTokenStorageToStream(dob);
@@ -570,8 +568,7 @@ static ContainerLaunchContext createTaskExecutorContext(
570568
Collection<Token<? extends TokenIdentifier>> userTokens = cred.getAllTokens();
571569
for (Token<? extends TokenIdentifier> token : userTokens) {
572570
if (!token.getKind().equals(AMRMTokenIdentifier.KIND_NAME)) {
573-
final Text id = new Text(token.getIdentifier());
574-
taskManagerCred.addToken(id, token);
571+
taskManagerCred.addToken(token.getService(), token);
575572
}
576573
}
577574

0 commit comments

Comments
 (0)