Return-Path: X-Original-To: apmail-flink-user-archive@minotaur.apache.org Delivered-To: apmail-flink-user-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id A8AD318230 for ; Thu, 5 Nov 2015 13:57:25 +0000 (UTC) Received: (qmail 54492 invoked by uid 500); 5 Nov 2015 13:57:20 -0000 Delivered-To: apmail-flink-user-archive@flink.apache.org Received: (qmail 54411 invoked by uid 500); 5 Nov 2015 13:57:20 -0000 Mailing-List: contact user-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@flink.apache.org Delivered-To: mailing list user@flink.apache.org Received: (qmail 54402 invoked by uid 99); 5 Nov 2015 13:57:20 -0000 Received: from mail-relay.apache.org (HELO mail-relay.apache.org) (140.211.11.15) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 05 Nov 2015 13:57:20 +0000 Received: from mail-yk0-f174.google.com (mail-yk0-f174.google.com [209.85.160.174]) by mail-relay.apache.org (ASF Mail Server at mail-relay.apache.org) with ESMTPSA id 0E2AE1A006D for ; Thu, 5 Nov 2015 13:57:20 +0000 (UTC) Received: by ykba4 with SMTP id a4so131216250ykb.3 for ; Thu, 05 Nov 2015 05:57:19 -0800 (PST) X-Gm-Message-State: ALoCoQlyZ0LmgfcrA4oBdlUpqnpoSBIf24V9vf9iJVohmK9GcnaCJp3jHQ9HiMaSKD6LB77QklNm X-Received: by 10.31.15.84 with SMTP id 81mr7055516vkp.142.1446731839245; Thu, 05 Nov 2015 05:57:19 -0800 (PST) MIME-Version: 1.0 Received: by 10.31.61.133 with HTTP; Thu, 5 Nov 2015 05:56:59 -0800 (PST) In-Reply-To: References: From: Maximilian Michels Date: Thu, 5 Nov 2015 14:56:59 +0100 X-Gmail-Original-Message-ID: Message-ID: Subject: Re: Running continuously on yarn with kerberos To: "user@flink.apache.org" Content-Type: multipart/alternative; boundary=001a114377925d19210523cb8198 --001a114377925d19210523cb8198 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: quoted-printable Thank you for looking into the problem, Niels. Let us know if you need anything. We would be happy to merge a pull request once you have verified the fix. On Thu, Nov 5, 2015 at 1:38 PM, Niels Basjes wrote: > I created https://issues.apache.org/jira/browse/FLINK-2977 > > On Thu, Nov 5, 2015 at 12:25 PM, Robert Metzger > wrote: > >> Hi Niels, >> thank you for analyzing the issue so properly. I agree with you. It seem= s >> that HDFS and HBase are using their own tokes which we need to transfer >> from the client to the YARN containers. We should be able to port the fi= x >> from Spark (which they got from Storm) into our YARN client. >> I think we would add this in org.apache.flink.yarn.Utils#setTokensFor(). >> >> Do you want to implement and verify the fix yourself? If you are to busy >> at the moment, we can also discuss how we share the work (I'm implementi= ng >> it, you test the fix) >> >> >> Robert >> >> On Tue, Nov 3, 2015 at 5:26 PM, Niels Basjes wrote: >> >>> Update on the status so far.... I suspect I found a problem in a secure >>> setup. >>> >>> 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 (t= hat >>> 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= : >>> >>> public static void main(String[] args) throws Exception { >>> System.setProperty("java.security.krb5.conf", "/etc/krb5.conf"); >>> UserGroupInformation.loginUserFromKeytab("nbasjes@xxxxxx.NET", "/ho= me/nbasjes/.krb/nbasjes.keytab"); >>> >>> final StreamExecutionEnvironment env =3D StreamExecutionEnvironment= .getExecutionEnvironment(); >>> env.setParallelism(1); >>> >>> DataStream stream =3D env.addSource(new TimerTicksSource())= ; >>> stream.addSink(new SetHBaseRowSink()); >>> env.execute("Long running Flink application"); >>> } >>> >>> 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: >>> >>> 17:13:24,466 INFO org.apache.hadoop.security.UserGroupInformation >>> - Login successful for user nbasjes@xxxxxx.NET 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 >>> >>> Which looks good. >>> >>> However ... no data goes into HBase. >>> After some digging I found this error in the task managers log: >>> >>> 17:13:42,677 WARN org.apache.hadoop.hbase.ipc.RpcClient = - Exception encountered while connecting to the server : javax.se= curity.sasl.SaslException: GSS initiate failed [Caused by GSSException: No = valid credentials provided (Mechanism level: Failed to find any Kerberos tg= t)] >>> 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 GSSEx= ception: No valid credentials provided (Mechanism level: Failed to find any= Kerberos tgt)] >>> at com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKr= b5Client.java:212) >>> at org.apache.hadoop.hbase.security.HBaseSaslRpcClient.saslConnect(HBa= seSaslRpcClient.java:177) >>> at org.apache.hadoop.hbase.ipc.RpcClient$Connection.setupSaslConnectio= n(RpcClient.java:815) >>> at org.apache.hadoop.hbase.ipc.RpcClient$Connection.access$800(RpcClie= nt.java:349) >>> >>> >>> 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 ) >>> >>> What do you guys think? >>> >>> Niels Basjes >>> >>> >>> >>> On Tue, Oct 27, 2015 at 6:12 PM, Maximilian Michels >>> wrote: >>> >>>> Hi Niels, >>>> >>>> You're welcome. Some more information on how this would be configured: >>>> >>>> In the kdc.conf, there are two variables: >>>> >>>> max_life =3D 2h 0m 0s >>>> max_renewable_life =3D 7d 0h 0m 0s >>>> >>>> max_life is the maximum life of the current ticket. However, it may be >>>> renewed up to a time span of max_renewable_life from the first ticket = issue >>>> on. This means that from the first ticket issue, new tickets may be >>>> requested for one week. Each renewed ticket has a life time of max_lif= e (2 >>>> hours in this case). >>>> >>>> Please let us know about any difficulties with long-running streaming >>>> application and Kerberos. >>>> >>>> Best regards, >>>> Max >>>> >>>> On Tue, Oct 27, 2015 at 2:46 PM, Niels Basjes wrote: >>>> >>>>> Hi, >>>>> >>>>> Thanks for your feedback. >>>>> So I guess I'll have to talk to the security guys about having specia= l >>>>> kerberos ticket expiry times for these types of jobs. >>>>> >>>>> Niels Basjes >>>>> >>>>> On Fri, Oct 23, 2015 at 11:45 AM, Maximilian Michels >>>>> wrote: >>>>> >>>>>> Hi Niels, >>>>>> >>>>>> Thank you for your question. Flink relies entirely on the Kerberos >>>>>> support of Hadoop. So your question could also be rephrased to "Does >>>>>> Hadoop support long-term authentication using Kerberos?". And the >>>>>> answer is: Yes! >>>>>> >>>>>> While Hadoop uses Kerberos tickets to authenticate users with servic= es >>>>>> initially, the authentication process continues differently >>>>>> afterwards. Instead of saving the ticket to authenticate on a later >>>>>> access, Hadoop creates its own security tockens (DelegationToken) th= at >>>>>> it passes around. These are authenticated to Kerberos periodically. = To >>>>>> my knowledge, the tokens have a life span identical to the Kerberos >>>>>> ticket maximum life span. So be sure to set the maximum life span ve= ry >>>>>> high for long streaming jobs. The renewal time, on the other hand, i= s >>>>>> not important because Hadoop abstracts this away using its own >>>>>> security tockens. >>>>>> >>>>>> I'm afraid there is not Kerberos how-to yet. If you are on Yarn, the= n >>>>>> it is sufficient to authenticate the client with Kerberos. On a Flin= k >>>>>> standalone cluster you need to ensure that, initially, all nodes are >>>>>> authenticated with Kerberos using the kinit tool. >>>>>> >>>>>> Feel free to ask if you have more questions and let us know about an= y >>>>>> difficulties. >>>>>> >>>>>> Best regards, >>>>>> Max >>>>>> >>>>>> >>>>>> >>>>>> On Thu, Oct 22, 2015 at 2:06 PM, Niels Basjes >>>>>> wrote: >>>>>> > Hi, >>>>>> > >>>>>> > I want to write a long running (i.e. never stop it) streaming flin= k >>>>>> > application on a kerberos secured Hadoop/Yarn cluster. My >>>>>> application needs >>>>>> > to do things with files on HDFS and HBase tables on that cluster s= o >>>>>> having >>>>>> > the correct kerberos tickets is very important. The stream is to b= e >>>>>> ingested >>>>>> > from Kafka. >>>>>> > >>>>>> > One of the things with Kerberos is that the tickets expire after a >>>>>> > predetermined time. My knowledge about kerberos is very limited so >>>>>> I hope >>>>>> > you guys can help me. >>>>>> > >>>>>> > My question is actually quite simple: Is there an howto somewhere >>>>>> on how to >>>>>> > correctly run a long running flink application with kerberos that >>>>>> includes a >>>>>> > solution for the kerberos ticket timeout ? >>>>>> > >>>>>> > Thanks >>>>>> > >>>>>> > Niels Basjes >>>>>> >>>>> >>>>> >>>>> >>>>> -- >>>>> Best regards / Met vriendelijke groeten, >>>>> >>>>> Niels Basjes >>>>> >>>> >>>> >>> >>> >>> -- >>> Best regards / Met vriendelijke groeten, >>> >>> Niels Basjes >>> >> >> > > > -- > Best regards / Met vriendelijke groeten, > > Niels Basjes > --001a114377925d19210523cb8198 Content-Type: text/html; charset=UTF-8 Content-Transfer-Encoding: quoted-printable
Thank you for looking into the problem, Niels. Let us know= if you need anything. We would be happy to merge a pull request once you h= ave verified the fix.

