You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by Andrea Sella <an...@radicalbit.io> on 2016/03/11 17:33:24 UTC

Flink and YARN ship folder

Hi,

There is a way to add external dependencies to Flink Job,  running on YARN,
not using HADOOP_CLASSPATH?
I am looking for a similar idea to standalone mode using lib folder.

BR,
Andrea

Re: Flink and YARN ship folder

Posted by Ufuk Celebi <uc...@apache.org>.
Thank you very much! Yes, please go ahead an open an issue for this.
Thanks for reporting this!

On Fri, Mar 18, 2016 at 5:08 PM, Stefano Baghino
<st...@radicalbit.io> wrote:
> Hi Ufuk,
>
> I've just run a long running session and in that mode the libraries are
> correctly shipped without the need to specify -t lib.
>
> If you can double check the issue I can open an issue on JIRA.
>
> Thanks for helping us.
>
> On Thu, Mar 17, 2016 at 2:16 PM, Ufuk Celebi <uc...@apache.org> wrote:
>>
>> Can you try the same thing without -yt, but a yarn-session?
>>
>> – Ufuk
>>
>> On Thu, Mar 17, 2016 at 12:29 PM, Stefano Baghino
>> <st...@radicalbit.io> wrote:
>> > I have another interesting test result on this matter, running again an
>> > individual job on a YARN cluster.
>> >
>> > When running bin/flink run -m yarn-cluster -yn 1
>> > examples/batch/WordCount.jar the job fails with an error in the Job
>> > Manager
>> >
>> > LogType:jobmanager.err
>> > Log Upload Time:Thu Mar 17 07:05:32 -0400 2016
>> > LogLength:1221
>> > Log Contents:
>> > SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
>> > SLF4J: Defaulting to no-operation (NOP) logger implementation
>> > SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for
>> > further
>> > details.
>> > Exception in thread "main" java.lang.ExceptionInInitializerError
>> > at
>> >
>> > org.apache.flink.yarn.ApplicationMasterBase.run(ApplicationMasterBase.scala:64)
>> > at
>> > org.apache.flink.yarn.ApplicationMaster$.main(ApplicationMaster.scala:36)
>> > at org.apache.flink.yarn.ApplicationMaster.main(ApplicationMaster.scala)
>> > Caused by: org.apache.commons.logging.LogConfigurationException:
>> > User-specified log class 'org.apache.commons.logging.impl.Log4JLogger'
>> > cannot be found or is not useable.
>> > at
>> >
>> > org.apache.commons.logging.impl.LogFactoryImpl.discoverLogImplementation(LogFactoryImpl.java:804)
>> > at
>> >
>> > org.apache.commons.logging.impl.LogFactoryImpl.newInstance(LogFactoryImpl.java:541)
>> > at
>> >
>> > org.apache.commons.logging.impl.LogFactoryImpl.getInstance(LogFactoryImpl.java:292)
>> > at
>> >
>> > org.apache.commons.logging.impl.LogFactoryImpl.getInstance(LogFactoryImpl.java:269)
>> > at org.apache.commons.logging.LogFactory.getLog(LogFactory.java:657)
>> > at
>> >
>> > org.apache.hadoop.security.UserGroupInformation.<clinit>(UserGroupInformation.java:84)
>> > ... 3 more
>> > End of LogType:jobmanager.err
>> >
>> > But if I just add -yt lib (the Flink lib folder already contains SLF4J
>> > and
>> > Log4J) the libraries are shipped and the job goes on.
>> >
>> >
>> > On Wed, Mar 16, 2016 at 9:15 PM, Ufuk Celebi <uc...@apache.org> wrote:
>> >>
>> >> Sounds like a bug. I've only tried this with yarn-sessions. Could it
>> >> be that the behaviour is different between the two variants? In any
>> >> case, I sounds a lot like a bug in Flink. Robert can probably chime in
>> >> here with details.
>> >>
>> >> On Wed, Mar 16, 2016 at 6:25 PM, Andrea Sella
>> >> <an...@radicalbit.io> wrote:
>> >> > Hi,
>> >> >
>> >> > After few tests I am able to write and read on Alluxio.
>> >> > I am using Flink 1.0.0 and in my case external libraries are not
>> >> > loaded
>> >> > from
>> >> > lib folder to classpath, it loads only flink-dist_2.11-1.0.0.jar. I
>> >> > need
>> >> > to
>> >> > specify the folder with -yt parameter to load the others.
>> >> >
>> >> > If I run `/bin/flink run -m yarn-cluster -yn 4 -yjm 2048 -ytm 4096
>> >> > some.jar
>> >> > --input alluxio://somepath` it will throws an exception related to a
>> >> > missing
>> >> > library ie. Class alluxio.hadoop.FileSystem not found
>> >> > If I run `./bin/flink run -m yarn-cluster -yt lib/ -yn 4 -yjm 2048
>> >> > -ytm
>> >> > 4096
>> >> > some.jar --input alluxio://somepath` with -yt params it will work
>> >> > fine.
>> >> >
>> >> > is it a bug?
>> >> >
>> >> > Cheers,
>> >> > Andrea
>> >> >
>> >> >
>> >> >
>> >> >
>> >> > 2016-03-14 15:00 GMT+01:00 Andrea Sella <an...@radicalbit.io>:
>> >> >>
>> >> >> Hi Robert,
>> >> >>
>> >> >> Ok, thank you.
>> >> >>
>> >> >> 2016-03-14 11:13 GMT+01:00 Robert Metzger <rm...@apache.org>:
>> >> >>>
>> >> >>> Hi Andrea,
>> >> >>>
>> >> >>> You don't have to manually replicate any operations on the slaves.
>> >> >>> All
>> >> >>> files in the lib/ folder are transferred to all containers
>> >> >>> (Jobmanagers and
>> >> >>> TaskManagers).
>> >> >>>
>> >> >>>
>> >> >>> On Sat, Mar 12, 2016 at 3:25 PM, Andrea Sella
>> >> >>> <an...@radicalbit.io> wrote:
>> >> >>>>
>> >> >>>> Hi Ufuk,
>> >> >>>>
>> >> >>>> I'm trying to execute the WordCount batch example with input and
>> >> >>>> output
>> >> >>>> on Alluxio, i followed Running Flink on Alluxio and added the
>> >> >>>> library
>> >> >>>> to lib
>> >> >>>> folder. Have I to replicate this operation on the slaves or YARN
>> >> >>>> manage that
>> >> >>>> and I must have the library just where I launch the job?
>> >> >>>>
>> >> >>>> Thanks,
>> >> >>>> Andrea
>> >> >>>>
>> >> >>>> 2016-03-11 19:23 GMT+01:00 Ufuk Celebi <uc...@apache.org>:
>> >> >>>>>
>> >> >>>>> Everything in the lib folder should be added to the classpath.
>> >> >>>>> Can
>> >> >>>>> you
>> >> >>>>> check the YARN client logs that the files are uploaded?
>> >> >>>>> Furthermore,
>> >> >>>>> you can check the classpath of the JVM in the YARN logs of the
>> >> >>>>> JobManager/TaskManager processes.
>> >> >>>>>
>> >> >>>>> – Ufuk
>> >> >>>>>
>> >> >>>>>
>> >> >>>>> On Fri, Mar 11, 2016 at 5:33 PM, Andrea Sella
>> >> >>>>> <an...@radicalbit.io> wrote:
>> >> >>>>> > Hi,
>> >> >>>>> >
>> >> >>>>> > There is a way to add external dependencies to Flink Job,
>> >> >>>>> > running
>> >> >>>>> > on
>> >> >>>>> > YARN,
>> >> >>>>> > not using HADOOP_CLASSPATH?
>> >> >>>>> > I am looking for a similar idea to standalone mode using lib
>> >> >>>>> > folder.
>> >> >>>>> >
>> >> >>>>> > BR,
>> >> >>>>> > Andrea
>> >> >>>>
>> >> >>>>
>> >> >>>
>> >> >>
>> >> >
>> >
>> >
>> >
>> >
>> > --
>> > BR,
>> > Stefano Baghino
>> >
>> > Software Engineer @ Radicalbit
>
>
>
>
> --
> BR,
> Stefano Baghino
>
> Software Engineer @ Radicalbit

