Hi all,
for the upcoming 1.11 release, I started looking into adding support for Hadoop 3[1] for Flink. I have explored a little bit already into adding a shaded hadoop 3 into “flink-shaded”, and some mechanisms for switching between Hadoop 2 and 3 dependencies in the Flink build. However, Chesnay made me aware that we could also go a different route: We let Flink depend on vanilla Hadoop dependencies and stop providing shaded fat jars for Hadoop through “flink-shaded”. Why? - Maintaining properly shaded Hadoop fat jars is a lot of work (we have insufficient test coverage for all kinds of Hadoop features) - For Hadoop 2, there are already some known and unresolved issues with our shaded jars that we didn’t manage to fix Users will have to use Flink with Hadoop by relying on vanilla or vendor-provided Hadoop dependencies. What do you think? Best, Robert [1] https://issues.apache.org/jira/browse/FLINK-11086 |
Hello,
I think that's a good idea. I suppose that most corporate users use a vendor version and already compile their flink version with vendor repos anyway. Arnaud PS - FYI, for CDH6 (based on hadoop3), I've managed to build a 1.10 version by modifying pom.xml files and using "hidden" Cloudera package : <dependency> <groupId>org.apache.flink</groupId> <artifactId>flink-shaded-hadoop-3</artifactId> <version>3.1.1.7.0.3.0-79-7.0</version> <optional>true</optional> </dependency> (same with flink-shaded-hadoop-3-uber) On 2020/04/21 14:37:33, Robert Metzger <[hidden email]> wrote: > Hi all,> > > for the upcoming 1.11 release, I started looking into adding support for> > Hadoop 3[1] for Flink. I have explored a little bit already into adding a> > shaded hadoop 3 into "flink-shaded", and some mechanisms for switching> > between Hadoop 2 and 3 dependencies in the Flink build.> > > However, Chesnay made me aware that we could also go a different route: We> > let Flink depend on vanilla Hadoop dependencies and stop providing shaded> > fat jars for Hadoop through "flink-shaded".> > > Why?> > - Maintaining properly shaded Hadoop fat jars is a lot of work (we have> > insufficient test coverage for all kinds of Hadoop features)> > - For Hadoop 2, there are already some known and unresolved issues with our> > shaded jars that we didn't manage to fix> > > Users will have to use Flink with Hadoop by relying on vanilla or> > vendor-provided Hadoop dependencies.> > > What do you think?> > > Best,> > Robert> > > [1] https://issues.apache.org/jira/browse/FLINK-11086> > ________________________________ L'int?grit? de ce message n'?tant pas assur?e sur internet, la soci?t? exp?ditrice ne peut ?tre tenue responsable de son contenu ni de ses pi?ces jointes. Toute utilisation ou diffusion non autoris?e est interdite. Si vous n'?tes pas destinataire de ce message, merci de le d?truire et d'avertir l'exp?diteur. The integrity of this message cannot be guaranteed on the Internet. The company that sent this message cannot therefore be held liable for its content nor attachments. Any unauthorized use or dissemination is prohibited. If you are not the intended recipient of this message, then please delete it and notify the sender. |
In reply to this post by Robert Metzger
Hi Robert,
I think it would be a helpful simplification of Flink's build setup if we can get rid of flink-shaded-hadoop. Moreover relying only on the vanilla Hadoop dependencies for the modules which interact with Hadoop/Yarn sounds like a good idea to me. Adding support for Hadoop 3 would also be nice. I'm not sure, though, how Hadoop's API's have changed between 2 and 3. It might be necessary to introduce some bridges in order to make it work. Cheers, Till On Tue, Apr 21, 2020 at 4:37 PM Robert Metzger <[hidden email]> wrote: > Hi all, > > for the upcoming 1.11 release, I started looking into adding support for > Hadoop 3[1] for Flink. I have explored a little bit already into adding a > shaded hadoop 3 into “flink-shaded”, and some mechanisms for switching > between Hadoop 2 and 3 dependencies in the Flink build. > > However, Chesnay made me aware that we could also go a different route: We > let Flink depend on vanilla Hadoop dependencies and stop providing shaded > fat jars for Hadoop through “flink-shaded”. > > Why? > - Maintaining properly shaded Hadoop fat jars is a lot of work (we have > insufficient test coverage for all kinds of Hadoop features) > - For Hadoop 2, there are already some known and unresolved issues with our > shaded jars that we didn’t manage to fix > > Users will have to use Flink with Hadoop by relying on vanilla or > vendor-provided Hadoop dependencies. > > What do you think? > > Best, > Robert > > [1] https://issues.apache.org/jira/browse/FLINK-11086 > |
+1 to getting rid of flink-shaded-hadoop. But we need to document how
people can now get a Flink dist that works with Hadoop. Currently, when you download the single shaded jar you immediately get support for submitting to YARN via bin/flink run. Aljoscha On 22.04.20 09:08, Till Rohrmann wrote: > Hi Robert, > > I think it would be a helpful simplification of Flink's build setup if we > can get rid of flink-shaded-hadoop. Moreover relying only on the vanilla > Hadoop dependencies for the modules which interact with Hadoop/Yarn sounds > like a good idea to me. > > Adding support for Hadoop 3 would also be nice. I'm not sure, though, how > Hadoop's API's have changed between 2 and 3. It might be necessary to > introduce some bridges in order to make it work. > > Cheers, > Till > > On Tue, Apr 21, 2020 at 4:37 PM Robert Metzger <[hidden email]> wrote: > >> Hi all, >> >> for the upcoming 1.11 release, I started looking into adding support for >> Hadoop 3[1] for Flink. I have explored a little bit already into adding a >> shaded hadoop 3 into “flink-shaded”, and some mechanisms for switching >> between Hadoop 2 and 3 dependencies in the Flink build. >> >> However, Chesnay made me aware that we could also go a different route: We >> let Flink depend on vanilla Hadoop dependencies and stop providing shaded >> fat jars for Hadoop through “flink-shaded”. >> >> Why? >> - Maintaining properly shaded Hadoop fat jars is a lot of work (we have >> insufficient test coverage for all kinds of Hadoop features) >> - For Hadoop 2, there are already some known and unresolved issues with our >> shaded jars that we didn’t manage to fix >> >> Users will have to use Flink with Hadoop by relying on vanilla or >> vendor-provided Hadoop dependencies. >> >> What do you think? >> >> Best, >> Robert >> >> [1] https://issues.apache.org/jira/browse/FLINK-11086 >> > |
I agree with Aljoscha. Otherwise I can see a lot of tickets getting created
saying the application is not running on YARN. Cheers, Sivaprasanna On Wed, Apr 22, 2020 at 1:00 PM Aljoscha Krettek <[hidden email]> wrote: > +1 to getting rid of flink-shaded-hadoop. But we need to document how > people can now get a Flink dist that works with Hadoop. Currently, when > you download the single shaded jar you immediately get support for > submitting to YARN via bin/flink run. > > Aljoscha > > > On 22.04.20 09:08, Till Rohrmann wrote: > > Hi Robert, > > > > I think it would be a helpful simplification of Flink's build setup if we > > can get rid of flink-shaded-hadoop. Moreover relying only on the vanilla > > Hadoop dependencies for the modules which interact with Hadoop/Yarn > sounds > > like a good idea to me. > > > > Adding support for Hadoop 3 would also be nice. I'm not sure, though, how > > Hadoop's API's have changed between 2 and 3. It might be necessary to > > introduce some bridges in order to make it work. > > > > Cheers, > > Till > > > > On Tue, Apr 21, 2020 at 4:37 PM Robert Metzger <[hidden email]> > wrote: > > > >> Hi all, > >> > >> for the upcoming 1.11 release, I started looking into adding support for > >> Hadoop 3[1] for Flink. I have explored a little bit already into adding > a > >> shaded hadoop 3 into “flink-shaded”, and some mechanisms for switching > >> between Hadoop 2 and 3 dependencies in the Flink build. > >> > >> However, Chesnay made me aware that we could also go a different route: > We > >> let Flink depend on vanilla Hadoop dependencies and stop providing > shaded > >> fat jars for Hadoop through “flink-shaded”. > >> > >> Why? > >> - Maintaining properly shaded Hadoop fat jars is a lot of work (we have > >> insufficient test coverage for all kinds of Hadoop features) > >> - For Hadoop 2, there are already some known and unresolved issues with > our > >> shaded jars that we didn’t manage to fix > >> > >> Users will have to use Flink with Hadoop by relying on vanilla or > >> vendor-provided Hadoop dependencies. > >> > >> What do you think? > >> > >> Best, > >> Robert > >> > >> [1] https://issues.apache.org/jira/browse/FLINK-11086 > >> > > > > |
Thanks Robert for starting this significant discussion.
Since hadoop3 has been released for long time and many companies have already put it in production. No matter you are using flink-shaded-hadoop2 or not, currently Flink could already run in yarn3(not sure about HDFS). Since the yarn api is always backward compatible. The difference is we could not benefit from the new features because we are using hadoop-2.4 as compile dependency. So then we need to use reflector for new features(node label, tags, etc.). All in all, i am in in favour of dropping the flink-shaded-hadoop. Just have some questions. 1. Do we still support "-include-hadoop" profile? If yes, what we will get in the lib dir? 2. I am not sure whether dropping the flink-shaded-hadoop will take some class conflicts problems. If we use "export HADOOP_CLASSPATH=`hadoop classpath`" for the hadoop env setup, then many jars will be appended to the Flink client classpath. 3. The compile hadoop version is still 2.4.1. Right? Best, Yang Sivaprasanna <[hidden email]> 于2020年4月22日周三 下午4:18写道: > I agree with Aljoscha. Otherwise I can see a lot of tickets getting created > saying the application is not running on YARN. > > Cheers, > Sivaprasanna > > On Wed, Apr 22, 2020 at 1:00 PM Aljoscha Krettek <[hidden email]> > wrote: > > > +1 to getting rid of flink-shaded-hadoop. But we need to document how > > people can now get a Flink dist that works with Hadoop. Currently, when > > you download the single shaded jar you immediately get support for > > submitting to YARN via bin/flink run. > > > > Aljoscha > > > > > > On 22.04.20 09:08, Till Rohrmann wrote: > > > Hi Robert, > > > > > > I think it would be a helpful simplification of Flink's build setup if > we > > > can get rid of flink-shaded-hadoop. Moreover relying only on the > vanilla > > > Hadoop dependencies for the modules which interact with Hadoop/Yarn > > sounds > > > like a good idea to me. > > > > > > Adding support for Hadoop 3 would also be nice. I'm not sure, though, > how > > > Hadoop's API's have changed between 2 and 3. It might be necessary to > > > introduce some bridges in order to make it work. > > > > > > Cheers, > > > Till > > > > > > On Tue, Apr 21, 2020 at 4:37 PM Robert Metzger <[hidden email]> > > wrote: > > > > > >> Hi all, > > >> > > >> for the upcoming 1.11 release, I started looking into adding support > for > > >> Hadoop 3[1] for Flink. I have explored a little bit already into > adding > > a > > >> shaded hadoop 3 into “flink-shaded”, and some mechanisms for switching > > >> between Hadoop 2 and 3 dependencies in the Flink build. > > >> > > >> However, Chesnay made me aware that we could also go a different > route: > > We > > >> let Flink depend on vanilla Hadoop dependencies and stop providing > > shaded > > >> fat jars for Hadoop through “flink-shaded”. > > >> > > >> Why? > > >> - Maintaining properly shaded Hadoop fat jars is a lot of work (we > have > > >> insufficient test coverage for all kinds of Hadoop features) > > >> - For Hadoop 2, there are already some known and unresolved issues > with > > our > > >> shaded jars that we didn’t manage to fix > > >> > > >> Users will have to use Flink with Hadoop by relying on vanilla or > > >> vendor-provided Hadoop dependencies. > > >> > > >> What do you think? > > >> > > >> Best, > > >> Robert > > >> > > >> [1] https://issues.apache.org/jira/browse/FLINK-11086 > > >> > > > > > > > > |
1) Likely not, as this again introduces a hard-dependency on
flink-shaded-hadoop. 2) Indeed; this will be something the user/cloud providers have to deal with now. 3) Yes. As a small note, we can still keep the hadoop-2 version of flink-shaded around for existing users. What I suggested was to just not release hadoop-3 versions. On 22/04/2020 14:19, Yang Wang wrote: > Thanks Robert for starting this significant discussion. > > Since hadoop3 has been released for long time and many companies have > already > put it in production. No matter you are using flink-shaded-hadoop2 or not, > currently > Flink could already run in yarn3(not sure about HDFS). Since the yarn api > is always > backward compatible. The difference is we could not benefit from the new > features > because we are using hadoop-2.4 as compile dependency. So then we need to > use > reflector for new features(node label, tags, etc.). > > All in all, i am in in favour of dropping the flink-shaded-hadoop. Just > have some questions. > 1. Do we still support "-include-hadoop" profile? If yes, what we will get > in the lib dir? > 2. I am not sure whether dropping the flink-shaded-hadoop will take some > class conflicts > problems. If we use "export HADOOP_CLASSPATH=`hadoop classpath`" for the > hadoop > env setup, then many jars will be appended to the Flink client classpath. > 3. The compile hadoop version is still 2.4.1. Right? > > > Best, > Yang > > > Sivaprasanna <[hidden email]> 于2020年4月22日周三 下午4:18写道: > >> I agree with Aljoscha. Otherwise I can see a lot of tickets getting created >> saying the application is not running on YARN. >> >> Cheers, >> Sivaprasanna >> >> On Wed, Apr 22, 2020 at 1:00 PM Aljoscha Krettek <[hidden email]> >> wrote: >> >>> +1 to getting rid of flink-shaded-hadoop. But we need to document how >>> people can now get a Flink dist that works with Hadoop. Currently, when >>> you download the single shaded jar you immediately get support for >>> submitting to YARN via bin/flink run. >>> >>> Aljoscha >>> >>> >>> On 22.04.20 09:08, Till Rohrmann wrote: >>>> Hi Robert, >>>> >>>> I think it would be a helpful simplification of Flink's build setup if >> we >>>> can get rid of flink-shaded-hadoop. Moreover relying only on the >> vanilla >>>> Hadoop dependencies for the modules which interact with Hadoop/Yarn >>> sounds >>>> like a good idea to me. >>>> >>>> Adding support for Hadoop 3 would also be nice. I'm not sure, though, >> how >>>> Hadoop's API's have changed between 2 and 3. It might be necessary to >>>> introduce some bridges in order to make it work. >>>> >>>> Cheers, >>>> Till >>>> >>>> On Tue, Apr 21, 2020 at 4:37 PM Robert Metzger <[hidden email]> >>> wrote: >>>>> Hi all, >>>>> >>>>> for the upcoming 1.11 release, I started looking into adding support >> for >>>>> Hadoop 3[1] for Flink. I have explored a little bit already into >> adding >>> a >>>>> shaded hadoop 3 into “flink-shaded”, and some mechanisms for switching >>>>> between Hadoop 2 and 3 dependencies in the Flink build. >>>>> >>>>> However, Chesnay made me aware that we could also go a different >> route: >>> We >>>>> let Flink depend on vanilla Hadoop dependencies and stop providing >>> shaded >>>>> fat jars for Hadoop through “flink-shaded”. >>>>> >>>>> Why? >>>>> - Maintaining properly shaded Hadoop fat jars is a lot of work (we >> have >>>>> insufficient test coverage for all kinds of Hadoop features) >>>>> - For Hadoop 2, there are already some known and unresolved issues >> with >>> our >>>>> shaded jars that we didn’t manage to fix >>>>> >>>>> Users will have to use Flink with Hadoop by relying on vanilla or >>>>> vendor-provided Hadoop dependencies. >>>>> >>>>> What do you think? >>>>> >>>>> Best, >>>>> Robert >>>>> >>>>> [1] https://issues.apache.org/jira/browse/FLINK-11086 >>>>> >>> |
+1 for supporting Hadoop 3.
I'm not familiar with the shading efforts, thus no comment on dropping the flink-shaded-hadoop. Correct me if I'm wrong. Despite currently the default Hadoop version for compiling is 2.4.1 in Flink, I think this does not mean Flink should support only Hadoop 2.4+. So no matter which Hadoop version we use for compiling by default, we need to use reflection for the Hadoop features/APIs that are not supported in all versions anyway. There're already many such reflections in `YarnClusterDescriptor` and `YarnResourceManager`, and might be more in future. I'm wondering whether we should have a unified mechanism (an interface / abstract class or so) that handles all these kind of Hadoop API reflections at one place. Not necessarily in the scope to this discussion though. Thank you~ Xintong Song On Wed, Apr 22, 2020 at 8:32 PM Chesnay Schepler <[hidden email]> wrote: > 1) Likely not, as this again introduces a hard-dependency on > flink-shaded-hadoop. > 2) Indeed; this will be something the user/cloud providers have to deal > with now. > 3) Yes. > > As a small note, we can still keep the hadoop-2 version of flink-shaded > around for existing users. > What I suggested was to just not release hadoop-3 versions. > > On 22/04/2020 14:19, Yang Wang wrote: > > Thanks Robert for starting this significant discussion. > > > > Since hadoop3 has been released for long time and many companies have > > already > > put it in production. No matter you are using flink-shaded-hadoop2 or > not, > > currently > > Flink could already run in yarn3(not sure about HDFS). Since the yarn api > > is always > > backward compatible. The difference is we could not benefit from the new > > features > > because we are using hadoop-2.4 as compile dependency. So then we need to > > use > > reflector for new features(node label, tags, etc.). > > > > All in all, i am in in favour of dropping the flink-shaded-hadoop. Just > > have some questions. > > 1. Do we still support "-include-hadoop" profile? If yes, what we will > get > > in the lib dir? > > 2. I am not sure whether dropping the flink-shaded-hadoop will take some > > class conflicts > > problems. If we use "export HADOOP_CLASSPATH=`hadoop classpath`" for the > > hadoop > > env setup, then many jars will be appended to the Flink client classpath. > > 3. The compile hadoop version is still 2.4.1. Right? > > > > > > Best, > > Yang > > > > > > Sivaprasanna <[hidden email]> 于2020年4月22日周三 下午4:18写道: > > > >> I agree with Aljoscha. Otherwise I can see a lot of tickets getting > created > >> saying the application is not running on YARN. > >> > >> Cheers, > >> Sivaprasanna > >> > >> On Wed, Apr 22, 2020 at 1:00 PM Aljoscha Krettek <[hidden email]> > >> wrote: > >> > >>> +1 to getting rid of flink-shaded-hadoop. But we need to document how > >>> people can now get a Flink dist that works with Hadoop. Currently, when > >>> you download the single shaded jar you immediately get support for > >>> submitting to YARN via bin/flink run. > >>> > >>> Aljoscha > >>> > >>> > >>> On 22.04.20 09:08, Till Rohrmann wrote: > >>>> Hi Robert, > >>>> > >>>> I think it would be a helpful simplification of Flink's build setup if > >> we > >>>> can get rid of flink-shaded-hadoop. Moreover relying only on the > >> vanilla > >>>> Hadoop dependencies for the modules which interact with Hadoop/Yarn > >>> sounds > >>>> like a good idea to me. > >>>> > >>>> Adding support for Hadoop 3 would also be nice. I'm not sure, though, > >> how > >>>> Hadoop's API's have changed between 2 and 3. It might be necessary to > >>>> introduce some bridges in order to make it work. > >>>> > >>>> Cheers, > >>>> Till > >>>> > >>>> On Tue, Apr 21, 2020 at 4:37 PM Robert Metzger <[hidden email]> > >>> wrote: > >>>>> Hi all, > >>>>> > >>>>> for the upcoming 1.11 release, I started looking into adding support > >> for > >>>>> Hadoop 3[1] for Flink. I have explored a little bit already into > >> adding > >>> a > >>>>> shaded hadoop 3 into “flink-shaded”, and some mechanisms for > switching > >>>>> between Hadoop 2 and 3 dependencies in the Flink build. > >>>>> > >>>>> However, Chesnay made me aware that we could also go a different > >> route: > >>> We > >>>>> let Flink depend on vanilla Hadoop dependencies and stop providing > >>> shaded > >>>>> fat jars for Hadoop through “flink-shaded”. > >>>>> > >>>>> Why? > >>>>> - Maintaining properly shaded Hadoop fat jars is a lot of work (we > >> have > >>>>> insufficient test coverage for all kinds of Hadoop features) > >>>>> - For Hadoop 2, there are already some known and unresolved issues > >> with > >>> our > >>>>> shaded jars that we didn’t manage to fix > >>>>> > >>>>> Users will have to use Flink with Hadoop by relying on vanilla or > >>>>> vendor-provided Hadoop dependencies. > >>>>> > >>>>> What do you think? > >>>>> > >>>>> Best, > >>>>> Robert > >>>>> > >>>>> [1] https://issues.apache.org/jira/browse/FLINK-11086 > >>>>> > >>> > > |
Hi all!
+1 for the simplification of dropping hadoop-shaded Have we ever investigated how much work it would be to load the HADOOP_CLASSPATH through the plugin loader? Then Hadoop's crazy dependency footprint would not spoil the main classpath. - HDFS might be very simple, because file systems are already Plugin aware - Yarn would need some extra work. In essence, we would need to discover executors also through plugins - Kerberos is the other remaining bit. We would need to switch security modules to ServiceLoaders (which we should do anyways) and also pull them from plugins. Best, Stephan On Thu, Apr 23, 2020 at 4:05 AM Xintong Song <[hidden email]> wrote: > +1 for supporting Hadoop 3. > > I'm not familiar with the shading efforts, thus no comment on dropping the > flink-shaded-hadoop. > > > Correct me if I'm wrong. Despite currently the default Hadoop version for > compiling is 2.4.1 in Flink, I think this does not mean Flink should > support only Hadoop 2.4+. So no matter which Hadoop version we use for > compiling by default, we need to use reflection for the Hadoop > features/APIs that are not supported in all versions anyway. > > > There're already many such reflections in `YarnClusterDescriptor` and > `YarnResourceManager`, and might be more in future. I'm wondering whether > we should have a unified mechanism (an interface / abstract class or so) > that handles all these kind of Hadoop API reflections at one place. Not > necessarily in the scope to this discussion though. > > > Thank you~ > > Xintong Song > > > > On Wed, Apr 22, 2020 at 8:32 PM Chesnay Schepler <[hidden email]> > wrote: > > > 1) Likely not, as this again introduces a hard-dependency on > > flink-shaded-hadoop. > > 2) Indeed; this will be something the user/cloud providers have to deal > > with now. > > 3) Yes. > > > > As a small note, we can still keep the hadoop-2 version of flink-shaded > > around for existing users. > > What I suggested was to just not release hadoop-3 versions. > > > > On 22/04/2020 14:19, Yang Wang wrote: > > > Thanks Robert for starting this significant discussion. > > > > > > Since hadoop3 has been released for long time and many companies have > > > already > > > put it in production. No matter you are using flink-shaded-hadoop2 or > > not, > > > currently > > > Flink could already run in yarn3(not sure about HDFS). Since the yarn > api > > > is always > > > backward compatible. The difference is we could not benefit from the > new > > > features > > > because we are using hadoop-2.4 as compile dependency. So then we need > to > > > use > > > reflector for new features(node label, tags, etc.). > > > > > > All in all, i am in in favour of dropping the flink-shaded-hadoop. Just > > > have some questions. > > > 1. Do we still support "-include-hadoop" profile? If yes, what we will > > get > > > in the lib dir? > > > 2. I am not sure whether dropping the flink-shaded-hadoop will take > some > > > class conflicts > > > problems. If we use "export HADOOP_CLASSPATH=`hadoop classpath`" for > the > > > hadoop > > > env setup, then many jars will be appended to the Flink client > classpath. > > > 3. The compile hadoop version is still 2.4.1. Right? > > > > > > > > > Best, > > > Yang > > > > > > > > > Sivaprasanna <[hidden email]> 于2020年4月22日周三 下午4:18写道: > > > > > >> I agree with Aljoscha. Otherwise I can see a lot of tickets getting > > created > > >> saying the application is not running on YARN. > > >> > > >> Cheers, > > >> Sivaprasanna > > >> > > >> On Wed, Apr 22, 2020 at 1:00 PM Aljoscha Krettek <[hidden email] > > > > >> wrote: > > >> > > >>> +1 to getting rid of flink-shaded-hadoop. But we need to document how > > >>> people can now get a Flink dist that works with Hadoop. Currently, > when > > >>> you download the single shaded jar you immediately get support for > > >>> submitting to YARN via bin/flink run. > > >>> > > >>> Aljoscha > > >>> > > >>> > > >>> On 22.04.20 09:08, Till Rohrmann wrote: > > >>>> Hi Robert, > > >>>> > > >>>> I think it would be a helpful simplification of Flink's build setup > if > > >> we > > >>>> can get rid of flink-shaded-hadoop. Moreover relying only on the > > >> vanilla > > >>>> Hadoop dependencies for the modules which interact with Hadoop/Yarn > > >>> sounds > > >>>> like a good idea to me. > > >>>> > > >>>> Adding support for Hadoop 3 would also be nice. I'm not sure, > though, > > >> how > > >>>> Hadoop's API's have changed between 2 and 3. It might be necessary > to > > >>>> introduce some bridges in order to make it work. > > >>>> > > >>>> Cheers, > > >>>> Till > > >>>> > > >>>> On Tue, Apr 21, 2020 at 4:37 PM Robert Metzger <[hidden email] > > > > >>> wrote: > > >>>>> Hi all, > > >>>>> > > >>>>> for the upcoming 1.11 release, I started looking into adding > support > > >> for > > >>>>> Hadoop 3[1] for Flink. I have explored a little bit already into > > >> adding > > >>> a > > >>>>> shaded hadoop 3 into “flink-shaded”, and some mechanisms for > > switching > > >>>>> between Hadoop 2 and 3 dependencies in the Flink build. > > >>>>> > > >>>>> However, Chesnay made me aware that we could also go a different > > >> route: > > >>> We > > >>>>> let Flink depend on vanilla Hadoop dependencies and stop providing > > >>> shaded > > >>>>> fat jars for Hadoop through “flink-shaded”. > > >>>>> > > >>>>> Why? > > >>>>> - Maintaining properly shaded Hadoop fat jars is a lot of work (we > > >> have > > >>>>> insufficient test coverage for all kinds of Hadoop features) > > >>>>> - For Hadoop 2, there are already some known and unresolved issues > > >> with > > >>> our > > >>>>> shaded jars that we didn’t manage to fix > > >>>>> > > >>>>> Users will have to use Flink with Hadoop by relying on vanilla or > > >>>>> vendor-provided Hadoop dependencies. > > >>>>> > > >>>>> What do you think? > > >>>>> > > >>>>> Best, > > >>>>> Robert > > >>>>> > > >>>>> [1] https://issues.apache.org/jira/browse/FLINK-11086 > > >>>>> > > >>> > > > > > |
@Stephan I'm not aware of anyone having tried that; possibly since we
have various connectors that require hadoop (hadoop-compat, hive, orc/parquet/hbase, hadoop inputformats). This would require connectors to be loaded as plugins (or having access to the plugin classloader) to be feasible. On 23/04/2020 09:59, Stephan Ewen wrote: > Hi all! > > +1 for the simplification of dropping hadoop-shaded > > > Have we ever investigated how much work it would be to load the > HADOOP_CLASSPATH through the plugin loader? Then Hadoop's crazy dependency > footprint would not spoil the main classpath. > > - HDFS might be very simple, because file systems are already Plugin aware > - Yarn would need some extra work. In essence, we would need to discover > executors also through plugins > - Kerberos is the other remaining bit. We would need to switch security > modules to ServiceLoaders (which we should do anyways) and also pull them > from plugins. > > Best, > Stephan > > > > On Thu, Apr 23, 2020 at 4:05 AM Xintong Song <[hidden email]> wrote: > >> +1 for supporting Hadoop 3. >> >> I'm not familiar with the shading efforts, thus no comment on dropping the >> flink-shaded-hadoop. >> >> >> Correct me if I'm wrong. Despite currently the default Hadoop version for >> compiling is 2.4.1 in Flink, I think this does not mean Flink should >> support only Hadoop 2.4+. So no matter which Hadoop version we use for >> compiling by default, we need to use reflection for the Hadoop >> features/APIs that are not supported in all versions anyway. >> >> >> There're already many such reflections in `YarnClusterDescriptor` and >> `YarnResourceManager`, and might be more in future. I'm wondering whether >> we should have a unified mechanism (an interface / abstract class or so) >> that handles all these kind of Hadoop API reflections at one place. Not >> necessarily in the scope to this discussion though. >> >> >> Thank you~ >> >> Xintong Song >> >> >> >> On Wed, Apr 22, 2020 at 8:32 PM Chesnay Schepler <[hidden email]> >> wrote: >> >>> 1) Likely not, as this again introduces a hard-dependency on >>> flink-shaded-hadoop. >>> 2) Indeed; this will be something the user/cloud providers have to deal >>> with now. >>> 3) Yes. >>> >>> As a small note, we can still keep the hadoop-2 version of flink-shaded >>> around for existing users. >>> What I suggested was to just not release hadoop-3 versions. >>> >>> On 22/04/2020 14:19, Yang Wang wrote: >>>> Thanks Robert for starting this significant discussion. >>>> >>>> Since hadoop3 has been released for long time and many companies have >>>> already >>>> put it in production. No matter you are using flink-shaded-hadoop2 or >>> not, >>>> currently >>>> Flink could already run in yarn3(not sure about HDFS). Since the yarn >> api >>>> is always >>>> backward compatible. The difference is we could not benefit from the >> new >>>> features >>>> because we are using hadoop-2.4 as compile dependency. So then we need >> to >>>> use >>>> reflector for new features(node label, tags, etc.). >>>> >>>> All in all, i am in in favour of dropping the flink-shaded-hadoop. Just >>>> have some questions. >>>> 1. Do we still support "-include-hadoop" profile? If yes, what we will >>> get >>>> in the lib dir? >>>> 2. I am not sure whether dropping the flink-shaded-hadoop will take >> some >>>> class conflicts >>>> problems. If we use "export HADOOP_CLASSPATH=`hadoop classpath`" for >> the >>>> hadoop >>>> env setup, then many jars will be appended to the Flink client >> classpath. >>>> 3. The compile hadoop version is still 2.4.1. Right? >>>> >>>> >>>> Best, >>>> Yang >>>> >>>> >>>> Sivaprasanna <[hidden email]> 于2020年4月22日周三 下午4:18写道: >>>> >>>>> I agree with Aljoscha. Otherwise I can see a lot of tickets getting >>> created >>>>> saying the application is not running on YARN. >>>>> >>>>> Cheers, >>>>> Sivaprasanna >>>>> >>>>> On Wed, Apr 22, 2020 at 1:00 PM Aljoscha Krettek <[hidden email] >>>>> wrote: >>>>> >>>>>> +1 to getting rid of flink-shaded-hadoop. But we need to document how >>>>>> people can now get a Flink dist that works with Hadoop. Currently, >> when >>>>>> you download the single shaded jar you immediately get support for >>>>>> submitting to YARN via bin/flink run. >>>>>> >>>>>> Aljoscha >>>>>> >>>>>> >>>>>> On 22.04.20 09:08, Till Rohrmann wrote: >>>>>>> Hi Robert, >>>>>>> >>>>>>> I think it would be a helpful simplification of Flink's build setup >> if >>>>> we >>>>>>> can get rid of flink-shaded-hadoop. Moreover relying only on the >>>>> vanilla >>>>>>> Hadoop dependencies for the modules which interact with Hadoop/Yarn >>>>>> sounds >>>>>>> like a good idea to me. >>>>>>> >>>>>>> Adding support for Hadoop 3 would also be nice. I'm not sure, >> though, >>>>> how >>>>>>> Hadoop's API's have changed between 2 and 3. It might be necessary >> to >>>>>>> introduce some bridges in order to make it work. >>>>>>> >>>>>>> Cheers, >>>>>>> Till >>>>>>> >>>>>>> On Tue, Apr 21, 2020 at 4:37 PM Robert Metzger <[hidden email] >>>>>> wrote: >>>>>>>> Hi all, >>>>>>>> >>>>>>>> for the upcoming 1.11 release, I started looking into adding >> support >>>>> for >>>>>>>> Hadoop 3[1] for Flink. I have explored a little bit already into >>>>> adding >>>>>> a >>>>>>>> shaded hadoop 3 into “flink-shaded”, and some mechanisms for >>> switching >>>>>>>> between Hadoop 2 and 3 dependencies in the Flink build. >>>>>>>> >>>>>>>> However, Chesnay made me aware that we could also go a different >>>>> route: >>>>>> We >>>>>>>> let Flink depend on vanilla Hadoop dependencies and stop providing >>>>>> shaded >>>>>>>> fat jars for Hadoop through “flink-shaded”. >>>>>>>> >>>>>>>> Why? >>>>>>>> - Maintaining properly shaded Hadoop fat jars is a lot of work (we >>>>> have >>>>>>>> insufficient test coverage for all kinds of Hadoop features) >>>>>>>> - For Hadoop 2, there are already some known and unresolved issues >>>>> with >>>>>> our >>>>>>>> shaded jars that we didn’t manage to fix >>>>>>>> >>>>>>>> Users will have to use Flink with Hadoop by relying on vanilla or >>>>>>>> vendor-provided Hadoop dependencies. >>>>>>>> >>>>>>>> What do you think? >>>>>>>> >>>>>>>> Best, >>>>>>>> Robert >>>>>>>> >>>>>>>> [1] https://issues.apache.org/jira/browse/FLINK-11086 >>>>>>>> >>> |
although, if you can load the HADOOP_CLASSPATH as a plugin, then you can
also load it in the user-code classloader. On 23/04/2020 10:50, Chesnay Schepler wrote: > @Stephan I'm not aware of anyone having tried that; possibly since we > have various connectors that require hadoop (hadoop-compat, hive, > orc/parquet/hbase, hadoop inputformats). This would require connectors > to be loaded as plugins (or having access to the plugin classloader) > to be feasible. > > On 23/04/2020 09:59, Stephan Ewen wrote: >> Hi all! >> >> +1 for the simplification of dropping hadoop-shaded >> >> >> Have we ever investigated how much work it would be to load the >> HADOOP_CLASSPATH through the plugin loader? Then Hadoop's crazy >> dependency >> footprint would not spoil the main classpath. >> >> - HDFS might be very simple, because file systems are already >> Plugin aware >> - Yarn would need some extra work. In essence, we would need to >> discover >> executors also through plugins >> - Kerberos is the other remaining bit. We would need to switch >> security >> modules to ServiceLoaders (which we should do anyways) and also pull >> them >> from plugins. >> >> Best, >> Stephan >> >> >> >> On Thu, Apr 23, 2020 at 4:05 AM Xintong Song <[hidden email]> >> wrote: >> >>> +1 for supporting Hadoop 3. >>> >>> I'm not familiar with the shading efforts, thus no comment on >>> dropping the >>> flink-shaded-hadoop. >>> >>> >>> Correct me if I'm wrong. Despite currently the default Hadoop >>> version for >>> compiling is 2.4.1 in Flink, I think this does not mean Flink should >>> support only Hadoop 2.4+. So no matter which Hadoop version we use for >>> compiling by default, we need to use reflection for the Hadoop >>> features/APIs that are not supported in all versions anyway. >>> >>> >>> There're already many such reflections in `YarnClusterDescriptor` and >>> `YarnResourceManager`, and might be more in future. I'm wondering >>> whether >>> we should have a unified mechanism (an interface / abstract class or >>> so) >>> that handles all these kind of Hadoop API reflections at one place. Not >>> necessarily in the scope to this discussion though. >>> >>> >>> Thank you~ >>> >>> Xintong Song >>> >>> >>> >>> On Wed, Apr 22, 2020 at 8:32 PM Chesnay Schepler <[hidden email]> >>> wrote: >>> >>>> 1) Likely not, as this again introduces a hard-dependency on >>>> flink-shaded-hadoop. >>>> 2) Indeed; this will be something the user/cloud providers have to >>>> deal >>>> with now. >>>> 3) Yes. >>>> >>>> As a small note, we can still keep the hadoop-2 version of >>>> flink-shaded >>>> around for existing users. >>>> What I suggested was to just not release hadoop-3 versions. >>>> >>>> On 22/04/2020 14:19, Yang Wang wrote: >>>>> Thanks Robert for starting this significant discussion. >>>>> >>>>> Since hadoop3 has been released for long time and many companies have >>>>> already >>>>> put it in production. No matter you are using flink-shaded-hadoop2 or >>>> not, >>>>> currently >>>>> Flink could already run in yarn3(not sure about HDFS). Since the yarn >>> api >>>>> is always >>>>> backward compatible. The difference is we could not benefit from the >>> new >>>>> features >>>>> because we are using hadoop-2.4 as compile dependency. So then we >>>>> need >>> to >>>>> use >>>>> reflector for new features(node label, tags, etc.). >>>>> >>>>> All in all, i am in in favour of dropping the flink-shaded-hadoop. >>>>> Just >>>>> have some questions. >>>>> 1. Do we still support "-include-hadoop" profile? If yes, what we >>>>> will >>>> get >>>>> in the lib dir? >>>>> 2. I am not sure whether dropping the flink-shaded-hadoop will take >>> some >>>>> class conflicts >>>>> problems. If we use "export HADOOP_CLASSPATH=`hadoop classpath`" for >>> the >>>>> hadoop >>>>> env setup, then many jars will be appended to the Flink client >>> classpath. >>>>> 3. The compile hadoop version is still 2.4.1. Right? >>>>> >>>>> >>>>> Best, >>>>> Yang >>>>> >>>>> >>>>> Sivaprasanna <[hidden email]> 于2020年4月22日周三 >>>>> 下午4:18写道: >>>>> >>>>>> I agree with Aljoscha. Otherwise I can see a lot of tickets getting >>>> created >>>>>> saying the application is not running on YARN. >>>>>> >>>>>> Cheers, >>>>>> Sivaprasanna >>>>>> >>>>>> On Wed, Apr 22, 2020 at 1:00 PM Aljoscha Krettek >>>>>> <[hidden email] >>>>>> wrote: >>>>>> >>>>>>> +1 to getting rid of flink-shaded-hadoop. But we need to >>>>>>> document how >>>>>>> people can now get a Flink dist that works with Hadoop. Currently, >>> when >>>>>>> you download the single shaded jar you immediately get support for >>>>>>> submitting to YARN via bin/flink run. >>>>>>> >>>>>>> Aljoscha >>>>>>> >>>>>>> >>>>>>> On 22.04.20 09:08, Till Rohrmann wrote: >>>>>>>> Hi Robert, >>>>>>>> >>>>>>>> I think it would be a helpful simplification of Flink's build >>>>>>>> setup >>> if >>>>>> we >>>>>>>> can get rid of flink-shaded-hadoop. Moreover relying only on the >>>>>> vanilla >>>>>>>> Hadoop dependencies for the modules which interact with >>>>>>>> Hadoop/Yarn >>>>>>> sounds >>>>>>>> like a good idea to me. >>>>>>>> >>>>>>>> Adding support for Hadoop 3 would also be nice. I'm not sure, >>> though, >>>>>> how >>>>>>>> Hadoop's API's have changed between 2 and 3. It might be necessary >>> to >>>>>>>> introduce some bridges in order to make it work. >>>>>>>> >>>>>>>> Cheers, >>>>>>>> Till >>>>>>>> >>>>>>>> On Tue, Apr 21, 2020 at 4:37 PM Robert Metzger >>>>>>>> <[hidden email] >>>>>>> wrote: >>>>>>>>> Hi all, >>>>>>>>> >>>>>>>>> for the upcoming 1.11 release, I started looking into adding >>> support >>>>>> for >>>>>>>>> Hadoop 3[1] for Flink. I have explored a little bit already into >>>>>> adding >>>>>>> a >>>>>>>>> shaded hadoop 3 into “flink-shaded”, and some mechanisms for >>>> switching >>>>>>>>> between Hadoop 2 and 3 dependencies in the Flink build. >>>>>>>>> >>>>>>>>> However, Chesnay made me aware that we could also go a different >>>>>> route: >>>>>>> We >>>>>>>>> let Flink depend on vanilla Hadoop dependencies and stop >>>>>>>>> providing >>>>>>> shaded >>>>>>>>> fat jars for Hadoop through “flink-shaded”. >>>>>>>>> >>>>>>>>> Why? >>>>>>>>> - Maintaining properly shaded Hadoop fat jars is a lot of work >>>>>>>>> (we >>>>>> have >>>>>>>>> insufficient test coverage for all kinds of Hadoop features) >>>>>>>>> - For Hadoop 2, there are already some known and unresolved >>>>>>>>> issues >>>>>> with >>>>>>> our >>>>>>>>> shaded jars that we didn’t manage to fix >>>>>>>>> >>>>>>>>> Users will have to use Flink with Hadoop by relying on vanilla or >>>>>>>>> vendor-provided Hadoop dependencies. >>>>>>>>> >>>>>>>>> What do you think? >>>>>>>>> >>>>>>>>> Best, >>>>>>>>> Robert >>>>>>>>> >>>>>>>>> [1] https://issues.apache.org/jira/browse/FLINK-11086 >>>>>>>>> >>>> > > |
True, connectors built on Hadoop make this a bit more complex. That is also
the reason why Hadoop is on the "parent first" patterns. Maybe this is a bit of a wild thought, but what would happen if we had a "first class" notion of a Hadoop Classloader in the system, and the user code classloader would explicitly fall back to that one whenever a class whose name starts with "org.apache.hadoop" is not found? We could also generalize this by associating plugin loaders with class name prefixes. Then it would try to load from the user code jar, and if the class was not found, load it from the hadoop classpath. On Thu, Apr 23, 2020 at 10:56 AM Chesnay Schepler <[hidden email]> wrote: > although, if you can load the HADOOP_CLASSPATH as a plugin, then you can > also load it in the user-code classloader. > > On 23/04/2020 10:50, Chesnay Schepler wrote: > > @Stephan I'm not aware of anyone having tried that; possibly since we > > have various connectors that require hadoop (hadoop-compat, hive, > > orc/parquet/hbase, hadoop inputformats). This would require connectors > > to be loaded as plugins (or having access to the plugin classloader) > > to be feasible. > > > > On 23/04/2020 09:59, Stephan Ewen wrote: > >> Hi all! > >> > >> +1 for the simplification of dropping hadoop-shaded > >> > >> > >> Have we ever investigated how much work it would be to load the > >> HADOOP_CLASSPATH through the plugin loader? Then Hadoop's crazy > >> dependency > >> footprint would not spoil the main classpath. > >> > >> - HDFS might be very simple, because file systems are already > >> Plugin aware > >> - Yarn would need some extra work. In essence, we would need to > >> discover > >> executors also through plugins > >> - Kerberos is the other remaining bit. We would need to switch > >> security > >> modules to ServiceLoaders (which we should do anyways) and also pull > >> them > >> from plugins. > >> > >> Best, > >> Stephan > >> > >> > >> > >> On Thu, Apr 23, 2020 at 4:05 AM Xintong Song <[hidden email]> > >> wrote: > >> > >>> +1 for supporting Hadoop 3. > >>> > >>> I'm not familiar with the shading efforts, thus no comment on > >>> dropping the > >>> flink-shaded-hadoop. > >>> > >>> > >>> Correct me if I'm wrong. Despite currently the default Hadoop > >>> version for > >>> compiling is 2.4.1 in Flink, I think this does not mean Flink should > >>> support only Hadoop 2.4+. So no matter which Hadoop version we use for > >>> compiling by default, we need to use reflection for the Hadoop > >>> features/APIs that are not supported in all versions anyway. > >>> > >>> > >>> There're already many such reflections in `YarnClusterDescriptor` and > >>> `YarnResourceManager`, and might be more in future. I'm wondering > >>> whether > >>> we should have a unified mechanism (an interface / abstract class or > >>> so) > >>> that handles all these kind of Hadoop API reflections at one place. Not > >>> necessarily in the scope to this discussion though. > >>> > >>> > >>> Thank you~ > >>> > >>> Xintong Song > >>> > >>> > >>> > >>> On Wed, Apr 22, 2020 at 8:32 PM Chesnay Schepler <[hidden email]> > >>> wrote: > >>> > >>>> 1) Likely not, as this again introduces a hard-dependency on > >>>> flink-shaded-hadoop. > >>>> 2) Indeed; this will be something the user/cloud providers have to > >>>> deal > >>>> with now. > >>>> 3) Yes. > >>>> > >>>> As a small note, we can still keep the hadoop-2 version of > >>>> flink-shaded > >>>> around for existing users. > >>>> What I suggested was to just not release hadoop-3 versions. > >>>> > >>>> On 22/04/2020 14:19, Yang Wang wrote: > >>>>> Thanks Robert for starting this significant discussion. > >>>>> > >>>>> Since hadoop3 has been released for long time and many companies have > >>>>> already > >>>>> put it in production. No matter you are using flink-shaded-hadoop2 or > >>>> not, > >>>>> currently > >>>>> Flink could already run in yarn3(not sure about HDFS). Since the yarn > >>> api > >>>>> is always > >>>>> backward compatible. The difference is we could not benefit from the > >>> new > >>>>> features > >>>>> because we are using hadoop-2.4 as compile dependency. So then we > >>>>> need > >>> to > >>>>> use > >>>>> reflector for new features(node label, tags, etc.). > >>>>> > >>>>> All in all, i am in in favour of dropping the flink-shaded-hadoop. > >>>>> Just > >>>>> have some questions. > >>>>> 1. Do we still support "-include-hadoop" profile? If yes, what we > >>>>> will > >>>> get > >>>>> in the lib dir? > >>>>> 2. I am not sure whether dropping the flink-shaded-hadoop will take > >>> some > >>>>> class conflicts > >>>>> problems. If we use "export HADOOP_CLASSPATH=`hadoop classpath`" for > >>> the > >>>>> hadoop > >>>>> env setup, then many jars will be appended to the Flink client > >>> classpath. > >>>>> 3. The compile hadoop version is still 2.4.1. Right? > >>>>> > >>>>> > >>>>> Best, > >>>>> Yang > >>>>> > >>>>> > >>>>> Sivaprasanna <[hidden email]> 于2020年4月22日周三 > >>>>> 下午4:18写道: > >>>>> > >>>>>> I agree with Aljoscha. Otherwise I can see a lot of tickets getting > >>>> created > >>>>>> saying the application is not running on YARN. > >>>>>> > >>>>>> Cheers, > >>>>>> Sivaprasanna > >>>>>> > >>>>>> On Wed, Apr 22, 2020 at 1:00 PM Aljoscha Krettek > >>>>>> <[hidden email] > >>>>>> wrote: > >>>>>> > >>>>>>> +1 to getting rid of flink-shaded-hadoop. But we need to > >>>>>>> document how > >>>>>>> people can now get a Flink dist that works with Hadoop. Currently, > >>> when > >>>>>>> you download the single shaded jar you immediately get support for > >>>>>>> submitting to YARN via bin/flink run. > >>>>>>> > >>>>>>> Aljoscha > >>>>>>> > >>>>>>> > >>>>>>> On 22.04.20 09:08, Till Rohrmann wrote: > >>>>>>>> Hi Robert, > >>>>>>>> > >>>>>>>> I think it would be a helpful simplification of Flink's build > >>>>>>>> setup > >>> if > >>>>>> we > >>>>>>>> can get rid of flink-shaded-hadoop. Moreover relying only on the > >>>>>> vanilla > >>>>>>>> Hadoop dependencies for the modules which interact with > >>>>>>>> Hadoop/Yarn > >>>>>>> sounds > >>>>>>>> like a good idea to me. > >>>>>>>> > >>>>>>>> Adding support for Hadoop 3 would also be nice. I'm not sure, > >>> though, > >>>>>> how > >>>>>>>> Hadoop's API's have changed between 2 and 3. It might be necessary > >>> to > >>>>>>>> introduce some bridges in order to make it work. > >>>>>>>> > >>>>>>>> Cheers, > >>>>>>>> Till > >>>>>>>> > >>>>>>>> On Tue, Apr 21, 2020 at 4:37 PM Robert Metzger > >>>>>>>> <[hidden email] > >>>>>>> wrote: > >>>>>>>>> Hi all, > >>>>>>>>> > >>>>>>>>> for the upcoming 1.11 release, I started looking into adding > >>> support > >>>>>> for > >>>>>>>>> Hadoop 3[1] for Flink. I have explored a little bit already into > >>>>>> adding > >>>>>>> a > >>>>>>>>> shaded hadoop 3 into “flink-shaded”, and some mechanisms for > >>>> switching > >>>>>>>>> between Hadoop 2 and 3 dependencies in the Flink build. > >>>>>>>>> > >>>>>>>>> However, Chesnay made me aware that we could also go a different > >>>>>> route: > >>>>>>> We > >>>>>>>>> let Flink depend on vanilla Hadoop dependencies and stop > >>>>>>>>> providing > >>>>>>> shaded > >>>>>>>>> fat jars for Hadoop through “flink-shaded”. > >>>>>>>>> > >>>>>>>>> Why? > >>>>>>>>> - Maintaining properly shaded Hadoop fat jars is a lot of work > >>>>>>>>> (we > >>>>>> have > >>>>>>>>> insufficient test coverage for all kinds of Hadoop features) > >>>>>>>>> - For Hadoop 2, there are already some known and unresolved > >>>>>>>>> issues > >>>>>> with > >>>>>>> our > >>>>>>>>> shaded jars that we didn’t manage to fix > >>>>>>>>> > >>>>>>>>> Users will have to use Flink with Hadoop by relying on vanilla or > >>>>>>>>> vendor-provided Hadoop dependencies. > >>>>>>>>> > >>>>>>>>> What do you think? > >>>>>>>>> > >>>>>>>>> Best, > >>>>>>>>> Robert > >>>>>>>>> > >>>>>>>>> [1] https://issues.apache.org/jira/browse/FLINK-11086 > >>>>>>>>> > >>>> > > > > > > |
This would only work so long as all Hadoop APIs do not directly expose
any transitive non-hadoop dependency. Otherwise the user code classloader might search for this transitive dependency in lib instead of the hadoop classpath (and possibly not find it). On 23/04/2020 11:34, Stephan Ewen wrote: > True, connectors built on Hadoop make this a bit more complex. That is also > the reason why Hadoop is on the "parent first" patterns. > > Maybe this is a bit of a wild thought, but what would happen if we had a > "first class" notion of a Hadoop Classloader in the system, and the user > code classloader would explicitly fall back to that one whenever a class > whose name starts with "org.apache.hadoop" is not found? We could also > generalize this by associating plugin loaders with class name prefixes. > > Then it would try to load from the user code jar, and if the class was not > found, load it from the hadoop classpath. > > On Thu, Apr 23, 2020 at 10:56 AM Chesnay Schepler <[hidden email]> > wrote: > >> although, if you can load the HADOOP_CLASSPATH as a plugin, then you can >> also load it in the user-code classloader. >> >> On 23/04/2020 10:50, Chesnay Schepler wrote: >>> @Stephan I'm not aware of anyone having tried that; possibly since we >>> have various connectors that require hadoop (hadoop-compat, hive, >>> orc/parquet/hbase, hadoop inputformats). This would require connectors >>> to be loaded as plugins (or having access to the plugin classloader) >>> to be feasible. >>> >>> On 23/04/2020 09:59, Stephan Ewen wrote: >>>> Hi all! >>>> >>>> +1 for the simplification of dropping hadoop-shaded >>>> >>>> >>>> Have we ever investigated how much work it would be to load the >>>> HADOOP_CLASSPATH through the plugin loader? Then Hadoop's crazy >>>> dependency >>>> footprint would not spoil the main classpath. >>>> >>>> - HDFS might be very simple, because file systems are already >>>> Plugin aware >>>> - Yarn would need some extra work. In essence, we would need to >>>> discover >>>> executors also through plugins >>>> - Kerberos is the other remaining bit. We would need to switch >>>> security >>>> modules to ServiceLoaders (which we should do anyways) and also pull >>>> them >>>> from plugins. >>>> >>>> Best, >>>> Stephan >>>> >>>> >>>> >>>> On Thu, Apr 23, 2020 at 4:05 AM Xintong Song <[hidden email]> >>>> wrote: >>>> >>>>> +1 for supporting Hadoop 3. >>>>> >>>>> I'm not familiar with the shading efforts, thus no comment on >>>>> dropping the >>>>> flink-shaded-hadoop. >>>>> >>>>> >>>>> Correct me if I'm wrong. Despite currently the default Hadoop >>>>> version for >>>>> compiling is 2.4.1 in Flink, I think this does not mean Flink should >>>>> support only Hadoop 2.4+. So no matter which Hadoop version we use for >>>>> compiling by default, we need to use reflection for the Hadoop >>>>> features/APIs that are not supported in all versions anyway. >>>>> >>>>> >>>>> There're already many such reflections in `YarnClusterDescriptor` and >>>>> `YarnResourceManager`, and might be more in future. I'm wondering >>>>> whether >>>>> we should have a unified mechanism (an interface / abstract class or >>>>> so) >>>>> that handles all these kind of Hadoop API reflections at one place. Not >>>>> necessarily in the scope to this discussion though. >>>>> >>>>> >>>>> Thank you~ >>>>> >>>>> Xintong Song >>>>> >>>>> >>>>> >>>>> On Wed, Apr 22, 2020 at 8:32 PM Chesnay Schepler <[hidden email]> >>>>> wrote: >>>>> >>>>>> 1) Likely not, as this again introduces a hard-dependency on >>>>>> flink-shaded-hadoop. >>>>>> 2) Indeed; this will be something the user/cloud providers have to >>>>>> deal >>>>>> with now. >>>>>> 3) Yes. >>>>>> >>>>>> As a small note, we can still keep the hadoop-2 version of >>>>>> flink-shaded >>>>>> around for existing users. >>>>>> What I suggested was to just not release hadoop-3 versions. >>>>>> >>>>>> On 22/04/2020 14:19, Yang Wang wrote: >>>>>>> Thanks Robert for starting this significant discussion. >>>>>>> >>>>>>> Since hadoop3 has been released for long time and many companies have >>>>>>> already >>>>>>> put it in production. No matter you are using flink-shaded-hadoop2 or >>>>>> not, >>>>>>> currently >>>>>>> Flink could already run in yarn3(not sure about HDFS). Since the yarn >>>>> api >>>>>>> is always >>>>>>> backward compatible. The difference is we could not benefit from the >>>>> new >>>>>>> features >>>>>>> because we are using hadoop-2.4 as compile dependency. So then we >>>>>>> need >>>>> to >>>>>>> use >>>>>>> reflector for new features(node label, tags, etc.). >>>>>>> >>>>>>> All in all, i am in in favour of dropping the flink-shaded-hadoop. >>>>>>> Just >>>>>>> have some questions. >>>>>>> 1. Do we still support "-include-hadoop" profile? If yes, what we >>>>>>> will >>>>>> get >>>>>>> in the lib dir? >>>>>>> 2. I am not sure whether dropping the flink-shaded-hadoop will take >>>>> some >>>>>>> class conflicts >>>>>>> problems. If we use "export HADOOP_CLASSPATH=`hadoop classpath`" for >>>>> the >>>>>>> hadoop >>>>>>> env setup, then many jars will be appended to the Flink client >>>>> classpath. >>>>>>> 3. The compile hadoop version is still 2.4.1. Right? >>>>>>> >>>>>>> >>>>>>> Best, >>>>>>> Yang >>>>>>> >>>>>>> >>>>>>> Sivaprasanna <[hidden email]> 于2020年4月22日周三 >>>>>>> 下午4:18写道: >>>>>>> >>>>>>>> I agree with Aljoscha. Otherwise I can see a lot of tickets getting >>>>>> created >>>>>>>> saying the application is not running on YARN. >>>>>>>> >>>>>>>> Cheers, >>>>>>>> Sivaprasanna >>>>>>>> >>>>>>>> On Wed, Apr 22, 2020 at 1:00 PM Aljoscha Krettek >>>>>>>> <[hidden email] >>>>>>>> wrote: >>>>>>>> >>>>>>>>> +1 to getting rid of flink-shaded-hadoop. But we need to >>>>>>>>> document how >>>>>>>>> people can now get a Flink dist that works with Hadoop. Currently, >>>>> when >>>>>>>>> you download the single shaded jar you immediately get support for >>>>>>>>> submitting to YARN via bin/flink run. >>>>>>>>> >>>>>>>>> Aljoscha >>>>>>>>> >>>>>>>>> >>>>>>>>> On 22.04.20 09:08, Till Rohrmann wrote: >>>>>>>>>> Hi Robert, >>>>>>>>>> >>>>>>>>>> I think it would be a helpful simplification of Flink's build >>>>>>>>>> setup >>>>> if >>>>>>>> we >>>>>>>>>> can get rid of flink-shaded-hadoop. Moreover relying only on the >>>>>>>> vanilla >>>>>>>>>> Hadoop dependencies for the modules which interact with >>>>>>>>>> Hadoop/Yarn >>>>>>>>> sounds >>>>>>>>>> like a good idea to me. >>>>>>>>>> >>>>>>>>>> Adding support for Hadoop 3 would also be nice. I'm not sure, >>>>> though, >>>>>>>> how >>>>>>>>>> Hadoop's API's have changed between 2 and 3. It might be necessary >>>>> to >>>>>>>>>> introduce some bridges in order to make it work. >>>>>>>>>> >>>>>>>>>> Cheers, >>>>>>>>>> Till >>>>>>>>>> >>>>>>>>>> On Tue, Apr 21, 2020 at 4:37 PM Robert Metzger >>>>>>>>>> <[hidden email] >>>>>>>>> wrote: >>>>>>>>>>> Hi all, >>>>>>>>>>> >>>>>>>>>>> for the upcoming 1.11 release, I started looking into adding >>>>> support >>>>>>>> for >>>>>>>>>>> Hadoop 3[1] for Flink. I have explored a little bit already into >>>>>>>> adding >>>>>>>>> a >>>>>>>>>>> shaded hadoop 3 into “flink-shaded”, and some mechanisms for >>>>>> switching >>>>>>>>>>> between Hadoop 2 and 3 dependencies in the Flink build. >>>>>>>>>>> >>>>>>>>>>> However, Chesnay made me aware that we could also go a different >>>>>>>> route: >>>>>>>>> We >>>>>>>>>>> let Flink depend on vanilla Hadoop dependencies and stop >>>>>>>>>>> providing >>>>>>>>> shaded >>>>>>>>>>> fat jars for Hadoop through “flink-shaded”. >>>>>>>>>>> >>>>>>>>>>> Why? >>>>>>>>>>> - Maintaining properly shaded Hadoop fat jars is a lot of work >>>>>>>>>>> (we >>>>>>>> have >>>>>>>>>>> insufficient test coverage for all kinds of Hadoop features) >>>>>>>>>>> - For Hadoop 2, there are already some known and unresolved >>>>>>>>>>> issues >>>>>>>> with >>>>>>>>> our >>>>>>>>>>> shaded jars that we didn’t manage to fix >>>>>>>>>>> >>>>>>>>>>> Users will have to use Flink with Hadoop by relying on vanilla or >>>>>>>>>>> vendor-provided Hadoop dependencies. >>>>>>>>>>> >>>>>>>>>>> What do you think? >>>>>>>>>>> >>>>>>>>>>> Best, >>>>>>>>>>> Robert >>>>>>>>>>> >>>>>>>>>>> [1] https://issues.apache.org/jira/browse/FLINK-11086 >>>>>>>>>>> >>> >> |
Indeed, that would be the assumption, that Hadoop does not expose its
transitive libraries on its public API surface. From vague memory, I think that pretty much true so far. I only remember Kinesis and Calcite as counter examples, who exposed Guava classes as part of the public API. But that is definitely the "weak spot" of this approach. Plus, as with all custom class loaders, the fact that the Thread Context Class Loader does not really work well any more. On Thu, Apr 23, 2020 at 11:50 AM Chesnay Schepler <[hidden email]> wrote: > This would only work so long as all Hadoop APIs do not directly expose > any transitive non-hadoop dependency. > Otherwise the user code classloader might search for this transitive > dependency in lib instead of the hadoop classpath (and possibly not find > it). > > On 23/04/2020 11:34, Stephan Ewen wrote: > > True, connectors built on Hadoop make this a bit more complex. That is > also > > the reason why Hadoop is on the "parent first" patterns. > > > > Maybe this is a bit of a wild thought, but what would happen if we had a > > "first class" notion of a Hadoop Classloader in the system, and the user > > code classloader would explicitly fall back to that one whenever a class > > whose name starts with "org.apache.hadoop" is not found? We could also > > generalize this by associating plugin loaders with class name prefixes. > > > > Then it would try to load from the user code jar, and if the class was > not > > found, load it from the hadoop classpath. > > > > On Thu, Apr 23, 2020 at 10:56 AM Chesnay Schepler <[hidden email]> > > wrote: > > > >> although, if you can load the HADOOP_CLASSPATH as a plugin, then you can > >> also load it in the user-code classloader. > >> > >> On 23/04/2020 10:50, Chesnay Schepler wrote: > >>> @Stephan I'm not aware of anyone having tried that; possibly since we > >>> have various connectors that require hadoop (hadoop-compat, hive, > >>> orc/parquet/hbase, hadoop inputformats). This would require connectors > >>> to be loaded as plugins (or having access to the plugin classloader) > >>> to be feasible. > >>> > >>> On 23/04/2020 09:59, Stephan Ewen wrote: > >>>> Hi all! > >>>> > >>>> +1 for the simplification of dropping hadoop-shaded > >>>> > >>>> > >>>> Have we ever investigated how much work it would be to load the > >>>> HADOOP_CLASSPATH through the plugin loader? Then Hadoop's crazy > >>>> dependency > >>>> footprint would not spoil the main classpath. > >>>> > >>>> - HDFS might be very simple, because file systems are already > >>>> Plugin aware > >>>> - Yarn would need some extra work. In essence, we would need to > >>>> discover > >>>> executors also through plugins > >>>> - Kerberos is the other remaining bit. We would need to switch > >>>> security > >>>> modules to ServiceLoaders (which we should do anyways) and also pull > >>>> them > >>>> from plugins. > >>>> > >>>> Best, > >>>> Stephan > >>>> > >>>> > >>>> > >>>> On Thu, Apr 23, 2020 at 4:05 AM Xintong Song <[hidden email]> > >>>> wrote: > >>>> > >>>>> +1 for supporting Hadoop 3. > >>>>> > >>>>> I'm not familiar with the shading efforts, thus no comment on > >>>>> dropping the > >>>>> flink-shaded-hadoop. > >>>>> > >>>>> > >>>>> Correct me if I'm wrong. Despite currently the default Hadoop > >>>>> version for > >>>>> compiling is 2.4.1 in Flink, I think this does not mean Flink should > >>>>> support only Hadoop 2.4+. So no matter which Hadoop version we use > for > >>>>> compiling by default, we need to use reflection for the Hadoop > >>>>> features/APIs that are not supported in all versions anyway. > >>>>> > >>>>> > >>>>> There're already many such reflections in `YarnClusterDescriptor` and > >>>>> `YarnResourceManager`, and might be more in future. I'm wondering > >>>>> whether > >>>>> we should have a unified mechanism (an interface / abstract class or > >>>>> so) > >>>>> that handles all these kind of Hadoop API reflections at one place. > Not > >>>>> necessarily in the scope to this discussion though. > >>>>> > >>>>> > >>>>> Thank you~ > >>>>> > >>>>> Xintong Song > >>>>> > >>>>> > >>>>> > >>>>> On Wed, Apr 22, 2020 at 8:32 PM Chesnay Schepler <[hidden email] > > > >>>>> wrote: > >>>>> > >>>>>> 1) Likely not, as this again introduces a hard-dependency on > >>>>>> flink-shaded-hadoop. > >>>>>> 2) Indeed; this will be something the user/cloud providers have to > >>>>>> deal > >>>>>> with now. > >>>>>> 3) Yes. > >>>>>> > >>>>>> As a small note, we can still keep the hadoop-2 version of > >>>>>> flink-shaded > >>>>>> around for existing users. > >>>>>> What I suggested was to just not release hadoop-3 versions. > >>>>>> > >>>>>> On 22/04/2020 14:19, Yang Wang wrote: > >>>>>>> Thanks Robert for starting this significant discussion. > >>>>>>> > >>>>>>> Since hadoop3 has been released for long time and many companies > have > >>>>>>> already > >>>>>>> put it in production. No matter you are using flink-shaded-hadoop2 > or > >>>>>> not, > >>>>>>> currently > >>>>>>> Flink could already run in yarn3(not sure about HDFS). Since the > yarn > >>>>> api > >>>>>>> is always > >>>>>>> backward compatible. The difference is we could not benefit from > the > >>>>> new > >>>>>>> features > >>>>>>> because we are using hadoop-2.4 as compile dependency. So then we > >>>>>>> need > >>>>> to > >>>>>>> use > >>>>>>> reflector for new features(node label, tags, etc.). > >>>>>>> > >>>>>>> All in all, i am in in favour of dropping the flink-shaded-hadoop. > >>>>>>> Just > >>>>>>> have some questions. > >>>>>>> 1. Do we still support "-include-hadoop" profile? If yes, what we > >>>>>>> will > >>>>>> get > >>>>>>> in the lib dir? > >>>>>>> 2. I am not sure whether dropping the flink-shaded-hadoop will take > >>>>> some > >>>>>>> class conflicts > >>>>>>> problems. If we use "export HADOOP_CLASSPATH=`hadoop classpath`" > for > >>>>> the > >>>>>>> hadoop > >>>>>>> env setup, then many jars will be appended to the Flink client > >>>>> classpath. > >>>>>>> 3. The compile hadoop version is still 2.4.1. Right? > >>>>>>> > >>>>>>> > >>>>>>> Best, > >>>>>>> Yang > >>>>>>> > >>>>>>> > >>>>>>> Sivaprasanna <[hidden email]> 于2020年4月22日周三 > >>>>>>> 下午4:18写道: > >>>>>>> > >>>>>>>> I agree with Aljoscha. Otherwise I can see a lot of tickets > getting > >>>>>> created > >>>>>>>> saying the application is not running on YARN. > >>>>>>>> > >>>>>>>> Cheers, > >>>>>>>> Sivaprasanna > >>>>>>>> > >>>>>>>> On Wed, Apr 22, 2020 at 1:00 PM Aljoscha Krettek > >>>>>>>> <[hidden email] > >>>>>>>> wrote: > >>>>>>>> > >>>>>>>>> +1 to getting rid of flink-shaded-hadoop. But we need to > >>>>>>>>> document how > >>>>>>>>> people can now get a Flink dist that works with Hadoop. > Currently, > >>>>> when > >>>>>>>>> you download the single shaded jar you immediately get support > for > >>>>>>>>> submitting to YARN via bin/flink run. > >>>>>>>>> > >>>>>>>>> Aljoscha > >>>>>>>>> > >>>>>>>>> > >>>>>>>>> On 22.04.20 09:08, Till Rohrmann wrote: > >>>>>>>>>> Hi Robert, > >>>>>>>>>> > >>>>>>>>>> I think it would be a helpful simplification of Flink's build > >>>>>>>>>> setup > >>>>> if > >>>>>>>> we > >>>>>>>>>> can get rid of flink-shaded-hadoop. Moreover relying only on the > >>>>>>>> vanilla > >>>>>>>>>> Hadoop dependencies for the modules which interact with > >>>>>>>>>> Hadoop/Yarn > >>>>>>>>> sounds > >>>>>>>>>> like a good idea to me. > >>>>>>>>>> > >>>>>>>>>> Adding support for Hadoop 3 would also be nice. I'm not sure, > >>>>> though, > >>>>>>>> how > >>>>>>>>>> Hadoop's API's have changed between 2 and 3. It might be > necessary > >>>>> to > >>>>>>>>>> introduce some bridges in order to make it work. > >>>>>>>>>> > >>>>>>>>>> Cheers, > >>>>>>>>>> Till > >>>>>>>>>> > >>>>>>>>>> On Tue, Apr 21, 2020 at 4:37 PM Robert Metzger > >>>>>>>>>> <[hidden email] > >>>>>>>>> wrote: > >>>>>>>>>>> Hi all, > >>>>>>>>>>> > >>>>>>>>>>> for the upcoming 1.11 release, I started looking into adding > >>>>> support > >>>>>>>> for > >>>>>>>>>>> Hadoop 3[1] for Flink. I have explored a little bit already > into > >>>>>>>> adding > >>>>>>>>> a > >>>>>>>>>>> shaded hadoop 3 into “flink-shaded”, and some mechanisms for > >>>>>> switching > >>>>>>>>>>> between Hadoop 2 and 3 dependencies in the Flink build. > >>>>>>>>>>> > >>>>>>>>>>> However, Chesnay made me aware that we could also go a > different > >>>>>>>> route: > >>>>>>>>> We > >>>>>>>>>>> let Flink depend on vanilla Hadoop dependencies and stop > >>>>>>>>>>> providing > >>>>>>>>> shaded > >>>>>>>>>>> fat jars for Hadoop through “flink-shaded”. > >>>>>>>>>>> > >>>>>>>>>>> Why? > >>>>>>>>>>> - Maintaining properly shaded Hadoop fat jars is a lot of work > >>>>>>>>>>> (we > >>>>>>>> have > >>>>>>>>>>> insufficient test coverage for all kinds of Hadoop features) > >>>>>>>>>>> - For Hadoop 2, there are already some known and unresolved > >>>>>>>>>>> issues > >>>>>>>> with > >>>>>>>>> our > >>>>>>>>>>> shaded jars that we didn’t manage to fix > >>>>>>>>>>> > >>>>>>>>>>> Users will have to use Flink with Hadoop by relying on vanilla > or > >>>>>>>>>>> vendor-provided Hadoop dependencies. > >>>>>>>>>>> > >>>>>>>>>>> What do you think? > >>>>>>>>>>> > >>>>>>>>>>> Best, > >>>>>>>>>>> Robert > >>>>>>>>>>> > >>>>>>>>>>> [1] https://issues.apache.org/jira/browse/FLINK-11086 > >>>>>>>>>>> > >>> > >> > > |
Hi Chesnay, Hi Robert,
I have a bit of a naive question. I assume the reason for introducing flink-shaded-hadoop were dependency conflicts between Hadoop, Flink and/or user code. When we drop it now is it because a) it was not worth it (value provided did not justify maintenance overhead and issues introduced) b) we don't think it is a problem anymore c) prioritizes have shifted and it *now *not worth it anymore d) something else Cheers, Konstantin On Sun, Apr 26, 2020 at 10:25 PM Stephan Ewen <[hidden email]> wrote: > Indeed, that would be the assumption, that Hadoop does not expose its > transitive libraries on its public API surface. > > From vague memory, I think that pretty much true so far. I only remember > Kinesis and Calcite as counter examples, who exposed Guava classes as part > of the public API. > But that is definitely the "weak spot" of this approach. Plus, as with all > custom class loaders, the fact that the Thread Context Class Loader does > not really work well any more. > > On Thu, Apr 23, 2020 at 11:50 AM Chesnay Schepler <[hidden email]> > wrote: > > > This would only work so long as all Hadoop APIs do not directly expose > > any transitive non-hadoop dependency. > > Otherwise the user code classloader might search for this transitive > > dependency in lib instead of the hadoop classpath (and possibly not find > > it). > > > > On 23/04/2020 11:34, Stephan Ewen wrote: > > > True, connectors built on Hadoop make this a bit more complex. That is > > also > > > the reason why Hadoop is on the "parent first" patterns. > > > > > > Maybe this is a bit of a wild thought, but what would happen if we had > a > > > "first class" notion of a Hadoop Classloader in the system, and the > user > > > code classloader would explicitly fall back to that one whenever a > class > > > whose name starts with "org.apache.hadoop" is not found? We could also > > > generalize this by associating plugin loaders with class name prefixes. > > > > > > Then it would try to load from the user code jar, and if the class was > > not > > > found, load it from the hadoop classpath. > > > > > > On Thu, Apr 23, 2020 at 10:56 AM Chesnay Schepler <[hidden email]> > > > wrote: > > > > > >> although, if you can load the HADOOP_CLASSPATH as a plugin, then you > can > > >> also load it in the user-code classloader. > > >> > > >> On 23/04/2020 10:50, Chesnay Schepler wrote: > > >>> @Stephan I'm not aware of anyone having tried that; possibly since we > > >>> have various connectors that require hadoop (hadoop-compat, hive, > > >>> orc/parquet/hbase, hadoop inputformats). This would require > connectors > > >>> to be loaded as plugins (or having access to the plugin classloader) > > >>> to be feasible. > > >>> > > >>> On 23/04/2020 09:59, Stephan Ewen wrote: > > >>>> Hi all! > > >>>> > > >>>> +1 for the simplification of dropping hadoop-shaded > > >>>> > > >>>> > > >>>> Have we ever investigated how much work it would be to load the > > >>>> HADOOP_CLASSPATH through the plugin loader? Then Hadoop's crazy > > >>>> dependency > > >>>> footprint would not spoil the main classpath. > > >>>> > > >>>> - HDFS might be very simple, because file systems are already > > >>>> Plugin aware > > >>>> - Yarn would need some extra work. In essence, we would need to > > >>>> discover > > >>>> executors also through plugins > > >>>> - Kerberos is the other remaining bit. We would need to switch > > >>>> security > > >>>> modules to ServiceLoaders (which we should do anyways) and also pull > > >>>> them > > >>>> from plugins. > > >>>> > > >>>> Best, > > >>>> Stephan > > >>>> > > >>>> > > >>>> > > >>>> On Thu, Apr 23, 2020 at 4:05 AM Xintong Song <[hidden email] > > > > >>>> wrote: > > >>>> > > >>>>> +1 for supporting Hadoop 3. > > >>>>> > > >>>>> I'm not familiar with the shading efforts, thus no comment on > > >>>>> dropping the > > >>>>> flink-shaded-hadoop. > > >>>>> > > >>>>> > > >>>>> Correct me if I'm wrong. Despite currently the default Hadoop > > >>>>> version for > > >>>>> compiling is 2.4.1 in Flink, I think this does not mean Flink > should > > >>>>> support only Hadoop 2.4+. So no matter which Hadoop version we use > > for > > >>>>> compiling by default, we need to use reflection for the Hadoop > > >>>>> features/APIs that are not supported in all versions anyway. > > >>>>> > > >>>>> > > >>>>> There're already many such reflections in `YarnClusterDescriptor` > and > > >>>>> `YarnResourceManager`, and might be more in future. I'm wondering > > >>>>> whether > > >>>>> we should have a unified mechanism (an interface / abstract class > or > > >>>>> so) > > >>>>> that handles all these kind of Hadoop API reflections at one place. > > Not > > >>>>> necessarily in the scope to this discussion though. > > >>>>> > > >>>>> > > >>>>> Thank you~ > > >>>>> > > >>>>> Xintong Song > > >>>>> > > >>>>> > > >>>>> > > >>>>> On Wed, Apr 22, 2020 at 8:32 PM Chesnay Schepler < > [hidden email] > > > > > >>>>> wrote: > > >>>>> > > >>>>>> 1) Likely not, as this again introduces a hard-dependency on > > >>>>>> flink-shaded-hadoop. > > >>>>>> 2) Indeed; this will be something the user/cloud providers have to > > >>>>>> deal > > >>>>>> with now. > > >>>>>> 3) Yes. > > >>>>>> > > >>>>>> As a small note, we can still keep the hadoop-2 version of > > >>>>>> flink-shaded > > >>>>>> around for existing users. > > >>>>>> What I suggested was to just not release hadoop-3 versions. > > >>>>>> > > >>>>>> On 22/04/2020 14:19, Yang Wang wrote: > > >>>>>>> Thanks Robert for starting this significant discussion. > > >>>>>>> > > >>>>>>> Since hadoop3 has been released for long time and many companies > > have > > >>>>>>> already > > >>>>>>> put it in production. No matter you are using > flink-shaded-hadoop2 > > or > > >>>>>> not, > > >>>>>>> currently > > >>>>>>> Flink could already run in yarn3(not sure about HDFS). Since the > > yarn > > >>>>> api > > >>>>>>> is always > > >>>>>>> backward compatible. The difference is we could not benefit from > > the > > >>>>> new > > >>>>>>> features > > >>>>>>> because we are using hadoop-2.4 as compile dependency. So then we > > >>>>>>> need > > >>>>> to > > >>>>>>> use > > >>>>>>> reflector for new features(node label, tags, etc.). > > >>>>>>> > > >>>>>>> All in all, i am in in favour of dropping the > flink-shaded-hadoop. > > >>>>>>> Just > > >>>>>>> have some questions. > > >>>>>>> 1. Do we still support "-include-hadoop" profile? If yes, what we > > >>>>>>> will > > >>>>>> get > > >>>>>>> in the lib dir? > > >>>>>>> 2. I am not sure whether dropping the flink-shaded-hadoop will > take > > >>>>> some > > >>>>>>> class conflicts > > >>>>>>> problems. If we use "export HADOOP_CLASSPATH=`hadoop classpath`" > > for > > >>>>> the > > >>>>>>> hadoop > > >>>>>>> env setup, then many jars will be appended to the Flink client > > >>>>> classpath. > > >>>>>>> 3. The compile hadoop version is still 2.4.1. Right? > > >>>>>>> > > >>>>>>> > > >>>>>>> Best, > > >>>>>>> Yang > > >>>>>>> > > >>>>>>> > > >>>>>>> Sivaprasanna <[hidden email]> 于2020年4月22日周三 > > >>>>>>> 下午4:18写道: > > >>>>>>> > > >>>>>>>> I agree with Aljoscha. Otherwise I can see a lot of tickets > > getting > > >>>>>> created > > >>>>>>>> saying the application is not running on YARN. > > >>>>>>>> > > >>>>>>>> Cheers, > > >>>>>>>> Sivaprasanna > > >>>>>>>> > > >>>>>>>> On Wed, Apr 22, 2020 at 1:00 PM Aljoscha Krettek > > >>>>>>>> <[hidden email] > > >>>>>>>> wrote: > > >>>>>>>> > > >>>>>>>>> +1 to getting rid of flink-shaded-hadoop. But we need to > > >>>>>>>>> document how > > >>>>>>>>> people can now get a Flink dist that works with Hadoop. > > Currently, > > >>>>> when > > >>>>>>>>> you download the single shaded jar you immediately get support > > for > > >>>>>>>>> submitting to YARN via bin/flink run. > > >>>>>>>>> > > >>>>>>>>> Aljoscha > > >>>>>>>>> > > >>>>>>>>> > > >>>>>>>>> On 22.04.20 09:08, Till Rohrmann wrote: > > >>>>>>>>>> Hi Robert, > > >>>>>>>>>> > > >>>>>>>>>> I think it would be a helpful simplification of Flink's build > > >>>>>>>>>> setup > > >>>>> if > > >>>>>>>> we > > >>>>>>>>>> can get rid of flink-shaded-hadoop. Moreover relying only on > the > > >>>>>>>> vanilla > > >>>>>>>>>> Hadoop dependencies for the modules which interact with > > >>>>>>>>>> Hadoop/Yarn > > >>>>>>>>> sounds > > >>>>>>>>>> like a good idea to me. > > >>>>>>>>>> > > >>>>>>>>>> Adding support for Hadoop 3 would also be nice. I'm not sure, > > >>>>> though, > > >>>>>>>> how > > >>>>>>>>>> Hadoop's API's have changed between 2 and 3. It might be > > necessary > > >>>>> to > > >>>>>>>>>> introduce some bridges in order to make it work. > > >>>>>>>>>> > > >>>>>>>>>> Cheers, > > >>>>>>>>>> Till > > >>>>>>>>>> > > >>>>>>>>>> On Tue, Apr 21, 2020 at 4:37 PM Robert Metzger > > >>>>>>>>>> <[hidden email] > > >>>>>>>>> wrote: > > >>>>>>>>>>> Hi all, > > >>>>>>>>>>> > > >>>>>>>>>>> for the upcoming 1.11 release, I started looking into adding > > >>>>> support > > >>>>>>>> for > > >>>>>>>>>>> Hadoop 3[1] for Flink. I have explored a little bit already > > into > > >>>>>>>> adding > > >>>>>>>>> a > > >>>>>>>>>>> shaded hadoop 3 into “flink-shaded”, and some mechanisms for > > >>>>>> switching > > >>>>>>>>>>> between Hadoop 2 and 3 dependencies in the Flink build. > > >>>>>>>>>>> > > >>>>>>>>>>> However, Chesnay made me aware that we could also go a > > different > > >>>>>>>> route: > > >>>>>>>>> We > > >>>>>>>>>>> let Flink depend on vanilla Hadoop dependencies and stop > > >>>>>>>>>>> providing > > >>>>>>>>> shaded > > >>>>>>>>>>> fat jars for Hadoop through “flink-shaded”. > > >>>>>>>>>>> > > >>>>>>>>>>> Why? > > >>>>>>>>>>> - Maintaining properly shaded Hadoop fat jars is a lot of > work > > >>>>>>>>>>> (we > > >>>>>>>> have > > >>>>>>>>>>> insufficient test coverage for all kinds of Hadoop features) > > >>>>>>>>>>> - For Hadoop 2, there are already some known and unresolved > > >>>>>>>>>>> issues > > >>>>>>>> with > > >>>>>>>>> our > > >>>>>>>>>>> shaded jars that we didn’t manage to fix > > >>>>>>>>>>> > > >>>>>>>>>>> Users will have to use Flink with Hadoop by relying on > vanilla > > or > > >>>>>>>>>>> vendor-provided Hadoop dependencies. > > >>>>>>>>>>> > > >>>>>>>>>>> What do you think? > > >>>>>>>>>>> > > >>>>>>>>>>> Best, > > >>>>>>>>>>> Robert > > >>>>>>>>>>> > > >>>>>>>>>>> [1] https://issues.apache.org/jira/browse/FLINK-11086 > > >>>>>>>>>>> > > >>> > > >> > > > > > -- Konstantin Knauf https://twitter.com/snntrable https://github.com/knaufk |
Hey Robert and others,
overall +1 to support Hadoop 3. It would be a great to unblock Flink support in EMR 6.0 as noted in the linked FLINK ticket. The arguments raised against flink-shaded-hadoop make sense to me. I have a few general questions still: 1) Will the flink-shaded-hadoop module (in apache/flink-shaded) be fully dropped after this change? Or do you plan to keep it (allowing users to build their own shaded Hadoop if needed)? 2) I find Stephan's ideas pretty interesting. Will there be an official follow-up to investigate those? 3) What will we tell users that run into class loading conflicts after this change? What are actually the "expected" conflicts we might see? – Ufuk PS: Robert opened a draft PR here: https://github.com/apache/flink/pull/11983 On Sun, May 3, 2020 at 12:02 PM Konstantin Knauf <[hidden email]> wrote: > Hi Chesnay, Hi Robert, > > I have a bit of a naive question. I assume the reason for introducing > flink-shaded-hadoop were dependency conflicts between Hadoop, Flink and/or > user code. When we drop it now is it because > > a) it was not worth it (value provided did not justify maintenance overhead > and issues introduced) > b) we don't think it is a problem anymore > c) prioritizes have shifted and it *now *not worth it anymore > d) something else > > Cheers, > > Konstantin > > On Sun, Apr 26, 2020 at 10:25 PM Stephan Ewen <[hidden email]> wrote: > > > Indeed, that would be the assumption, that Hadoop does not expose its > > transitive libraries on its public API surface. > > > > From vague memory, I think that pretty much true so far. I only remember > > Kinesis and Calcite as counter examples, who exposed Guava classes as > part > > of the public API. > > But that is definitely the "weak spot" of this approach. Plus, as with > all > > custom class loaders, the fact that the Thread Context Class Loader does > > not really work well any more. > > > > On Thu, Apr 23, 2020 at 11:50 AM Chesnay Schepler <[hidden email]> > > wrote: > > > > > This would only work so long as all Hadoop APIs do not directly expose > > > any transitive non-hadoop dependency. > > > Otherwise the user code classloader might search for this transitive > > > dependency in lib instead of the hadoop classpath (and possibly not > find > > > it). > > > > > > On 23/04/2020 11:34, Stephan Ewen wrote: > > > > True, connectors built on Hadoop make this a bit more complex. That > is > > > also > > > > the reason why Hadoop is on the "parent first" patterns. > > > > > > > > Maybe this is a bit of a wild thought, but what would happen if we > had > > a > > > > "first class" notion of a Hadoop Classloader in the system, and the > > user > > > > code classloader would explicitly fall back to that one whenever a > > class > > > > whose name starts with "org.apache.hadoop" is not found? We could > also > > > > generalize this by associating plugin loaders with class name > prefixes. > > > > > > > > Then it would try to load from the user code jar, and if the class > was > > > not > > > > found, load it from the hadoop classpath. > > > > > > > > On Thu, Apr 23, 2020 at 10:56 AM Chesnay Schepler < > [hidden email]> > > > > wrote: > > > > > > > >> although, if you can load the HADOOP_CLASSPATH as a plugin, then you > > can > > > >> also load it in the user-code classloader. > > > >> > > > >> On 23/04/2020 10:50, Chesnay Schepler wrote: > > > >>> @Stephan I'm not aware of anyone having tried that; possibly since > we > > > >>> have various connectors that require hadoop (hadoop-compat, hive, > > > >>> orc/parquet/hbase, hadoop inputformats). This would require > > connectors > > > >>> to be loaded as plugins (or having access to the plugin > classloader) > > > >>> to be feasible. > > > >>> > > > >>> On 23/04/2020 09:59, Stephan Ewen wrote: > > > >>>> Hi all! > > > >>>> > > > >>>> +1 for the simplification of dropping hadoop-shaded > > > >>>> > > > >>>> > > > >>>> Have we ever investigated how much work it would be to load the > > > >>>> HADOOP_CLASSPATH through the plugin loader? Then Hadoop's crazy > > > >>>> dependency > > > >>>> footprint would not spoil the main classpath. > > > >>>> > > > >>>> - HDFS might be very simple, because file systems are already > > > >>>> Plugin aware > > > >>>> - Yarn would need some extra work. In essence, we would need > to > > > >>>> discover > > > >>>> executors also through plugins > > > >>>> - Kerberos is the other remaining bit. We would need to switch > > > >>>> security > > > >>>> modules to ServiceLoaders (which we should do anyways) and also > pull > > > >>>> them > > > >>>> from plugins. > > > >>>> > > > >>>> Best, > > > >>>> Stephan > > > >>>> > > > >>>> > > > >>>> > > > >>>> On Thu, Apr 23, 2020 at 4:05 AM Xintong Song < > [hidden email] > > > > > > >>>> wrote: > > > >>>> > > > >>>>> +1 for supporting Hadoop 3. > > > >>>>> > > > >>>>> I'm not familiar with the shading efforts, thus no comment on > > > >>>>> dropping the > > > >>>>> flink-shaded-hadoop. > > > >>>>> > > > >>>>> > > > >>>>> Correct me if I'm wrong. Despite currently the default Hadoop > > > >>>>> version for > > > >>>>> compiling is 2.4.1 in Flink, I think this does not mean Flink > > should > > > >>>>> support only Hadoop 2.4+. So no matter which Hadoop version we > use > > > for > > > >>>>> compiling by default, we need to use reflection for the Hadoop > > > >>>>> features/APIs that are not supported in all versions anyway. > > > >>>>> > > > >>>>> > > > >>>>> There're already many such reflections in `YarnClusterDescriptor` > > and > > > >>>>> `YarnResourceManager`, and might be more in future. I'm wondering > > > >>>>> whether > > > >>>>> we should have a unified mechanism (an interface / abstract class > > or > > > >>>>> so) > > > >>>>> that handles all these kind of Hadoop API reflections at one > place. > > > Not > > > >>>>> necessarily in the scope to this discussion though. > > > >>>>> > > > >>>>> > > > >>>>> Thank you~ > > > >>>>> > > > >>>>> Xintong Song > > > >>>>> > > > >>>>> > > > >>>>> > > > >>>>> On Wed, Apr 22, 2020 at 8:32 PM Chesnay Schepler < > > [hidden email] > > > > > > > >>>>> wrote: > > > >>>>> > > > >>>>>> 1) Likely not, as this again introduces a hard-dependency on > > > >>>>>> flink-shaded-hadoop. > > > >>>>>> 2) Indeed; this will be something the user/cloud providers have > to > > > >>>>>> deal > > > >>>>>> with now. > > > >>>>>> 3) Yes. > > > >>>>>> > > > >>>>>> As a small note, we can still keep the hadoop-2 version of > > > >>>>>> flink-shaded > > > >>>>>> around for existing users. > > > >>>>>> What I suggested was to just not release hadoop-3 versions. > > > >>>>>> > > > >>>>>> On 22/04/2020 14:19, Yang Wang wrote: > > > >>>>>>> Thanks Robert for starting this significant discussion. > > > >>>>>>> > > > >>>>>>> Since hadoop3 has been released for long time and many > companies > > > have > > > >>>>>>> already > > > >>>>>>> put it in production. No matter you are using > > flink-shaded-hadoop2 > > > or > > > >>>>>> not, > > > >>>>>>> currently > > > >>>>>>> Flink could already run in yarn3(not sure about HDFS). Since > the > > > yarn > > > >>>>> api > > > >>>>>>> is always > > > >>>>>>> backward compatible. The difference is we could not benefit > from > > > the > > > >>>>> new > > > >>>>>>> features > > > >>>>>>> because we are using hadoop-2.4 as compile dependency. So then > we > > > >>>>>>> need > > > >>>>> to > > > >>>>>>> use > > > >>>>>>> reflector for new features(node label, tags, etc.). > > > >>>>>>> > > > >>>>>>> All in all, i am in in favour of dropping the > > flink-shaded-hadoop. > > > >>>>>>> Just > > > >>>>>>> have some questions. > > > >>>>>>> 1. Do we still support "-include-hadoop" profile? If yes, what > we > > > >>>>>>> will > > > >>>>>> get > > > >>>>>>> in the lib dir? > > > >>>>>>> 2. I am not sure whether dropping the flink-shaded-hadoop will > > take > > > >>>>> some > > > >>>>>>> class conflicts > > > >>>>>>> problems. If we use "export HADOOP_CLASSPATH=`hadoop > classpath`" > > > for > > > >>>>> the > > > >>>>>>> hadoop > > > >>>>>>> env setup, then many jars will be appended to the Flink client > > > >>>>> classpath. > > > >>>>>>> 3. The compile hadoop version is still 2.4.1. Right? > > > >>>>>>> > > > >>>>>>> > > > >>>>>>> Best, > > > >>>>>>> Yang > > > >>>>>>> > > > >>>>>>> > > > >>>>>>> Sivaprasanna <[hidden email]> 于2020年4月22日周三 > > > >>>>>>> 下午4:18写道: > > > >>>>>>> > > > >>>>>>>> I agree with Aljoscha. Otherwise I can see a lot of tickets > > > getting > > > >>>>>> created > > > >>>>>>>> saying the application is not running on YARN. > > > >>>>>>>> > > > >>>>>>>> Cheers, > > > >>>>>>>> Sivaprasanna > > > >>>>>>>> > > > >>>>>>>> On Wed, Apr 22, 2020 at 1:00 PM Aljoscha Krettek > > > >>>>>>>> <[hidden email] > > > >>>>>>>> wrote: > > > >>>>>>>> > > > >>>>>>>>> +1 to getting rid of flink-shaded-hadoop. But we need to > > > >>>>>>>>> document how > > > >>>>>>>>> people can now get a Flink dist that works with Hadoop. > > > Currently, > > > >>>>> when > > > >>>>>>>>> you download the single shaded jar you immediately get > support > > > for > > > >>>>>>>>> submitting to YARN via bin/flink run. > > > >>>>>>>>> > > > >>>>>>>>> Aljoscha > > > >>>>>>>>> > > > >>>>>>>>> > > > >>>>>>>>> On 22.04.20 09:08, Till Rohrmann wrote: > > > >>>>>>>>>> Hi Robert, > > > >>>>>>>>>> > > > >>>>>>>>>> I think it would be a helpful simplification of Flink's > build > > > >>>>>>>>>> setup > > > >>>>> if > > > >>>>>>>> we > > > >>>>>>>>>> can get rid of flink-shaded-hadoop. Moreover relying only on > > the > > > >>>>>>>> vanilla > > > >>>>>>>>>> Hadoop dependencies for the modules which interact with > > > >>>>>>>>>> Hadoop/Yarn > > > >>>>>>>>> sounds > > > >>>>>>>>>> like a good idea to me. > > > >>>>>>>>>> > > > >>>>>>>>>> Adding support for Hadoop 3 would also be nice. I'm not > sure, > > > >>>>> though, > > > >>>>>>>> how > > > >>>>>>>>>> Hadoop's API's have changed between 2 and 3. It might be > > > necessary > > > >>>>> to > > > >>>>>>>>>> introduce some bridges in order to make it work. > > > >>>>>>>>>> > > > >>>>>>>>>> Cheers, > > > >>>>>>>>>> Till > > > >>>>>>>>>> > > > >>>>>>>>>> On Tue, Apr 21, 2020 at 4:37 PM Robert Metzger > > > >>>>>>>>>> <[hidden email] > > > >>>>>>>>> wrote: > > > >>>>>>>>>>> Hi all, > > > >>>>>>>>>>> > > > >>>>>>>>>>> for the upcoming 1.11 release, I started looking into > adding > > > >>>>> support > > > >>>>>>>> for > > > >>>>>>>>>>> Hadoop 3[1] for Flink. I have explored a little bit already > > > into > > > >>>>>>>> adding > > > >>>>>>>>> a > > > >>>>>>>>>>> shaded hadoop 3 into “flink-shaded”, and some mechanisms > for > > > >>>>>> switching > > > >>>>>>>>>>> between Hadoop 2 and 3 dependencies in the Flink build. > > > >>>>>>>>>>> > > > >>>>>>>>>>> However, Chesnay made me aware that we could also go a > > > different > > > >>>>>>>> route: > > > >>>>>>>>> We > > > >>>>>>>>>>> let Flink depend on vanilla Hadoop dependencies and stop > > > >>>>>>>>>>> providing > > > >>>>>>>>> shaded > > > >>>>>>>>>>> fat jars for Hadoop through “flink-shaded”. > > > >>>>>>>>>>> > > > >>>>>>>>>>> Why? > > > >>>>>>>>>>> - Maintaining properly shaded Hadoop fat jars is a lot of > > work > > > >>>>>>>>>>> (we > > > >>>>>>>> have > > > >>>>>>>>>>> insufficient test coverage for all kinds of Hadoop > features) > > > >>>>>>>>>>> - For Hadoop 2, there are already some known and unresolved > > > >>>>>>>>>>> issues > > > >>>>>>>> with > > > >>>>>>>>> our > > > >>>>>>>>>>> shaded jars that we didn’t manage to fix > > > >>>>>>>>>>> > > > >>>>>>>>>>> Users will have to use Flink with Hadoop by relying on > > vanilla > > > or > > > >>>>>>>>>>> vendor-provided Hadoop dependencies. > > > >>>>>>>>>>> > > > >>>>>>>>>>> What do you think? > > > >>>>>>>>>>> > > > >>>>>>>>>>> Best, > > > >>>>>>>>>>> Robert > > > >>>>>>>>>>> > > > >>>>>>>>>>> [1] https://issues.apache.org/jira/browse/FLINK-11086 > > > >>>>>>>>>>> > > > >>> > > > >> > > > > > > > > > > > -- > > Konstantin Knauf > > https://twitter.com/snntrable > > https://github.com/knaufk > |
@Konstantin
It is part a) since it is not used by all hadoop users (as they may be using their existing hadoop infrastructure or one provided by the cloud service), but I'd say it's mostly for maintenance reasons. The reality is that we cannot truly maintain flink-shaded-hadoop. There are too many different versions, with too many features, that are too old, to make any change with any form of guarantee that things still work afterwards. This will only get worse when adding another major version. I'm currently actively avoiding making any changes because I have no way of properly testing the correctness. @Ufuk 1) Initially I thought about keeping flink-shaded-hadoop around, but there actually isn't really a reason to do so. Users can build their own version in any case with the existing source releases. 3) This is where things get finicky. At the end of the day the user will have to deal with it; be it by creating custom Hadoop/Flink distributions, updating to later versions, raising hell in the Hadoop/Flink JIRAs, ... We can maybe turn affected components into a plugin. We should still make sure that Flink works with Hadoop, just not that it works against all versions in the last 5 years. As for "expected" conflicts; like, the thing is that who knows how things look like in 2 years. On 04/05/2020 12:24, Ufuk Celebi wrote: > Hey Robert and others, > > overall +1 to support Hadoop 3. It would be a great to unblock Flink > support in EMR 6.0 as noted in the linked FLINK ticket. > > The arguments raised against flink-shaded-hadoop make sense to me. I have a > few general questions still: > > 1) Will the flink-shaded-hadoop module (in apache/flink-shaded) be fully > dropped after this change? Or do you plan to keep it (allowing users to > build their own shaded Hadoop if needed)? > > 2) I find Stephan's ideas pretty interesting. Will there be an official > follow-up to investigate those? > > 3) What will we tell users that run into class loading conflicts after this > change? What are actually the "expected" conflicts we might see? > > – Ufuk > > PS: Robert opened a draft PR here: > https://github.com/apache/flink/pull/11983 > > > On Sun, May 3, 2020 at 12:02 PM Konstantin Knauf <[hidden email]> wrote: > >> Hi Chesnay, Hi Robert, >> >> I have a bit of a naive question. I assume the reason for introducing >> flink-shaded-hadoop were dependency conflicts between Hadoop, Flink and/or >> user code. When we drop it now is it because >> >> a) it was not worth it (value provided did not justify maintenance overhead >> and issues introduced) >> b) we don't think it is a problem anymore >> c) prioritizes have shifted and it *now *not worth it anymore >> d) something else >> >> Cheers, >> >> Konstantin >> >> On Sun, Apr 26, 2020 at 10:25 PM Stephan Ewen <[hidden email]> wrote: >> >>> Indeed, that would be the assumption, that Hadoop does not expose its >>> transitive libraries on its public API surface. >>> >>> From vague memory, I think that pretty much true so far. I only remember >>> Kinesis and Calcite as counter examples, who exposed Guava classes as >> part >>> of the public API. >>> But that is definitely the "weak spot" of this approach. Plus, as with >> all >>> custom class loaders, the fact that the Thread Context Class Loader does >>> not really work well any more. >>> >>> On Thu, Apr 23, 2020 at 11:50 AM Chesnay Schepler <[hidden email]> >>> wrote: >>> >>>> This would only work so long as all Hadoop APIs do not directly expose >>>> any transitive non-hadoop dependency. >>>> Otherwise the user code classloader might search for this transitive >>>> dependency in lib instead of the hadoop classpath (and possibly not >> find >>>> it). >>>> >>>> On 23/04/2020 11:34, Stephan Ewen wrote: >>>>> True, connectors built on Hadoop make this a bit more complex. That >> is >>>> also >>>>> the reason why Hadoop is on the "parent first" patterns. >>>>> >>>>> Maybe this is a bit of a wild thought, but what would happen if we >> had >>> a >>>>> "first class" notion of a Hadoop Classloader in the system, and the >>> user >>>>> code classloader would explicitly fall back to that one whenever a >>> class >>>>> whose name starts with "org.apache.hadoop" is not found? We could >> also >>>>> generalize this by associating plugin loaders with class name >> prefixes. >>>>> Then it would try to load from the user code jar, and if the class >> was >>>> not >>>>> found, load it from the hadoop classpath. >>>>> >>>>> On Thu, Apr 23, 2020 at 10:56 AM Chesnay Schepler < >> [hidden email]> >>>>> wrote: >>>>> >>>>>> although, if you can load the HADOOP_CLASSPATH as a plugin, then you >>> can >>>>>> also load it in the user-code classloader. >>>>>> >>>>>> On 23/04/2020 10:50, Chesnay Schepler wrote: >>>>>>> @Stephan I'm not aware of anyone having tried that; possibly since >> we >>>>>>> have various connectors that require hadoop (hadoop-compat, hive, >>>>>>> orc/parquet/hbase, hadoop inputformats). This would require >>> connectors >>>>>>> to be loaded as plugins (or having access to the plugin >> classloader) >>>>>>> to be feasible. >>>>>>> >>>>>>> On 23/04/2020 09:59, Stephan Ewen wrote: >>>>>>>> Hi all! >>>>>>>> >>>>>>>> +1 for the simplification of dropping hadoop-shaded >>>>>>>> >>>>>>>> >>>>>>>> Have we ever investigated how much work it would be to load the >>>>>>>> HADOOP_CLASSPATH through the plugin loader? Then Hadoop's crazy >>>>>>>> dependency >>>>>>>> footprint would not spoil the main classpath. >>>>>>>> >>>>>>>> - HDFS might be very simple, because file systems are already >>>>>>>> Plugin aware >>>>>>>> - Yarn would need some extra work. In essence, we would need >> to >>>>>>>> discover >>>>>>>> executors also through plugins >>>>>>>> - Kerberos is the other remaining bit. We would need to switch >>>>>>>> security >>>>>>>> modules to ServiceLoaders (which we should do anyways) and also >> pull >>>>>>>> them >>>>>>>> from plugins. >>>>>>>> >>>>>>>> Best, >>>>>>>> Stephan >>>>>>>> >>>>>>>> >>>>>>>> >>>>>>>> On Thu, Apr 23, 2020 at 4:05 AM Xintong Song < >> [hidden email] >>>>>>>> wrote: >>>>>>>> >>>>>>>>> +1 for supporting Hadoop 3. >>>>>>>>> >>>>>>>>> I'm not familiar with the shading efforts, thus no comment on >>>>>>>>> dropping the >>>>>>>>> flink-shaded-hadoop. >>>>>>>>> >>>>>>>>> >>>>>>>>> Correct me if I'm wrong. Despite currently the default Hadoop >>>>>>>>> version for >>>>>>>>> compiling is 2.4.1 in Flink, I think this does not mean Flink >>> should >>>>>>>>> support only Hadoop 2.4+. So no matter which Hadoop version we >> use >>>> for >>>>>>>>> compiling by default, we need to use reflection for the Hadoop >>>>>>>>> features/APIs that are not supported in all versions anyway. >>>>>>>>> >>>>>>>>> >>>>>>>>> There're already many such reflections in `YarnClusterDescriptor` >>> and >>>>>>>>> `YarnResourceManager`, and might be more in future. I'm wondering >>>>>>>>> whether >>>>>>>>> we should have a unified mechanism (an interface / abstract class >>> or >>>>>>>>> so) >>>>>>>>> that handles all these kind of Hadoop API reflections at one >> place. >>>> Not >>>>>>>>> necessarily in the scope to this discussion though. >>>>>>>>> >>>>>>>>> >>>>>>>>> Thank you~ >>>>>>>>> >>>>>>>>> Xintong Song >>>>>>>>> >>>>>>>>> >>>>>>>>> >>>>>>>>> On Wed, Apr 22, 2020 at 8:32 PM Chesnay Schepler < >>> [hidden email] >>>>>>>>> wrote: >>>>>>>>> >>>>>>>>>> 1) Likely not, as this again introduces a hard-dependency on >>>>>>>>>> flink-shaded-hadoop. >>>>>>>>>> 2) Indeed; this will be something the user/cloud providers have >> to >>>>>>>>>> deal >>>>>>>>>> with now. >>>>>>>>>> 3) Yes. >>>>>>>>>> >>>>>>>>>> As a small note, we can still keep the hadoop-2 version of >>>>>>>>>> flink-shaded >>>>>>>>>> around for existing users. >>>>>>>>>> What I suggested was to just not release hadoop-3 versions. >>>>>>>>>> >>>>>>>>>> On 22/04/2020 14:19, Yang Wang wrote: >>>>>>>>>>> Thanks Robert for starting this significant discussion. >>>>>>>>>>> >>>>>>>>>>> Since hadoop3 has been released for long time and many >> companies >>>> have >>>>>>>>>>> already >>>>>>>>>>> put it in production. No matter you are using >>> flink-shaded-hadoop2 >>>> or >>>>>>>>>> not, >>>>>>>>>>> currently >>>>>>>>>>> Flink could already run in yarn3(not sure about HDFS). Since >> the >>>> yarn >>>>>>>>> api >>>>>>>>>>> is always >>>>>>>>>>> backward compatible. The difference is we could not benefit >> from >>>> the >>>>>>>>> new >>>>>>>>>>> features >>>>>>>>>>> because we are using hadoop-2.4 as compile dependency. So then >> we >>>>>>>>>>> need >>>>>>>>> to >>>>>>>>>>> use >>>>>>>>>>> reflector for new features(node label, tags, etc.). >>>>>>>>>>> >>>>>>>>>>> All in all, i am in in favour of dropping the >>> flink-shaded-hadoop. >>>>>>>>>>> Just >>>>>>>>>>> have some questions. >>>>>>>>>>> 1. Do we still support "-include-hadoop" profile? If yes, what >> we >>>>>>>>>>> will >>>>>>>>>> get >>>>>>>>>>> in the lib dir? >>>>>>>>>>> 2. I am not sure whether dropping the flink-shaded-hadoop will >>> take >>>>>>>>> some >>>>>>>>>>> class conflicts >>>>>>>>>>> problems. If we use "export HADOOP_CLASSPATH=`hadoop >> classpath`" >>>> for >>>>>>>>> the >>>>>>>>>>> hadoop >>>>>>>>>>> env setup, then many jars will be appended to the Flink client >>>>>>>>> classpath. >>>>>>>>>>> 3. The compile hadoop version is still 2.4.1. Right? >>>>>>>>>>> >>>>>>>>>>> >>>>>>>>>>> Best, >>>>>>>>>>> Yang >>>>>>>>>>> >>>>>>>>>>> >>>>>>>>>>> Sivaprasanna <[hidden email]> 于2020年4月22日周三 >>>>>>>>>>> 下午4:18写道: >>>>>>>>>>> >>>>>>>>>>>> I agree with Aljoscha. Otherwise I can see a lot of tickets >>>> getting >>>>>>>>>> created >>>>>>>>>>>> saying the application is not running on YARN. >>>>>>>>>>>> >>>>>>>>>>>> Cheers, >>>>>>>>>>>> Sivaprasanna >>>>>>>>>>>> >>>>>>>>>>>> On Wed, Apr 22, 2020 at 1:00 PM Aljoscha Krettek >>>>>>>>>>>> <[hidden email] >>>>>>>>>>>> wrote: >>>>>>>>>>>> >>>>>>>>>>>>> +1 to getting rid of flink-shaded-hadoop. But we need to >>>>>>>>>>>>> document how >>>>>>>>>>>>> people can now get a Flink dist that works with Hadoop. >>>> Currently, >>>>>>>>> when >>>>>>>>>>>>> you download the single shaded jar you immediately get >> support >>>> for >>>>>>>>>>>>> submitting to YARN via bin/flink run. >>>>>>>>>>>>> >>>>>>>>>>>>> Aljoscha >>>>>>>>>>>>> >>>>>>>>>>>>> >>>>>>>>>>>>> On 22.04.20 09:08, Till Rohrmann wrote: >>>>>>>>>>>>>> Hi Robert, >>>>>>>>>>>>>> >>>>>>>>>>>>>> I think it would be a helpful simplification of Flink's >> build >>>>>>>>>>>>>> setup >>>>>>>>> if >>>>>>>>>>>> we >>>>>>>>>>>>>> can get rid of flink-shaded-hadoop. Moreover relying only on >>> the >>>>>>>>>>>> vanilla >>>>>>>>>>>>>> Hadoop dependencies for the modules which interact with >>>>>>>>>>>>>> Hadoop/Yarn >>>>>>>>>>>>> sounds >>>>>>>>>>>>>> like a good idea to me. >>>>>>>>>>>>>> >>>>>>>>>>>>>> Adding support for Hadoop 3 would also be nice. I'm not >> sure, >>>>>>>>> though, >>>>>>>>>>>> how >>>>>>>>>>>>>> Hadoop's API's have changed between 2 and 3. It might be >>>> necessary >>>>>>>>> to >>>>>>>>>>>>>> introduce some bridges in order to make it work. >>>>>>>>>>>>>> >>>>>>>>>>>>>> Cheers, >>>>>>>>>>>>>> Till >>>>>>>>>>>>>> >>>>>>>>>>>>>> On Tue, Apr 21, 2020 at 4:37 PM Robert Metzger >>>>>>>>>>>>>> <[hidden email] >>>>>>>>>>>>> wrote: >>>>>>>>>>>>>>> Hi all, >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> for the upcoming 1.11 release, I started looking into >> adding >>>>>>>>> support >>>>>>>>>>>> for >>>>>>>>>>>>>>> Hadoop 3[1] for Flink. I have explored a little bit already >>>> into >>>>>>>>>>>> adding >>>>>>>>>>>>> a >>>>>>>>>>>>>>> shaded hadoop 3 into “flink-shaded”, and some mechanisms >> for >>>>>>>>>> switching >>>>>>>>>>>>>>> between Hadoop 2 and 3 dependencies in the Flink build. >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> However, Chesnay made me aware that we could also go a >>>> different >>>>>>>>>>>> route: >>>>>>>>>>>>> We >>>>>>>>>>>>>>> let Flink depend on vanilla Hadoop dependencies and stop >>>>>>>>>>>>>>> providing >>>>>>>>>>>>> shaded >>>>>>>>>>>>>>> fat jars for Hadoop through “flink-shaded”. >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> Why? >>>>>>>>>>>>>>> - Maintaining properly shaded Hadoop fat jars is a lot of >>> work >>>>>>>>>>>>>>> (we >>>>>>>>>>>> have >>>>>>>>>>>>>>> insufficient test coverage for all kinds of Hadoop >> features) >>>>>>>>>>>>>>> - For Hadoop 2, there are already some known and unresolved >>>>>>>>>>>>>>> issues >>>>>>>>>>>> with >>>>>>>>>>>>> our >>>>>>>>>>>>>>> shaded jars that we didn’t manage to fix >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> Users will have to use Flink with Hadoop by relying on >>> vanilla >>>> or >>>>>>>>>>>>>>> vendor-provided Hadoop dependencies. >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> What do you think? >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> Best, >>>>>>>>>>>>>>> Robert >>>>>>>>>>>>>>> >>>>>>>>>>>>>>> [1] https://issues.apache.org/jira/browse/FLINK-11086 >>>>>>>>>>>>>>> >>>> >> >> -- >> >> Konstantin Knauf >> >> https://twitter.com/snntrable >> >> https://github.com/knaufk >> |
Free forum by Nabble | Edit this page |