On Thu, Nov 5, 2015 at 1:38 PM, Niels Basjes <Niels@basjes.nl<= /a>> wrote:
=

On Thu, Nov 5, 2015 at 12:25 PM, Robert Metzger <= rmetzger@apache.org> wrote:
Hi Niels,
thank you for analyzing the issue so proper= ly. I agree with you. It seems that HDFS and HBase are using their own toke= s which we need to transfer from the client to the YARN containers. We shou= ld be able to port the fix from Spark (which they got from Storm) into our = YARN client.=C2=A0
I think we would add this in=C2=A0= org.apache.flink.yarn.Utils#setTokensFo= r().

Do you want to implement and verify the fix= yourself? If you are to busy at the moment, we can also discuss how we sha= re the work (I'm implementing it, you test the fix)


Robert

On Tu= e, Nov 3, 2015 at 5:26 PM, Niels Basjes <Niels@basjes.nl> wrot= e:
Update on the status = so far.... I suspect I found a problem in a secure setup.

I have created a very simple Flink topology consisting of a streamin= g Source (the outputs the timestamp a few times per second) and a Sink (tha= t puts that timestamp into a single record in HBase).
Running thi= s on a non-secure Yarn cluster works fine.

To run = it on a secured Yarn cluster my main routine now looks like this:

public static void main(String[] args) throws Exception {
System.setProperty("java.security.krb5.conf", "/etc/krb5.conf"= );
UserGroupInformation.loginUserF= romKeytab("= nbasjes@xxxxxx.NET", "/home/nbasjes/.krb/nbasjes.keytab");

= final StreamExe= cutionEnvironment env =3D StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(1);

DataStream<String&= gt; stream =3D env.addSource(new TimerTicksSource());
stream.addSink(new SetHBaseRowSink());
= env.execute("Long = running Flink application");
}
When I ru= n this=C2=A0
=C2=A0 =C2=A0 =C2=A0flink run -m yarn-cluster -yn 1 = -yjm 1024 -ytm 4096 ./kerberos-1.0-SNAPSHOT.jar

I see after the startup messages:

17:13= :24,466 INFO =C2=A0org.apache.hadoop.security.UserGroupInformation =C2=A0 = =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 - Login successful for user nbasj= es@xxxxxx.NET using keytab file /home/nbasjes/.krb/nbasjes.keytab
11/03/2015 17:13:25 Job execut= ion switched to status RUNNING.
11/03/2015 17:13:25 Custom Source -> Stream Sink(1/1) switche= d to SCHEDULED=C2=A0
11/03/2015 17:13:25 Custom Source -> Stream Sink(1/1) switched to DEPLOY= ING=C2=A0
11/03/2015 17:13:25 Custom Source -> Stream Sink(1/1) switched to RUNNING=C2=A0

Which looks good.

Howe= ver ... no data goes into HBase.
After some digging I found this = error in the task managers log:

17:13:42,677 WARN  org.apache.hado=
op.hbase.ipc.RpcClient                         - Exception encountered whil=
e connecting to the server : javax.security.sasl.SaslException: GSS initiat=
e 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 inv=
alid credentials. Consider 'kinit'.
javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSExcept=
ion: No valid credentials provided (Mechanism level: Failed to find any Ker=
beros tgt)]
	at com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Cl=