Re: Flink and YARN ship folder

Posted by Stefano Baghino <st...@radicalbit.io>.
Hi Ufuk,

I've just run a long running session and in that mode the libraries are
correctly shipped without the need to specify -t lib.

If you can double check the issue I can open an issue on JIRA.

Thanks for helping us.

On Thu, Mar 17, 2016 at 2:16 PM, Ufuk Celebi <uc...@apache.org> wrote:

> Can you try the same thing without -yt, but a yarn-session?
>
> – Ufuk
>
> On Thu, Mar 17, 2016 at 12:29 PM, Stefano Baghino
> <st...@radicalbit.io> wrote:
> > I have another interesting test result on this matter, running again an
> > individual job on a YARN cluster.
> >
> > When running bin/flink run -m yarn-cluster -yn 1
> > examples/batch/WordCount.jar the job fails with an error in the Job
> Manager
> >
> > LogType:jobmanager.err
> > Log Upload Time:Thu Mar 17 07:05:32 -0400 2016
> > LogLength:1221
> > Log Contents:
> > SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> > SLF4J: Defaulting to no-operation (NOP) logger implementation
> > SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for
> further
> > details.
> > Exception in thread "main" java.lang.ExceptionInInitializerError
> > at
> >
> org.apache.flink.yarn.ApplicationMasterBase.run(ApplicationMasterBase.scala:64)
> > at
> org.apache.flink.yarn.ApplicationMaster$.main(ApplicationMaster.scala:36)
> > at org.apache.flink.yarn.ApplicationMaster.main(ApplicationMaster.scala)
> > Caused by: org.apache.commons.logging.LogConfigurationException:
> > User-specified log class 'org.apache.commons.logging.impl.Log4JLogger'
> > cannot be found or is not useable.
> > at
> >
> org.apache.commons.logging.impl.LogFactoryImpl.discoverLogImplementation(LogFactoryImpl.java:804)
> > at
> >
> org.apache.commons.logging.impl.LogFactoryImpl.newInstance(LogFactoryImpl.java:541)
> > at
> >
> org.apache.commons.logging.impl.LogFactoryImpl.getInstance(LogFactoryImpl.java:292)
> > at
> >
> org.apache.commons.logging.impl.LogFactoryImpl.getInstance(LogFactoryImpl.java:269)
> > at org.apache.commons.logging.LogFactory.getLog(LogFactory.java:657)
> > at
> >
> org.apache.hadoop.security.UserGroupInformation.<clinit>(UserGroupInformation.java:84)
> > ... 3 more
> > End of LogType:jobmanager.err
> >
> > But if I just add -yt lib (the Flink lib folder already contains SLF4J
> and
> > Log4J) the libraries are shipped and the job goes on.
> >
> >
> > On Wed, Mar 16, 2016 at 9:15 PM, Ufuk Celebi <uc...@apache.org> wrote:
> >>
> >> Sounds like a bug. I've only tried this with yarn-sessions. Could it
> >> be that the behaviour is different between the two variants? In any
> >> case, I sounds a lot like a bug in Flink. Robert can probably chime in
> >> here with details.
> >>
> >> On Wed, Mar 16, 2016 at 6:25 PM, Andrea Sella
> >> <an...@radicalbit.io> wrote:
> >> > Hi,
> >> >
> >> > After few tests I am able to write and read on Alluxio.
> >> > I am using Flink 1.0.0 and in my case external libraries are not
> loaded
> >> > from
> >> > lib folder to classpath, it loads only flink-dist_2.11-1.0.0.jar. I
> need
> >> > to
> >> > specify the folder with -yt parameter to load the others.
> >> >
> >> > If I run `/bin/flink run -m yarn-cluster -yn 4 -yjm 2048 -ytm 4096
> >> > some.jar
> >> > --input alluxio://somepath` it will throws an exception related to a
> >> > missing
> >> > library ie. Class alluxio.hadoop.FileSystem not found
> >> > If I run `./bin/flink run -m yarn-cluster -yt lib/ -yn 4 -yjm 2048
> -ytm
> >> > 4096
> >> > some.jar --input alluxio://somepath` with -yt params it will work
> fine.
> >> >
> >> > is it a bug?
> >> >
> >> > Cheers,
> >> > Andrea
> >> >
> >> >
> >> >
> >> >
> >> > 2016-03-14 15:00 GMT+01:00 Andrea Sella <an...@radicalbit.io>:
> >> >>
> >> >> Hi Robert,
> >> >>
> >> >> Ok, thank you.
> >> >>
> >> >> 2016-03-14 11:13 GMT+01:00 Robert Metzger <rm...@apache.org>:
> >> >>>
> >> >>> Hi Andrea,
> >> >>>
> >> >>> You don't have to manually replicate any operations on the slaves.
> All
> >> >>> files in the lib/ folder are transferred to all containers
> >> >>> (Jobmanagers and
> >> >>> TaskManagers).
> >> >>>
> >> >>>
> >> >>> On Sat, Mar 12, 2016 at 3:25 PM, Andrea Sella
> >> >>> <an...@radicalbit.io> wrote:
> >> >>>>
> >> >>>> Hi Ufuk,
> >> >>>>
> >> >>>> I'm trying to execute the WordCount batch example with input and
> >> >>>> output
> >> >>>> on Alluxio, i followed Running Flink on Alluxio and added the
> library
> >> >>>> to lib
> >> >>>> folder. Have I to replicate this operation on the slaves or YARN
> >> >>>> manage that
> >> >>>> and I must have the library just where I launch the job?
> >> >>>>
> >> >>>> Thanks,
> >> >>>> Andrea
> >> >>>>
> >> >>>> 2016-03-11 19:23 GMT+01:00 Ufuk Celebi <uc...@apache.org>:
> >> >>>>>
> >> >>>>> Everything in the lib folder should be added to the classpath. Can
> >> >>>>> you
> >> >>>>> check the YARN client logs that the files are uploaded?
> Furthermore,
> >> >>>>> you can check the classpath of the JVM in the YARN logs of the
> >> >>>>> JobManager/TaskManager processes.
> >> >>>>>
> >> >>>>> – Ufuk
> >> >>>>>
> >> >>>>>
> >> >>>>> On Fri, Mar 11, 2016 at 5:33 PM, Andrea Sella
> >> >>>>> <an...@radicalbit.io> wrote:
> >> >>>>> > Hi,
> >> >>>>> >
> >> >>>>> > There is a way to add external dependencies to Flink Job,
> running
> >> >>>>> > on
> >> >>>>> > YARN,
> >> >>>>> > not using HADOOP_CLASSPATH?
> >> >>>>> > I am looking for a similar idea to standalone mode using lib
> >> >>>>> > folder.
> >> >>>>> >
> >> >>>>> > BR,
> >> >>>>> > Andrea
> >> >>>>
> >> >>>>
> >> >>>
> >> >>
> >> >
> >
> >
> >
> >
> > --
> > BR,
> > Stefano Baghino
> >
> > Software Engineer @ Radicalbit
>



