[
https://issues.apache.org/jira/browse/FLINK-2977?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14998382#comment-14998382
]
ASF GitHub Bot commented on FLINK-2977:
---------------------------------------
Github user nielsbasjes commented on a diff in the pull request:
https://github.com/apache/flink/pull/1342#discussion_r44391146
--- Diff: flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java ---
@@ -135,7 +142,40 @@ public static void setTokensFor(ContainerLaunchContext
amContainer, Path[] paths
ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(), 0,
dob.getLength());
amContainer.setTokens(securityTokens);
}
-
+
+ /**
+ * Obtain Kerberos security token for HBase.
+ */
+ private static void obtainTokenForHBase(Credentials credentials,
Configuration conf) {
+ if (UserGroupInformation.isSecurityEnabled()) {
+ LOG.info("Attempting to obtain Kerberos security token
for HBase");
+ try {
+ HBaseConfiguration.addHbaseResources(conf);
+ if
(!"kerberos".equals(conf.get("hbase.security.authentication"))) {
+ LOG.info("HBase has not been configured
to use Kerberos.");
+ return;
+ }
+
+ LOG.info("Connecting to HBase");
+ Connection connection =
ConnectionFactory.createConnection(conf);
+
+ LOG.info("Obtaining Kerberos security token for
HBase");
+ Token<AuthenticationTokenIdentifier> token =
TokenUtil.obtainToken(connection);
+
+ if (token == null) {
+ LOG.error("No Kerberos security token
for HBase available");
+ return;
+ }
+
+ credentials.addToken(token.getService(), token);
+ LOG.info("Added HBase Kerberos security token
to credentials.");
+ } catch (IOException e) {
+ LOG.error("Caught exception while trying to
obtain HBase Kerberos security token.");
+ e.printStackTrace();
--- End diff --
Yes I considered that. I figured that continuing would be 'better'. But
simply rethrowing would make more sense.
I'll update the patch.
> Cannot access HBase in a Kerberos secured Yarn cluster
> ------------------------------------------------------
>
> Key: FLINK-2977
> URL: https://issues.apache.org/jira/browse/FLINK-2977
> Project: Flink
> Issue Type: Bug
> Components: YARN Client
> Reporter: Niels Basjes
> Assignee: Niels Basjes
> Attachments: FLINK-2977-20151005-untested.patch,
> FLINK-2977-20151009.patch
>
>
> I have created a very simple Flink topology consisting of a streaming Source
> (the outputs the timestamp a few times per second) and a Sink (that puts that
> timestamp into a single record in HBase).
> Running this on a non-secure Yarn cluster works fine.
> To run it on a secured Yarn cluster my main routine now looks like this:
> {code}
> public static void main(String[] args) throws Exception {
> System.setProperty("java.security.krb5.conf", "/etc/krb5.conf");
> UserGroupInformation.loginUserFromKeytab("[email protected]",
> "/home/nbasjes/.krb/nbasjes.keytab");
> final StreamExecutionEnvironment env =
> StreamExecutionEnvironment.getExecutionEnvironment();
> env.setParallelism(1);
> DataStream<String> stream = env.addSource(new TimerTicksSource());
> stream.addSink(new SetHBaseRowSink());
> env.execute("Long running Flink application");
> }
> {code}
> When I run this
> flink run -m yarn-cluster -yn 1 -yjm 1024 -ytm 4096
> ./kerberos-1.0-SNAPSHOT.jar
> I see after the startup messages:
> {quote}
> 17:13:24,466 INFO org.apache.hadoop.security.UserGroupInformation
> - Login successful for user [email protected] using keytab file
> /home/nbasjes/.krb/nbasjes.keytab
> 11/03/2015 17:13:25 Job execution switched to status RUNNING.
> 11/03/2015 17:13:25 Custom Source -> Stream Sink(1/1) switched to SCHEDULED
> 11/03/2015 17:13:25 Custom Source -> Stream Sink(1/1) switched to DEPLOYING
> 11/03/2015 17:13:25 Custom Source -> Stream Sink(1/1) switched to RUNNING
> {quote}
> Which looks good.
> However ... no data goes into HBase.
> After some digging I found this error in the task managers log:
> {quote}
> 17:13:42,677 WARN org.apache.hadoop.hbase.ipc.RpcClient
> - Exception encountered while connecting to the server :
> javax.security.sasl.SaslException: GSS initiate failed [Caused by
> GSSException: No valid credentials provided (Mechanism level: Failed to find
> any Kerberos tgt)]
> 17:13:42,677 FATAL org.apache.hadoop.hbase.ipc.RpcClient
> - SASL authentication failed. The most likely cause is missing or invalid
> credentials. Consider 'kinit'.
> javax.security.sasl.SaslException: GSS initiate failed [Caused by
> GSSException: No valid credentials provided (Mechanism level: Failed to find
> any Kerberos tgt)]
> at
> com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:212)
> at
> org.apache.hadoop.hbase.security.HBaseSaslRpcClient.saslConnect(HBaseSaslRpcClient.java:177)
> at
> org.apache.hadoop.hbase.ipc.RpcClient$Connection.setupSaslConnection(RpcClient.java:815)
> at
> org.apache.hadoop.hbase.ipc.RpcClient$Connection.access$800(RpcClient.java:349)
> {quote}
> First starting a yarn-session and then loading my job gives the same error.
> My best guess at this point is that Flink needs the same fix as described
> here:
> https://issues.apache.org/jira/browse/SPARK-6918 (
> https://github.com/apache/spark/pull/5586 )
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)