ient.java:212)
	at org.apache.hadoop.hbase.security.HBaseSaslRpcClient.saslConnect(HBaseSa=
slRpcClient.java:177)
	at org.apache.hadoop.hbase.ipc.RpcClient$Connection.setupSaslConnection(Rp=
cClient.java:815)
	at org.apache.hadoop.hbase.ipc.RpcClient$Connection.access$800(RpcClient.j=
ava:349)

First starting a yarn-session and t= hen loading my job gives the same error.

My best g= uess at this point is that Flink needs the same fix as described here:


What do you guys think?

=
Niels Basjes



On Tue, Oct 27, 2015 at 6:12 PM, Maximilian Michels &l= t;mxm@apache.org>= ; wrote:
Hi Niels,

You're welcome. Some more infor= mation on how this would be configured:

In the kdc.conf, there= are two variables:

=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 max_l= ife =3D 2h 0m 0s
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 max_renewabl= e_life =3D 7d 0h 0m 0s

max_life is the maximum life of the cur= rent ticket. However, it may be renewed up to a time span of max_renewable_= life from the first ticket issue on. This means that from the first ticket = issue, new tickets may be requested for one week. Each renewed ticket has = a life time of max_life (2 hours in this case).

Please le= t us know about any difficulties with long-running streaming application a= nd Kerberos.