-- 
BR,
Stefano Baghino

Software Engineer @ Radicalbit

Re: Flink and YARN ship folder

Posted by Ufuk Celebi <uc...@apache.org>.
Can you try the same thing without -yt, but a yarn-session?

– Ufuk

On Thu, Mar 17, 2016 at 12:29 PM, Stefano Baghino
<st...@radicalbit.io> wrote:
> I have another interesting test result on this matter, running again an
> individual job on a YARN cluster.
>
> When running bin/flink run -m yarn-cluster -yn 1
> examples/batch/WordCount.jar the job fails with an error in the Job Manager
>
> LogType:jobmanager.err
> Log Upload Time:Thu Mar 17 07:05:32 -0400 2016
> LogLength:1221
> Log Contents:
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further
> details.
> Exception in thread "main" java.lang.ExceptionInInitializerError
> at
> org.apache.flink.yarn.ApplicationMasterBase.run(ApplicationMasterBase.scala:64)
> at org.apache.flink.yarn.ApplicationMaster$.main(ApplicationMaster.scala:36)
> at org.apache.flink.yarn.ApplicationMaster.main(ApplicationMaster.scala)
> Caused by: org.apache.commons.logging.LogConfigurationException:
> User-specified log class 'org.apache.commons.logging.impl.Log4JLogger'
> cannot be found or is not useable.
> at
> org.apache.commons.logging.impl.LogFactoryImpl.discoverLogImplementation(LogFactoryImpl.java:804)
> at
> org.apache.commons.logging.impl.LogFactoryImpl.newInstance(LogFactoryImpl.java:541)
> at
> org.apache.commons.logging.impl.LogFactoryImpl.getInstance(LogFactoryImpl.java:292)
> at
> org.apache.commons.logging.impl.LogFactoryImpl.getInstance(LogFactoryImpl.java:269)
> at org.apache.commons.logging.LogFactory.getLog(LogFactory.java:657)
> at
> org.apache.hadoop.security.UserGroupInformation.<clinit>(UserGroupInformation.java:84)
> ... 3 more
> End of LogType:jobmanager.err
>
> But if I just add -yt lib (the Flink lib folder already contains SLF4J and
> Log4J) the libraries are shipped and the job goes on.
>
>
> On Wed, Mar 16, 2016 at 9:15 PM, Ufuk Celebi <uc...@apache.org> wrote:
>>
>> Sounds like a bug. I've only tried this with yarn-sessions. Could it
>> be that the behaviour is different between the two variants? In any
>> case, I sounds a lot like a bug in Flink. Robert can probably chime in
>> here with details.
>>
>> On Wed, Mar 16, 2016 at 6:25 PM, Andrea Sella
>> <an...@radicalbit.io> wrote:
>> > Hi,
>> >
>> > After few tests I am able to write and read on Alluxio.
>> > I am using Flink 1.0.0 and in my case external libraries are not loaded
>> > from
>> > lib folder to classpath, it loads only flink-dist_2.11-1.0.0.jar. I need
>> > to
>> > specify the folder with -yt parameter to load the others.
>> >
>> > If I run `/bin/flink run -m yarn-cluster -yn 4 -yjm 2048 -ytm 4096
>> > some.jar
>> > --input alluxio://somepath` it will throws an exception related to a
>> > missing
>> > library ie. Class alluxio.hadoop.FileSystem not found
>> > If I run `./bin/flink run -m yarn-cluster -yt lib/ -yn 4 -yjm 2048 -ytm
>> > 4096
>> > some.jar --input alluxio://somepath` with -yt params it will work fine.
>> >
>> > is it a bug?
>> >
>> > Cheers,
>> > Andrea
>> >
>> >
>> >
>> >
>> > 2016-03-14 15:00 GMT+01:00 Andrea Sella <an...@radicalbit.io>:
>> >>
>> >> Hi Robert,
>> >>
>> >> Ok, thank you.
>> >>
>> >> 2016-03-14 11:13 GMT+01:00 Robert Metzger <rm...@apache.org>:
>> >>>
>> >>> Hi Andrea,
>> >>>
>> >>> You don't have to manually replicate any operations on the slaves. All
>> >>> files in the lib/ folder are transferred to all containers
>> >>> (Jobmanagers and
>> >>> TaskManagers).
>> >>>
>> >>>
>> >>> On Sat, Mar 12, 2016 at 3:25 PM, Andrea Sella
>> >>> <an...@radicalbit.io> wrote:
>> >>>>
>> >>>> Hi Ufuk,
>> >>>>
>> >>>> I'm trying to execute the WordCount batch example with input and
>> >>>> output
>> >>>> on Alluxio, i followed Running Flink on Alluxio and added the library
>> >>>> to lib
>> >>>> folder. Have I to replicate this operation on the slaves or YARN
>> >>>> manage that
>> >>>> and I must have the library just where I launch the job?
>> >>>>
>> >>>> Thanks,
>> >>>> Andrea
>> >>>>
>> >>>> 2016-03-11 19:23 GMT+01:00 Ufuk Celebi <uc...@apache.org>:
>> >>>>>
>> >>>>> Everything in the lib folder should be added to the classpath. Can
>> >>>>> you
>> >>>>> check the YARN client logs that the files are uploaded? Furthermore,
>> >>>>> you can check the classpath of the JVM in the YARN logs of the
>> >>>>> JobManager/TaskManager processes.
>> >>>>>
>> >>>>> – Ufuk
>> >>>>>
>> >>>>>
>> >>>>> On Fri, Mar 11, 2016 at 5:33 PM, Andrea Sella
>> >>>>> <an...@radicalbit.io> wrote:
>> >>>>> > Hi,
>> >>>>> >
>> >>>>> > There is a way to add external dependencies to Flink Job,  running
>> >>>>> > on
>> >>>>> > YARN,
>> >>>>> > not using HADOOP_CLASSPATH?
>> >>>>> > I am looking for a similar idea to standalone mode using lib
>> >>>>> > folder.
>> >>>>> >
>> >>>>> > BR,
>> >>>>> > Andrea
>> >>>>
>> >>>>
>> >>>
>> >>
>> >
>
>
>
>
> --
> BR,
> Stefano Baghino
>
> Software Engineer @ Radicalbit

