How long should this take if you have HDD and about 8GB of RAM? Is that 10 minutes? 20?
Niels On Sat, Nov 7, 2015 at 2:51 PM, Stephan Ewen <se...@apache.org> wrote: > Hi Niels! > > Usually, you simply build the binaries by invoking "mvn -DskipTests clean > package" in the root flink directory. The resulting program should be in > the "build-target" directory. > > If the program gets stuck, let us know where and what the last message on > the command line is. > > Please be aware that the final step of building the "flink-dist" project > may take a while, especially on systems with hard disks (as opposed to > SSDs) and a comparatively low amount of memory. The reason is that the > building of the final JAR file is quite expensive, because the system > re-packages certain libraries in order to avoid conflicts between different > versions. > > Stephan > > > On Sat, Nov 7, 2015 at 2:40 PM, Niels Basjes <ni...@basj.es> wrote: > >> Hi, >> >> Excellent. >> What you can help me with are the commands to build the binary >> distribution from source. >> I tried it last Thursday and the build seemed to get stuck at some point >> (at the end of/just after building the dist module). >> I haven't been able to figure out why yet. >> >> Niels >> On 5 Nov 2015 14:57, "Maximilian Michels" <m...@apache.org> wrote: >> >>> 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 <ni...@basjes.nl> wrote: >>> >>>> I created https://issues.apache.org/jira/browse/FLINK-2977 >>>> >>>> On Thu, Nov 5, 2015 at 12:25 PM, Robert Metzger <rmetz...@apache.org> >>>> wrote: >>>> >>>>> Hi Niels, >>>>> thank you for analyzing the issue so properly. I agree with you. It >>>>> seems 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 fix 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 >>>>> implementing it, you test the fix) >>>>> >>>>> >>>>> Robert >>>>> >>>>> On Tue, Nov 3, 2015 at 5:26 PM, Niels Basjes <ni...@basjes.nl> 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 >>>>>> (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: >>>>>> >>>>>> public static void main(String[] args) throws Exception { >>>>>> System.setProperty("java.security.krb5.conf", "/etc/krb5.conf"); >>>>>> UserGroupInformation.loginUserFromKeytab("nbas...@xxxxxx.net", >>>>>> "/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"); >>>>>> } >>>>>> >>>>>> 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 nbas...@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.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) >>>>>> >>>>>> >>>>>> 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 <m...@apache.org> >>>>>> 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 = 2h 0m 0s >>>>>>> max_renewable_life = 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_life >>>>>>> (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 <ni...@basjes.nl> >>>>>>> wrote: >>>>>>> >>>>>>>> Hi, >>>>>>>> >>>>>>>> Thanks for your feedback. >>>>>>>> So I guess I'll have to talk to the security guys about having >>>>>>>> special >>>>>>>> kerberos ticket expiry times for these types of jobs. >>>>>>>> >>>>>>>> Niels Basjes >>>>>>>> >>>>>>>> On Fri, Oct 23, 2015 at 11:45 AM, Maximilian Michels < >>>>>>>> m...@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 <ni...@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 having >>>>>>>>> > the correct kerberos tickets is very important. The stream is to >>>>>>>>> be 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 >>>> >>> >>> > -- Best regards / Met vriendelijke groeten, Niels Basjes