Best regards,
Max

On Tue, O= ct 27, 2015 at 2:46 PM, Niels Basjes <Niels@basjes.nl> wrote:<= br>
Hi,

T= hanks for your feedback.
So I guess I'll have to talk to the = security guys about having special=C2=A0
kerberos ticket expiry t= imes for these types of jobs.

Niels Basjes

On = Fri, Oct 23, 2015 at 11:45 AM, Maximilian Michels <mxm@apache.org> wrote:
Hi Niels,

Thank you for your question. Flink relies entirely on the Kerberos
support of Hadoop. So your question could also be rephrased to "Does Hadoop support long-term authentication using Kerberos?". And the
answer is: Yes!

While Hadoop uses Kerberos tickets to authenticate users with services
initially, the authentication process continues differently
afterwards. Instead of saving the ticket to authenticate on a later
access, Hadoop creates its own security tockens (DelegationToken) that
it passes around. These are authenticated to Kerberos periodically. To
my knowledge, the tokens have a life span identical to the Kerberos
ticket maximum life span. So be sure to set the maximum life span very
high for long streaming jobs. The renewal time, on the other hand, is
not important because Hadoop abstracts this away using its own
security tockens.

I'm afraid there is not Kerberos how-to yet. If you are on Yarn, then it is sufficient to authenticate the client with Kerberos. On a Flink
standalone cluster you need to ensure that, initially, all nodes are
authenticated with Kerberos using the kinit tool.

Feel free to ask if you have more questions and let us know about any
difficulties.

Best regards,
Max



On Thu, Oct 22, 2015 at 2:06 PM, Niels Basjes <Niels@basjes.nl> wrote:
> Hi,
>
> I want to write a long running (i.e. never stop it) streaming flink > application on a kerberos secured Hadoop/Yarn cluster. My application = needs
> to do things with files on HDFS and HBase tables on that cluster so ha= ving
> the correct kerberos tickets is very important. The stream is to be in= gested
> from Kafka.
>
> One of the things with Kerberos is that the tickets expire after a
> predetermined time. My knowledge about kerberos is very limited so I h= ope
> you guys can help me.
>
> My question is actually quite simple: Is there an howto somewhere on h= ow to
> correctly run a long running flink application with kerberos that incl= udes a
> solution for the kerberos ticket timeout=C2=A0 ?
>
> Thanks
>
> Niels Basjes



<= /div>--
Best regards / Met vriendeli= jke groeten,

Niels Basjes




--
=
Best regards / Met vriendelijke groeten,

Niels Basjes




--
=
Best regards / Met vriendelijke groeten,

Niels Basjes

--001a114377925d19210523cb8198--