Re: Flink and YARN ship folder

Posted by Stefano Baghino <st...@radicalbit.io>.
I have another interesting test result on this matter, running again an
individual job on a YARN cluster.

When running bin/flink run -m yarn-cluster -yn 1
examples/batch/WordCount.jar the job fails with an error in the Job Manager

LogType:jobmanager.err
Log Upload Time:Thu Mar 17 07:05:32 -0400 2016
LogLength:1221
Log Contents:
SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
SLF4J: Defaulting to no-operation (NOP) logger implementation
SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further
details.
Exception in thread "main" java.lang.ExceptionInInitializerError
at
org.apache.flink.yarn.ApplicationMasterBase.run(ApplicationMasterBase.scala:64)
at org.apache.flink.yarn.ApplicationMaster$.main(ApplicationMaster.scala:36)
at org.apache.flink.yarn.ApplicationMaster.main(ApplicationMaster.scala)
Caused by: org.apache.commons.logging.LogConfigurationException:
User-specified log class 'org.apache.commons.logging.impl.Log4JLogger'
cannot be found or is not useable.
at
org.apache.commons.logging.impl.LogFactoryImpl.discoverLogImplementation(LogFactoryImpl.java:804)
at
org.apache.commons.logging.impl.LogFactoryImpl.newInstance(LogFactoryImpl.java:541)
at
org.apache.commons.logging.impl.LogFactoryImpl.getInstance(LogFactoryImpl.java:292)
at
org.apache.commons.logging.impl.LogFactoryImpl.getInstance(LogFactoryImpl.java:269)
at org.apache.commons.logging.LogFactory.getLog(LogFactory.java:657)
at
org.apache.hadoop.security.UserGroupInformation.<clinit>(UserGroupInformation.java:84)
... 3 more
End of LogType:jobmanager.err

