@Fabian Good point. For Flink 2.0, I would suggest to remove them from the Environment and add them to a Utility. The way it is now, it ties Flink very strongly to Hadoop.
You are right, before we do that, there is no way to make a Hadoop independent distribution. On Fri, Oct 14, 2016 at 10:37 AM, Fabian Hueske <fhue...@gmail.com> wrote: > +1 for dropping Hadoop1 support. > > Regarding a binary release without Hadoop: > > What would we do about the readHadoopFile() and createHadoopInput() on the > ExecutionEnvironment? > These methods are declared as @PublicEvolving, so we did not commit to keep > them. > However that does not necessarily mean we should easily break the API here > esp. since the methods have not been declared @deprecated. > > Best, Fabian > > > > 2016-10-14 10:29 GMT+02:00 Stephan Ewen <se...@apache.org>: > > > @Greg > > > > I think that would be amazing. It does require a bit of cleanup, though. > As > > far as I know, the Hadoop dependency is additionally used for some > Kerberos > > utilities and for its S3 file system implementation. > > We would need to make the Kerberos part Hadoop independent and the > > FileSystem loading dynamic (with a good exception that the Hadoop > > dependency should be added if the filesystem cannot be loaded). > > > > Stephan > > > > > > On Thu, Oct 13, 2016 at 8:55 PM, Greg Hogan <c...@greghogan.com> wrote: > > > > > Okay, this sounds prudent. Would this be the right time to implement > > > FLINK-2268 "Provide Flink binary release without Hadoop"? > > > > > > On Thu, Oct 13, 2016 at 11:25 AM, Stephan Ewen <se...@apache.org> > wrote: > > > > > > > +1 for dropping Hadoop1 support > > > > > > > > @greg There is quite some complexity in the build setup and release > > > scripts > > > > and testing to support Hadoop 1. Also, we have to prepare to add > > support > > > > for Hadoop 3, and then supporting in addition Hadoop 1 seems very > > tough. > > > > > > > > Stephan > > > > > > > > > > > > On Thu, Oct 13, 2016 at 5:04 PM, Greg Hogan <c...@greghogan.com> > > wrote: > > > > > > > > > Hi Robert, > > > > > > > > > > What are the benefits to Flink for dropping Hadoop 1 support? Is > > there > > > > > significant code cleanup or would we simply be publishing one less > > set > > > of > > > > > artifacts? > > > > > > > > > > Greg > > > > > > > > > > On Thu, Oct 13, 2016 at 10:47 AM, Robert Metzger < > > rmetz...@apache.org> > > > > > wrote: > > > > > > > > > > > Hi, > > > > > > > > > > > > The Apache Hadoop community has recently released the first alpha > > > > version > > > > > > for Hadoop 3.0.0, while we are still supporting Hadoop 1. I think > > its > > > > > time > > > > > > to finally drop Hadoop 1 support in Flink. > > > > > > > > > > > > The last minor Hadoop 1 release was in 27 June, 2014. > > > > > > Apache Spark dropped Hadoop 1 support with their 2.0 release in > > July > > > > > 2016. > > > > > > Hadoop 2.2 was first released in October 2013, so there was > enough > > > time > > > > > > for users to upgrade. > > > > > > > > > > > > I added also the user@ list to the discussion to get opinions > > about > > > > this > > > > > > from there as well. > > > > > > > > > > > > Let me know what you think about this! > > > > > > > > > > > > > > > > > > Regards, > > > > > > Robert > > > > > > > > > > > > > > > > > > > > >