Skip to content

Commit 1e9d68b

Browse files
wForgetbowenliang123
authored andcommitted
[KYUUBI #6368] Flink engine supports user impersonation
# 🔍 Description ## Issue References 🔗 This pull request fixes #6368 ## Describe Your Solution 🔧 Support impersonation mode for flink sql engine. ## Types of changes 🔖 - [ ] Bugfix (non-breaking change which fixes an issue) - [X] New feature (non-breaking change which adds functionality) - [ ] Breaking change (fix or feature that would cause existing functionality to change) ## Test Plan 🧪 #### Behavior Without This Pull Request ⚰️ #### Behavior With This Pull Request 🎉 Test in hadoop-testing env. Connection: ``` beeline -u "jdbc:hive2://hadoop-master1.orb.local:10009/default;hive.server2.proxy.user=spark;principal=kyuubi/_HOSTTEST.ORG?kyuubi.engine.type=FLINK_SQL;flink.execution.target=yarn-application;kyuubi.engine.share.level=CONNECTION;kyuubi.engine.flink.doAs.enabled=true;" ``` sql: ``` select 1; ``` result: ![image](https://github.com/apache/kyuubi/assets/17894939/4bde3e4e-0dac-4e09-ac7c-a2c3a3607a13) launch engine command: ``` 2024-06-12 03:22:10.242 INFO KyuubiSessionManager-exec-pool: Thread-62 org.apache.kyuubi.engine.EngineRef: Launching engine: /opt/flink-1.18.1/bin/flink run-application \ -t yarn-application \ -Dyarn.ship-files=/opt/flink/opt/flink-sql-client-1.18.1.jar;/opt/flink/opt/flink-sql-gateway-1.18.1.jar;/etc/hive/conf/hive-site.xml \ -Dyarn.application.name=kyuubi_CONNECTION_FLINK_SQL_spark_6170b9aa-c690-4b50-938f-d59cca9aa2d6 \ -Dyarn.tags=KYUUBI,6170b9aa-c690-4b50-938f-d59cca9aa2d6 \ -Dcontainerized.master.env.FLINK_CONF_DIR=. \ -Dcontainerized.master.env.HIVE_CONF_DIR=. \ -Dyarn.security.appmaster.delegation.token.services=kyuubi \ -Dsecurity.delegation.token.provider.HiveServer2.enabled=false \ -Dsecurity.delegation.token.provider.hbase.enabled=false \ -Dexecution.target=yarn-application \ -Dsecurity.module.factory.classes=org.apache.flink.runtime.security.modules.JaasModuleFactory;org.apache.flink.runtime.security.modules.ZookeeperModuleFa ctory \ -Dsecurity.delegation.token.provider.hadoopfs.enabled=false \ -c org.apache.kyuubi.engine.flink.FlinkSQLEngine /opt/apache-kyuubi-1.10.0-SNAPSHOT-bin/externals/engines/flink/kyuubi-flink-sql-engine_2.12-1.10.0-SNAPS HOT.jar \ --conf kyuubi.session.user=spark \ --conf kyuubi.client.ipAddress=172.20.0.5 \ --conf kyuubi.engine.credentials=SERUUwACJnRocmlmdDovL2hhZG9vcC1tYXN0ZXIxLm9yYi5sb2NhbDo5MDgzRQAFc3BhcmsEaGl2ZShreXV1YmkvaGFkb29wLW1hc3RlcjEub3JiLmxvY2Fs QFRFU1QuT1JHigGQCneevIoBkC6EIrwWDxSg03pnAB8dA295wh+Dim7Fx4FNxhVISVZFX0RFTEVHQVRJT05fVE9LRU4ADzE3Mi4yMC4wLjU6ODAyMEEABXNwYXJrAChreXV1YmkvaGFkb29wLW1hc3RlcjEub3JiL mxvY2FsQFRFU1QuT1JHigGQCneekIoBkC6EIpBHHBSket0SQnlXT5EIMN0U2fUKFRIVvBVIREZTX0RFTEVHQVRJT05fVE9LRU4PMTcyLjIwLjAuNTo4MDIwAA== \ --conf kyuubi.engine.flink.doAs.enabled=true \ --conf kyuubi.engine.hive.extra.classpath=/opt/hadoop/share/hadoop/client/*:/opt/hadoop/share/hadoop/mapreduce/* \ --conf kyuubi.engine.share.level=CONNECTION \ --conf kyuubi.engine.submit.time=1718162530017 \ --conf kyuubi.engine.type=FLINK_SQL \ --conf kyuubi.frontend.protocols=THRIFT_BINARY,REST \ --conf kyuubi.ha.addresses=hadoop-master1.orb.local:2181 \ --conf kyuubi.ha.engine.ref.id=6170b9aa-c690-4b50-938f-d59cca9aa2d6 \ --conf kyuubi.ha.namespace=/kyuubi_1.10.0-SNAPSHOT_CONNECTION_FLINK_SQL/spark/6170b9aa-c690-4b50-938f-d59cca9aa2d6 \ --conf kyuubi.server.ipAddress=172.20.0.5 \ --conf kyuubi.session.connection.url=hadoop-master1.orb.local:10009 \ --conf kyuubi.session.engine.startup.waitCompletion=false \ --conf kyuubi.session.real.user=spark ``` launch engine log: ![image](https://github.com/apache/kyuubi/assets/17894939/590463a8-2858-47a2-8897-0ddfbe3ffdf6) jobmanager job: ``` 2024-06-12 03:22:26,400 INFO org.apache.flink.runtime.security.token.DefaultDelegationTokenManager [] - Loading delegation token providers 2024-06-12 03:22:26,992 INFO org.apache.kyuubi.engine.flink.security.token.KyuubiDelegationTokenProvider [] - Renew delegation token with engine credentials: SERUUwACJnRocmlmdDovL2hhZG9vcC1tYXN0ZXIxLm9yYi5sb2NhbDo5MDgzRQAFc3BhcmsEaGl2ZShreXV1YmkvaGFkb29wLW1hc3RlcjEub3JiLmxvY2FsQFRFU1QuT1JHigGQCneevIoBkC6EIrwWDxSg03pnAB8dA295wh+Dim7Fx4FNxhVISVZFX0RFTEVHQVRJT05fVE9LRU4ADzE3Mi4yMC4wLjU6ODAyMEEABXNwYXJrAChreXV1YmkvaGFkb29wLW1hc3RlcjEub3JiLmxvY2FsQFRFU1QuT1JHigGQCneekIoBkC6EIpBHHBSket0SQnlXT5EIMN0U2fUKFRIVvBVIREZTX0RFTEVHQVRJT05fVE9LRU4PMTcyLjIwLjAuNTo4MDIwAA== 2024-06-12 03:22:27,100 INFO org.apache.kyuubi.engine.flink.FlinkEngineUtils [] - Add new unknown token Kind: HIVE_DELEGATION_TOKEN, Service: , Ident: 00 05 73 70 61 72 6b 04 68 69 76 65 28 6b 79 75 75 62 69 2f 68 61 64 6f 6f 70 2d 6d 61 73 74 65 72 31 2e 6f 72 62 2e 6c 6f 63 61 6c 40 54 45 53 54 2e 4f 52 47 8a 01 90 0a 77 9e bc 8a 01 90 2e 84 22 bc 16 0f 2024-06-12 03:22:27,104 WARN org.apache.kyuubi.engine.flink.FlinkEngineUtils [] - Ignore token with earlier issue date: Kind: HDFS_DELEGATION_TOKEN, Service: 172.20.0.5:8020, Ident: (token for spark: HDFS_DELEGATION_TOKEN owner=spark, renewer=, realUser=kyuubi/hadoop-master1.orb.localTEST.ORG, issueDate=1718162529936, maxDate=1718767329936, sequenceNumber=71, masterKeyId=28) 2024-06-12 03:22:27,104 INFO org.apache.kyuubi.engine.flink.FlinkEngineUtils [] - Update delegation tokens. The number of tokens sent by the server is 2. The actual number of updated tokens is 1. ...... 4-06-12 03:22:29,414 INFO org.apache.flink.runtime.security.token.DefaultDelegationTokenManager [] - Starting tokens update task 2024-06-12 03:22:29,415 INFO org.apache.flink.runtime.security.token.DelegationTokenReceiverRepository [] - New delegation tokens arrived, sending them to receivers 2024-06-12 03:22:29,422 INFO org.apache.kyuubi.engine.flink.security.token.KyuubiDelegationTokenReceiver [] - Updating delegation tokens for current user 2024-06-12 03:22:29,422 INFO org.apache.kyuubi.engine.flink.security.token.KyuubiDelegationTokenReceiver [] - Token Service: Identifier:[10, 13, 10, 9, 8, 10, 16, -78, -36, -49, -17, -5, 49, 16, 1, 16, -100, -112, -60, -127, -8, -1, -1, -1, -1, 1] 2024-06-12 03:22:29,422 INFO org.apache.kyuubi.engine.flink.security.token.KyuubiDelegationTokenReceiver [] - Token Service: Identifier:[0, 5, 115, 112, 97, 114, 107, 4, 104, 105, 118, 101, 40, 107, 121, 117, 117, 98, 105, 47, 104, 97, 100, 111, 111, 112, 45, 109, 97, 115, 116, 101, 114, 49, 46, 111, 114, 98, 46, 108, 111, 99, 97, 108, 64, 84, 69, 83, 84, 46, 79, 82, 71, -118, 1, -112, 10, 119, -98, -68, -118, 1, -112, 46, -124, 34, -68, 22, 15] 2024-06-12 03:22:29,422 INFO org.apache.kyuubi.engine.flink.security.token.KyuubiDelegationTokenReceiver [] - Token Service:172.20.0.5:8020 Identifier:[0, 5, 115, 112, 97, 114, 107, 0, 40, 107, 121, 117, 117, 98, 105, 47, 104, 97, 100, 111, 111, 112, 45, 109, 97, 115, 116, 101, 114, 49, 46, 111, 114, 98, 46, 108, 111, 99, 97, 108, 64, 84, 69, 83, 84, 46, 79, 82, 71, -118, 1, -112, 10, 119, -98, -112, -118, 1, -112, 46, -124, 34, -112, 71, 28] 2024-06-12 03:22:29,422 INFO org.apache.kyuubi.engine.flink.security.token.KyuubiDelegationTokenReceiver [] - Updated delegation tokens for current user successfully ``` taskmanager log: ``` 2024-06-12 03:45:06,622 INFO org.apache.flink.runtime.taskexecutor.TaskExecutor [] - Receive initial delegation tokens from resource manager 2024-06-12 03:45:06,627 INFO org.apache.flink.runtime.security.token.DelegationTokenReceiverRepository [] - New delegation tokens arrived, sending them to receivers 2024-06-12 03:45:06,628 INFO org.apache.kyuubi.engine.flink.security.token.KyuubiDelegationTokenReceiver [] - Updating delegation tokens for current user 2024-06-12 03:45:06,629 INFO org.apache.kyuubi.engine.flink.security.token.KyuubiDelegationTokenReceiver [] - Token Service: Identifier:[10, 13, 10, 9, 8, 10, 16, -78, -36, -49, -17, -5, 49, 16, 1, 16, -100, -112, -60, -127, -8, -1, -1, -1, -1, 1] 2024-06-12 03:45:06,630 INFO org.apache.kyuubi.engine.flink.security.token.KyuubiDelegationTokenReceiver [] - Token Service: Identifier:[0, 5, 115, 112, 97, 114, 107, 4, 104, 105, 118, 101, 40, 107, 121, 117, 117, 98, 105, 47, 104, 97, 100, 111, 111, 112, 45, 109, 97, 115, 116, 101, 114, 49, 46, 111, 114, 98, 46, 108, 111, 99, 97, 108, 64, 84, 69, 83, 84, 46, 79, 82, 71, -118, 1, -112, 10, 119, -98, -68, -118, 1, -112, 46, -124, 34, -68, 22, 15] 2024-06-12 03:45:06,630 INFO org.apache.kyuubi.engine.flink.security.token.KyuubiDelegationTokenReceiver [] - Token Service:172.20.0.5:8020 Identifier:[0, 5, 115, 112, 97, 114, 107, 0, 40, 107, 121, 117, 117, 98, 105, 47, 104, 97, 100, 111, 111, 112, 45, 109, 97, 115, 116, 101, 114, 49, 46, 111, 114, 98, 46, 108, 111, 99, 97, 108, 64, 84, 69, 83, 84, 46, 79, 82, 71, -118, 1, -112, 10, 119, -98, -112, -118, 1, -112, 46, -124, 34, -112, 71, 28] 2024-06-12 03:45:06,636 INFO org.apache.kyuubi.engine.flink.security.token.KyuubiDelegationTokenReceiver [] - Updated delegation tokens for current user successfully 2024-06-12 03:45:06,636 INFO org.apache.flink.runtime.security.token.DelegationTokenReceiverRepository [] - Delegation tokens sent to receivers ``` #### Related Unit Tests --- # Checklist 📝 - [X] This patch was not authored or co-authored using [Generative Tooling](https://www.apache.org/legal/generative-tooling.html) **Be nice. Be informative.** Closes #6383 from wForget/KYUUBI-6368. Closes #6368 47df43e [wforget] remove doAsEnabled 984b96c [wforget] update settings.md c7f8d47 [wforget] make generateTokenFile conf to internal 8632176 [wforget] address comments 2ec270e [wforget] licenses ed0e22f [wforget] separate kyuubi-flink-token-provider module b66b855 [wforget] address comment d4fc2bd [wforget] fix 1a3dc46 [wforget] fix style 825e2a7 [wforget] address comments a679ba1 [wforget] revert remove renewer cdd499b [wforget] fix and comment 19caec6 [wforget] pass token to submit process b2991d4 [wforget] fix 7c3bdde [wforget] remove security.delegation.tokens.enabled check 8987c91 [wforget] fix 5bd8cfe [wforget] fix 0899264 [wforget] Implement KyuubiDelegationToken Provider/Receiver fa16d7d [wforget] enable delegation token manager e50db74 [wforget] [KYUUBI #6368] Support impersonation mode for flink sql engine Authored-by: wforget <[email protected]> Signed-off-by: Bowen Liang <[email protected]>
1 parent fb65a12 commit 1e9d68b

File tree

13 files changed

+525
-43
lines changed

13 files changed

+525
-43
lines changed

docs/configuration/settings.md

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -149,6 +149,7 @@ You can configure the Kyuubi properties in `$KYUUBI_HOME/conf/kyuubi-defaults.co
149149
| kyuubi.engine.event.json.log.path | file:///tmp/kyuubi/events | The location where all the engine events go for the built-in JSON logger.<ul><li>Local Path: start with 'file://'</li><li>HDFS Path: start with 'hdfs://'</li></ul> | string | 1.3.0 |
150150
| kyuubi.engine.event.loggers | SPARK | A comma-separated list of engine history loggers, where engine/session/operation etc events go.<ul> <li>SPARK: the events will be written to the Spark listener bus.</li> <li>JSON: the events will be written to the location of kyuubi.engine.event.json.log.path</li> <li>JDBC: to be done</li> <li>CUSTOM: User-defined event handlers.</li></ul> Note that: Kyuubi supports custom event handlers with the Java SPI. To register a custom event handler, the user needs to implement a subclass of `org.apache.kyuubi.events.handler.CustomEventHandlerProvider` which has a zero-arg constructor. | seq | 1.3.0 |
151151
| kyuubi.engine.flink.application.jars | &lt;undefined&gt; | A comma-separated list of the local jars to be shipped with the job to the cluster. For example, SQL UDF jars. Only effective in yarn application mode. | string | 1.8.0 |
152+
| kyuubi.engine.flink.doAs.enabled | false | When enabled, the session user is used as the proxy user to launch the Flink engine, otherwise, the server user. Note, due to the limitation of Apache Flink, it can only be enabled on Kerberized environment. | boolean | 1.10.0 |
152153
| kyuubi.engine.flink.extra.classpath | &lt;undefined&gt; | The extra classpath for the Flink SQL engine, for configuring the location of hadoop client jars, etc. Only effective in yarn session mode. | string | 1.6.0 |
153154
| kyuubi.engine.flink.initialize.sql | SHOW DATABASES | The initialize sql for Flink engine. It fallback to `kyuubi.engine.initialize.sql`. | seq | 1.8.1 |
154155
| kyuubi.engine.flink.java.options | &lt;undefined&gt; | The extra Java options for the Flink SQL engine. Only effective in yarn session mode. | string | 1.6.0 |
Lines changed: 78 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,78 @@
1+
<?xml version="1.0" encoding="UTF-8"?>
2+
<!--
3+
~ Licensed to the Apache Software Foundation (ASF) under one or more
4+
~ contributor license agreements. See the NOTICE file distributed with
5+
~ this work for additional information regarding copyright ownership.
6+
~ The ASF licenses this file to You under the Apache License, Version 2.0
7+
~ (the "License"); you may not use this file except in compliance with
8+
~ the License. You may obtain a copy of the License at
9+
~
10+
~ http://www.apache.org/licenses/LICENSE-2.0
11+
~
12+
~ Unless required by applicable law or agreed to in writing, software
13+
~ distributed under the License is distributed on an "AS IS" BASIS,
14+
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
~ See the License for the specific language governing permissions and
16+
~ limitations under the License.
17+
-->
18+
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
19+
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
20+
<modelVersion>4.0.0</modelVersion>
21+
<parent>
22+
<groupId>org.apache.kyuubi</groupId>
23+
<artifactId>kyuubi-parent</artifactId>
24+
<version>1.10.0-SNAPSHOT</version>
25+
<relativePath>../../../pom.xml</relativePath>
26+
</parent>
27+
28+
<artifactId>kyuubi-flink-token-provider</artifactId>
29+
<packaging>jar</packaging>
30+
<name>Kyuubi Flink Token Provider</name>
31+
<url>https://kyuubi.apache.org/</url>
32+
33+
<dependencies>
34+
<!-- flink -->
35+
<dependency>
36+
<groupId>org.apache.flink</groupId>
37+
<artifactId>flink-core</artifactId>
38+
<scope>provided</scope>
39+
</dependency>
40+
41+
<dependency>
42+
<groupId>org.apache.flink</groupId>
43+
<artifactId>flink-runtime</artifactId>
44+
<scope>provided</scope>
45+
</dependency>
46+
47+
<dependency>
48+
<groupId>org.apache.flink</groupId>
49+
<artifactId>flink-clients</artifactId>
50+
<scope>provided</scope>
51+
</dependency>
52+
53+
<!-- hadoop client -->
54+
<dependency>
55+
<groupId>org.apache.hadoop</groupId>
56+
<artifactId>hadoop-client-api</artifactId>
57+
<scope>provided</scope>
58+
</dependency>
59+
60+
<dependency>
61+
<groupId>org.apache.hadoop</groupId>
62+
<artifactId>hadoop-client-runtime</artifactId>
63+
<scope>provided</scope>
64+
</dependency>
65+
</dependencies>
66+
67+
<build>
68+
<plugins>
69+
<plugin>
70+
<groupId>net.alchim31.maven</groupId>
71+
<artifactId>scala-maven-plugin</artifactId>
72+
<configuration>
73+
<skip>true</skip>
74+
</configuration>
75+
</plugin>
76+
</plugins>
77+
</build>
78+
</project>
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,97 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.kyuubi.flink.security.token;
19+
20+
import static org.apache.flink.client.deployment.application.ApplicationConfiguration.APPLICATION_ARGS;
21+
22+
import java.util.HashMap;
23+
import java.util.List;
24+
import java.util.Map;
25+
import java.util.Optional;
26+
import org.apache.commons.lang3.StringUtils;
27+
import org.apache.flink.configuration.ConfigUtils;
28+
import org.apache.flink.configuration.Configuration;
29+
import org.apache.flink.core.security.token.DelegationTokenProvider;
30+
import org.apache.flink.runtime.security.token.hadoop.HadoopDelegationTokenConverter;
31+
import org.apache.hadoop.io.Text;
32+
import org.apache.hadoop.security.Credentials;
33+
import org.apache.hadoop.security.UserGroupInformation;
34+
import org.apache.hadoop.security.token.Token;
35+
import org.apache.hadoop.security.token.TokenIdentifier;
36+
import org.apache.kyuubi.flink.security.token.utils.KyuubiUtils;
37+
import org.slf4j.Logger;
38+
import org.slf4j.LoggerFactory;
39+
40+
public class KyuubiDelegationTokenProvider implements DelegationTokenProvider {
41+
42+
private static final Logger LOG = LoggerFactory.getLogger(KyuubiDelegationTokenProvider.class);
43+
44+
public static volatile Map<Text, Token<? extends TokenIdentifier>> previousTokens;
45+
46+
private long renewalInterval;
47+
48+
@Override
49+
public void init(Configuration configuration) throws Exception {
50+
final List<String> programArgsList =
51+
ConfigUtils.decodeListFromConfig(configuration, APPLICATION_ARGS, String::new);
52+
Map<String, String> kyuubiConf = KyuubiUtils.fromCommandLineArgs(programArgsList);
53+
String engineCredentials =
54+
kyuubiConf.getOrDefault(KyuubiUtils.KYUUBI_ENGINE_CREDENTIALS_KEY, "");
55+
if (StringUtils.isNotBlank(engineCredentials)) {
56+
LOG.info("Renew delegation token with engine credentials: {}", engineCredentials);
57+
KyuubiUtils.renewDelegationToken(engineCredentials);
58+
}
59+
Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials();
60+
previousTokens = new HashMap<>(credentials.getTokenMap());
61+
String interval = kyuubiConf.get(KyuubiUtils.KYUUBI_CREDENTIALS_RENEWAL_INTERVAL_KEY);
62+
if (StringUtils.isNotBlank(interval)) {
63+
renewalInterval = Long.parseLong(interval);
64+
} else {
65+
renewalInterval = KyuubiUtils.KYUUBI_CREDENTIALS_RENEWAL_INTERVAL_DEFAULT;
66+
}
67+
}
68+
69+
@Override
70+
public ObtainedDelegationTokens obtainDelegationTokens() throws Exception {
71+
Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials();
72+
Credentials newCredentials = new Credentials();
73+
for (Map.Entry<Text, Token<? extends TokenIdentifier>> tokenEntry :
74+
credentials.getTokenMap().entrySet()) {
75+
Text alias = tokenEntry.getKey();
76+
Token<? extends TokenIdentifier> token = tokenEntry.getValue();
77+
Token<? extends TokenIdentifier> previousToken = previousTokens.get(alias);
78+
if (previousToken == null || KyuubiUtils.compareIssueDate(token, previousToken) > 0) {
79+
newCredentials.addToken(alias, token);
80+
}
81+
}
82+
previousTokens = new HashMap<>(credentials.getTokenMap());
83+
Optional<Long> validUntil = Optional.of(System.currentTimeMillis() + renewalInterval);
84+
return new ObtainedDelegationTokens(
85+
HadoopDelegationTokenConverter.serialize(credentials), validUntil);
86+
}
87+
88+
@Override
89+
public boolean delegationTokensRequired() throws Exception {
90+
return true;
91+
}
92+
93+
@Override
94+
public String serviceName() {
95+
return "kyuubi";
96+
}
97+
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,28 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.kyuubi.flink.security.token;
19+
20+
import org.apache.flink.runtime.security.token.hadoop.HadoopDelegationTokenReceiver;
21+
22+
public class KyuubiDelegationTokenReceiver extends HadoopDelegationTokenReceiver {
23+
24+
@Override
25+
public String serviceName() {
26+
return "kyuubi";
27+
}
28+
}

0 commit comments

Comments
 (0)