But if I just add -yt lib (the Flink lib folder already contains SLF4J and
Log4J) the libraries are shipped and the job goes on.

On Wed, Mar 16, 2016 at 9:15 PM, Ufuk Celebi <uc...@apache.org> wrote:

> Sounds like a bug. I've only tried this with yarn-sessions. Could it
> be that the behaviour is different between the two variants? In any
> case, I sounds a lot like a bug in Flink. Robert can probably chime in
> here with details.
>
> On Wed, Mar 16, 2016 at 6:25 PM, Andrea Sella
> <an...@radicalbit.io> wrote:
> > Hi,
> >
> > After few tests I am able to write and read on Alluxio.
> > I am using Flink 1.0.0 and in my case external libraries are not loaded
> from
> > lib folder to classpath, it loads only flink-dist_2.11-1.0.0.jar. I need
> to
> > specify the folder with -yt parameter to load the others.
> >
> > If I run `/bin/flink run -m yarn-cluster -yn 4 -yjm 2048 -ytm 4096
> some.jar
> > --input alluxio://somepath` it will throws an exception related to a
> missing
> > library ie. Class alluxio.hadoop.FileSystem not found
> > If I run `./bin/flink run -m yarn-cluster -yt lib/ -yn 4 -yjm 2048 -ytm
> 4096
> > some.jar --input alluxio://somepath` with -yt params it will work fine.
> >
> > is it a bug?
> >
> > Cheers,
> > Andrea
> >
> >
> >
> >
> > 2016-03-14 15:00 GMT+01:00 Andrea Sella <an...@radicalbit.io>:
> >>
> >> Hi Robert,
> >>
> >> Ok, thank you.
> >>
> >> 2016-03-14 11:13 GMT+01:00 Robert Metzger <rm...@apache.org>:
> >>>
> >>> Hi Andrea,
> >>>
> >>> You don't have to manually replicate any operations on the slaves. All
> >>> files in the lib/ folder are transferred to all containers
> (Jobmanagers and
> >>> TaskManagers).
> >>>
> >>>
> >>> On Sat, Mar 12, 2016 at 3:25 PM, Andrea Sella
> >>> <an...@radicalbit.io> wrote:
> >>>>
> >>>> Hi Ufuk,
> >>>>
> >>>> I'm trying to execute the WordCount batch example with input and
> output
> >>>> on Alluxio, i followed Running Flink on Alluxio and added the library
> to lib
> >>>> folder. Have I to replicate this operation on the slaves or YARN
> manage that
> >>>> and I must have the library just where I launch the job?
> >>>>
> >>>> Thanks,
> >>>> Andrea
> >>>>
> >>>> 2016-03-11 19:23 GMT+01:00 Ufuk Celebi <uc...@apache.org>:
> >>>>>
> >>>>> Everything in the lib folder should be added to the classpath. Can
> you
> >>>>> check the YARN client logs that the files are uploaded? Furthermore,
> >>>>> you can check the classpath of the JVM in the YARN logs of the
> >>>>> JobManager/TaskManager processes.
> >>>>>
> >>>>> – Ufuk
> >>>>>
> >>>>>
> >>>>> On Fri, Mar 11, 2016 at 5:33 PM, Andrea Sella
> >>>>> <an...@radicalbit.io> wrote:
> >>>>> > Hi,
> >>>>> >
> >>>>> > There is a way to add external dependencies to Flink Job,  running
> on
> >>>>> > YARN,
> >>>>> > not using HADOOP_CLASSPATH?
> >>>>> > I am looking for a similar idea to standalone mode using lib
> folder.
> >>>>> >
> >>>>> > BR,
> >>>>> > Andrea
> >>>>
> >>>>
> >>>
> >>
> >
>



