You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by gy...@apache.org on 2020/09/28 14:33:21 UTC
[flink] 01/02: [FLINK-18795][hbase] Support HBase 2 delegation
tokens
This is an automated email from the ASF dual-hosted git repository.
gyfora pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
commit f59b5a1d375c8e7fe6af5502eba2fbacacf0c837
Author: Felipe Lolas <fl...@alumnos.uai.cl>
AuthorDate: Tue Aug 25 13:18:19 2020 +0200
[FLINK-18795][hbase] Support HBase 2 delegation tokens
---
.../src/main/java/org/apache/flink/yarn/Utils.java | 49 ++++++++++++++++------
1 file changed, 37 insertions(+), 12 deletions(-)
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
index 8f2eb0f..20264e6 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.yarn.util.Records;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.io.Closeable;
import java.io.File;
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
@@ -211,9 +212,9 @@ public final class Utils {
// ----
// Intended call: HBaseConfiguration.addHbaseResources(conf);
Class
- .forName("org.apache.hadoop.hbase.HBaseConfiguration")
- .getMethod("addHbaseResources", Configuration.class)
- .invoke(null, conf);
+ .forName("org.apache.hadoop.hbase.HBaseConfiguration")
+ .getMethod("addHbaseResources", Configuration.class)
+ .invoke(null, conf);
// ----
LOG.info("HBase security setting: {}", conf.get("hbase.security.authentication"));
@@ -223,14 +224,38 @@ public final class Utils {
return;
}
- LOG.info("Obtaining Kerberos security token for HBase");
- // ----
- // Intended call: Token<AuthenticationTokenIdentifier> token = TokenUtil.obtainToken(conf);
- Token<?> token = (Token<?>) Class
+ Token<?> token;
+ try {
+ LOG.info("Obtaining Kerberos security token for HBase");
+ // ----
+ // Intended call: Token<AuthenticationTokenIdentifier> token = TokenUtil.obtainToken(conf);
+ token = (Token<?>) Class
.forName("org.apache.hadoop.hbase.security.token.TokenUtil")
.getMethod("obtainToken", Configuration.class)
.invoke(null, conf);
- // ----
+ // ----
+ } catch (NoSuchMethodException e){
+ // for HBase 2
+
+ // ----
+ // Intended call: ConnectionFactory connectionFactory = ConnectionFactory.createConnection(conf);
+ Closeable connectionFactory = (Closeable) Class
+ .forName("org.apache.hadoop.hbase.client.ConnectionFactory")
+ .getMethod("createConnection", Configuration.class)
+ .invoke(null, conf);
+ // ----
+ Class<?> connectionClass = Class.forName("org.apache.hadoop.hbase.client.Connection");
+ // ----
+ // Intended call: Token<AuthenticationTokenIdentifier> token = TokenUtil.obtainToken(connectionFactory);
+ token = (Token<?>) Class
+ .forName("org.apache.hadoop.hbase.security.token.TokenUtil")
+ .getMethod("obtainToken", connectionClass)
+ .invoke(null, connectionFactory);
+ // ----
+ if (null != connectionFactory){
+ connectionFactory.close();
+ }
+ }
if (token == null) {
LOG.error("No Kerberos security token for HBase available");
@@ -240,11 +265,11 @@ public final class Utils {
credentials.addToken(token.getService(), token);
LOG.info("Added HBase Kerberos security token to credentials.");
} catch (ClassNotFoundException
- | NoSuchMethodException
- | IllegalAccessException
- | InvocationTargetException e) {
+ | NoSuchMethodException
+ | IllegalAccessException
+ | InvocationTargetException e) {
LOG.info("HBase is not available (not packaged with this application): {} : \"{}\".",
- e.getClass().getSimpleName(), e.getMessage());
+ e.getClass().getSimpleName(), e.getMessage());
}
}
}