-- 
BR,
Stefano Baghino

Software Engineer @ Radicalbit

Re: Flink and YARN ship folder

Posted by Ufuk Celebi <uc...@apache.org>.
Sounds like a bug. I've only tried this with yarn-sessions. Could it
be that the behaviour is different between the two variants? In any
case, I sounds a lot like a bug in Flink. Robert can probably chime in
here with details.

On Wed, Mar 16, 2016 at 6:25 PM, Andrea Sella
<an...@radicalbit.io> wrote:
> Hi,
>
> After few tests I am able to write and read on Alluxio.
> I am using Flink 1.0.0 and in my case external libraries are not loaded from
> lib folder to classpath, it loads only flink-dist_2.11-1.0.0.jar. I need to
> specify the folder with -yt parameter to load the others.
>
> If I run `/bin/flink run -m yarn-cluster -yn 4 -yjm 2048 -ytm 4096 some.jar
> --input alluxio://somepath` it will throws an exception related to a missing
> library ie. Class alluxio.hadoop.FileSystem not found
> If I run `./bin/flink run -m yarn-cluster -yt lib/ -yn 4 -yjm 2048 -ytm 4096
> some.jar --input alluxio://somepath` with -yt params it will work fine.
>
> is it a bug?
>
> Cheers,
> Andrea
>
>
>
>
> 2016-03-14 15:00 GMT+01:00 Andrea Sella <an...@radicalbit.io>:
>>
>> Hi Robert,
>>
>> Ok, thank you.
>>
>> 2016-03-14 11:13 GMT+01:00 Robert Metzger <rm...@apache.org>:
>>>
>>> Hi Andrea,
>>>
>>> You don't have to manually replicate any operations on the slaves. All
>>> files in the lib/ folder are transferred to all containers (Jobmanagers and
>>> TaskManagers).
>>>
>>>
>>> On Sat, Mar 12, 2016 at 3:25 PM, Andrea Sella
>>> <an...@radicalbit.io> wrote:
>>>>
>>>> Hi Ufuk,
>>>>
>>>> I'm trying to execute the WordCount batch example with input and output
>>>> on Alluxio, i followed Running Flink on Alluxio and added the library to lib
>>>> folder. Have I to replicate this operation on the slaves or YARN manage that
>>>> and I must have the library just where I launch the job?
>>>>
>>>> Thanks,
>>>> Andrea
>>>>
>>>> 2016-03-11 19:23 GMT+01:00 Ufuk Celebi <uc...@apache.org>:
>>>>>
>>>>> Everything in the lib folder should be added to the classpath. Can you
>>>>> check the YARN client logs that the files are uploaded? Furthermore,
>>>>> you can check the classpath of the JVM in the YARN logs of the
>>>>> JobManager/TaskManager processes.
>>>>>
>>>>> – Ufuk
>>>>>
>>>>>
>>>>> On Fri, Mar 11, 2016 at 5:33 PM, Andrea Sella
>>>>> <an...@radicalbit.io> wrote:
>>>>> > Hi,
>>>>> >
>>>>> > There is a way to add external dependencies to Flink Job,  running on
>>>>> > YARN,
>>>>> > not using HADOOP_CLASSPATH?
>>>>> > I am looking for a similar idea to standalone mode using lib folder.
>>>>> >
>>>>> > BR,
>>>>> > Andrea
>>>>
>>>>
>>>
>>
>

Re: Flink and YARN ship folder

Posted by Andrea Sella <an...@radicalbit.io>.
Hi,

After few tests I am able to write and read on Alluxio.
I am using Flink 1.0.0 and in my case external libraries are not loaded
from lib folder to classpath, it loads only flink-dist_2.11-1.0.0.jar. I
need to specify the folder with -yt parameter to load the others.

If I run `/bin/flink run -m yarn-cluster -yn 4 -yjm 2048 -ytm 4096 some.jar
--input alluxio://somepath` it will throws an exception related to a
missing library ie. Class alluxio.hadoop.FileSystem not found
If I run `./bin/flink run -m yarn-cluster -yt lib/ -yn 4 -yjm 2048 -ytm
4096 some.jar --input alluxio://somepath` with -yt params it will work fine.

is it a bug?

Cheers,
Andrea




2016-03-14 15:00 GMT+01:00 Andrea Sella <an...@radicalbit.io>:

> Hi Robert,
>
> Ok, thank you.
>
> 2016-03-14 11:13 GMT+01:00 Robert Metzger <rm...@apache.org>:
>
>> Hi Andrea,
>>
>> You don't have to manually replicate any operations on the slaves. All
>> files in the lib/ folder are transferred to all containers (Jobmanagers and
>> TaskManagers).
>>
>>
>> On Sat, Mar 12, 2016 at 3:25 PM, Andrea Sella <andrea.sella@radicalbit.io
>> > wrote:
>>
>>> Hi Ufuk,
>>>
>>> I'm trying to execute the WordCount batch example with input and output
>>> on Alluxio, i followed Running Flink on Alluxio
>>> <http://www.alluxio.org/documentation/en/Running-Flink-on-Alluxio.html> and
>>> added the library to lib folder. Have I to replicate this operation on the
>>> slaves or YARN manage that and I must have the library just where I launch
>>> the job?
>>>
>>> Thanks,
>>> Andrea
>>>
>>> 2016-03-11 19:23 GMT+01:00 Ufuk Celebi <uc...@apache.org>:
>>>
>>>> Everything in the lib folder should be added to the classpath. Can you
>>>> check the YARN client logs that the files are uploaded? Furthermore,
>>>> you can check the classpath of the JVM in the YARN logs of the
>>>> JobManager/TaskManager processes.
>>>>
>>>> – Ufuk
>>>>
>>>>
>>>> On Fri, Mar 11, 2016 at 5:33 PM, Andrea Sella
>>>> <an...@radicalbit.io> wrote:
>>>> > Hi,
>>>> >
>>>> > There is a way to add external dependencies to Flink Job,  running on
>>>> YARN,
>>>> > not using HADOOP_CLASSPATH?
>>>> > I am looking for a similar idea to standalone mode using lib folder.
>>>> >
>>>> > BR,
>>>> > Andrea
>>>>
>>>
>>>
>>
>

Re: Flink and YARN ship folder

Posted by Andrea Sella <an...@radicalbit.io>.
Hi Robert,

Ok, thank you.

2016-03-14 11:13 GMT+01:00 Robert Metzger <rm...@apache.org>:

> Hi Andrea,
>
> You don't have to manually replicate any operations on the slaves. All
> files in the lib/ folder are transferred to all containers (Jobmanagers and
> TaskManagers).
>
>
> On Sat, Mar 12, 2016 at 3:25 PM, Andrea Sella <an...@radicalbit.io>
> wrote:
>
>> Hi Ufuk,
>>
>> I'm trying to execute the WordCount batch example with input and output
>> on Alluxio, i followed Running Flink on Alluxio
>> <http://www.alluxio.org/documentation/en/Running-Flink-on-Alluxio.html> and
>> added the library to lib folder. Have I to replicate this operation on the
>> slaves or YARN manage that and I must have the library just where I launch
>> the job?
>>
>> Thanks,
>> Andrea
>>
>> 2016-03-11 19:23 GMT+01:00 Ufuk Celebi <uc...@apache.org>:
>>
>>> Everything in the lib folder should be added to the classpath. Can you
>>> check the YARN client logs that the files are uploaded? Furthermore,
>>> you can check the classpath of the JVM in the YARN logs of the
>>> JobManager/TaskManager processes.
>>>
>>> – Ufuk
>>>
>>>
>>> On Fri, Mar 11, 2016 at 5:33 PM, Andrea Sella
>>> <an...@radicalbit.io> wrote:
>>> > Hi,
>>> >
>>> > There is a way to add external dependencies to Flink Job,  running on
>>> YARN,
>>> > not using HADOOP_CLASSPATH?
>>> > I am looking for a similar idea to standalone mode using lib folder.
>>> >
>>> > BR,
>>> > Andrea
>>>
>>
>>
>

Re: Flink and YARN ship folder

Posted by Robert Metzger <rm...@apache.org>.
Hi Andrea,

You don't have to manually replicate any operations on the slaves. All
files in the lib/ folder are transferred to all containers (Jobmanagers and
TaskManagers).


On Sat, Mar 12, 2016 at 3:25 PM, Andrea Sella <an...@radicalbit.io>
wrote:

> Hi Ufuk,
>
> I'm trying to execute the WordCount batch example with input and output on
> Alluxio, i followed Running Flink on Alluxio
> <http://www.alluxio.org/documentation/en/Running-Flink-on-Alluxio.html> and
> added the library to lib folder. Have I to replicate this operation on the
> slaves or YARN manage that and I must have the library just where I launch
> the job?
>
> Thanks,
> Andrea
>
> 2016-03-11 19:23 GMT+01:00 Ufuk Celebi <uc...@apache.org>:
>
>> Everything in the lib folder should be added to the classpath. Can you
>> check the YARN client logs that the files are uploaded? Furthermore,
>> you can check the classpath of the JVM in the YARN logs of the
>> JobManager/TaskManager processes.
>>
>> – Ufuk
>>
>>
>> On Fri, Mar 11, 2016 at 5:33 PM, Andrea Sella
>> <an...@radicalbit.io> wrote:
>> > Hi,
>> >
>> > There is a way to add external dependencies to Flink Job,  running on
>> YARN,
>> > not using HADOOP_CLASSPATH?
>> > I am looking for a similar idea to standalone mode using lib folder.
>> >
>> > BR,
>> > Andrea
>>
>
>

Re: Flink and YARN ship folder

Posted by Andrea Sella <an...@radicalbit.io>.
Hi Ufuk,

I'm trying to execute the WordCount batch example with input and output on
Alluxio, i followed Running Flink on Alluxio
<http://www.alluxio.org/documentation/en/Running-Flink-on-Alluxio.html> and
added the library to lib folder. Have I to replicate this operation on the
slaves or YARN manage that and I must have the library just where I launch
the job?

Thanks,
Andrea

2016-03-11 19:23 GMT+01:00 Ufuk Celebi <uc...@apache.org>:

> Everything in the lib folder should be added to the classpath. Can you
> check the YARN client logs that the files are uploaded? Furthermore,
> you can check the classpath of the JVM in the YARN logs of the
> JobManager/TaskManager processes.
>
> – Ufuk
>
>
> On Fri, Mar 11, 2016 at 5:33 PM, Andrea Sella
> <an...@radicalbit.io> wrote:
> > Hi,
> >
> > There is a way to add external dependencies to Flink Job,  running on
> YARN,
> > not using HADOOP_CLASSPATH?
> > I am looking for a similar idea to standalone mode using lib folder.
> >
> > BR,
> > Andrea
>

Re: Flink and YARN ship folder

Posted by Ufuk Celebi <uc...@apache.org>.
Everything in the lib folder should be added to the classpath. Can you
check the YARN client logs that the files are uploaded? Furthermore,
you can check the classpath of the JVM in the YARN logs of the
JobManager/TaskManager processes.

– Ufuk


On Fri, Mar 11, 2016 at 5:33 PM, Andrea Sella
<an...@radicalbit.io> wrote:
> Hi,
>
> There is a way to add external dependencies to Flink Job,  running on YARN,
> not using HADOOP_CLASSPATH?
> I am looking for a similar idea to standalone mode using lib folder.
>
> BR,
> Andrea