Hey everyone, I also got some pushback in Iceberg, so I've taken the time to revert the change to continue <https://github.com/apache/parquet-mr/pull/1084/> to support Hadoop <2.9. We now have two mechanisms to check if the stream is byte buffer readable. First it will use the new mechanism (For Hadoop 2.9+ and Hadoop 3.3+. Otherwise, it will fall back to the previous method. Please review and let me know what you think. Once in, I can backport this to 1.13.1.
Kind regards, Fokko Driesprong Op vr 28 apr 2023 om 10:06 schreef Fokko Driesprong <[email protected]>: > And it is in Hive as well: https://github.com/apache/hive/pull/427 > > Kind regards, > Fokko > > Op wo 26 apr 2023 om 18:25 schreef Fokko Driesprong <[email protected]>: > >> Hi Xinli, >> >> I know that some folks are waiting on running Apache Flink without Hadoop >> (but using the parquet-mr library in Iceberg). Spark already upgraded to >> Parquet 1.3.0, and I did some checks today with Hive, and I don't see any >> blockers there. What's your understanding of a reasonable time window? >> Personally, I don't mind running a release, especially a patch release >> should be quite straightforward. >> >> Kind regards, >> Fokko Driesprong >> >> Op wo 26 apr 2023 om 04:37 schreef Xinli shang <[email protected]>: >> >>> Hi Fokko, >>> >>> Thanks for volunteering to release 1.13.1! That would be great and I am >>> looking forward to you being the release manager for that. >>> >>> We can have the 1.13.1 release to add back the support old Hadoop >>> version, >>> but the question is should we release ASAP or wait for a reasonable time >>> window? The new version 1.13.0 is just released and I am not sure if >>> there >>> are more issues coming so that we can put together the fixes into 1.13.1. >>> Is Iceberg urgently blocked on this? >>> >>> Xinli Shang >>> >>> >>> >>> On Tue, Apr 25, 2023 at 6:51 PM Gang Wu <[email protected]> wrote: >>> >>> > That sounds good to me. >>> > >>> > I have just released 1.13.0, just let me know if you need anything >>> > on my end to make the next release. >>> > >>> > Best, >>> > Gang >>> > >>> > On Tue, Apr 25, 2023 at 10:31 PM Fokko Driesprong <[email protected]> >>> > wrote: >>> > >>> > > Hey Gang, >>> > > >>> > > Thanks for the quick reply. I think 2.8.x is water under the bridge, >>> but >>> > I >>> > > can be convinced otherwise. I also spend a few cycles to see if we >>> can >>> > get >>> > > compatibility with 2.7.3+, but it doesn't seem trivial >>> > > < >>> https://github.com/apache/parquet-mr/pull/1075#issuecomment-1514518094 >>> > >. >>> > > As Gabor said on the ticket, it is fine to drop support for older >>> systems >>> > > from time to time. The public Hadoop 2.8 >>> > > <https://github.com/apache/hadoop/tree/branch-2.8> doesn't seem to >>> get >>> > any >>> > > active updates. I don't fully agree with the ticket, you can still >>> read >>> > > Parquet, but using an older version of the library. >>> > > >>> > > Kind regards, >>> > > Fokko Driesprong >>> > > >>> > > Op di 25 apr 2023 om 16:13 schreef Gang Wu <[email protected]>: >>> > > >>> > > > Hi Fokko, >>> > > > >>> > > > There is an issue of the 1.13.0 release: >>> > > > https://issues.apache.org/jira/browse/PARQUET-2276. >>> > > > >>> > > > It seems that Hadoop 2.8.x is no longer supported after 1.13.0. I >>> have >>> > > seen >>> > > > that >>> > > > you have added CI checks for Hadoop 2.9.x. Not sure if this is a >>> > > > blocking issue. >>> > > > >>> > > > Best, >>> > > > Gang >>> > > > >>> > > > >>> > > > >>> > > > On Tue, Apr 25, 2023 at 3:25 PM Fokko Driesprong <[email protected] >>> > >>> > > wrote: >>> > > > >>> > > > > Hi all, >>> > > > > >>> > > > > I would like to discuss releasing Parquet 1.13.1. For Iceberg we >>> ran >>> > > into >>> > > > > two things: >>> > > > > >>> > > > > - We noticed that support for Hadoop 2 was dropped. Iceberg is >>> > still >>> > > > on >>> > > > > 2.7.3, and we're aware of the fact that has been released in >>> > August >>> > > > > 2016. >>> > > > > The PR that I've created >>> > > > > <https://github.com/apache/parquet-mr/pull/1083/> bumps the >>> lower >>> > > > bound >>> > > > > to Hadoop 2.9.2. Which is also old, but if possible we would >>> like >>> > to >>> > > > > cater >>> > > > > to the widest audience possible. >>> > > > > - At Iceberg we also have the Apache Flink integration, and >>> Flink >>> > is >>> > > > > able to run without Hadoop. This required some minor changes >>> > (#1074 >>> > > > > <https://github.com/apache/parquet-mr/pull/1074>, #1073 >>> > > > > <https://github.com/apache/parquet-mr/pull/1073>) that >>> already >>> > have >>> > > > > been >>> > > > > backported. It would be awesome to get these out. >>> > > > > >>> > > > > My question is, after the release of 1.13.0 are there any issues >>> that >>> > > > came >>> > > > > up, or anything that you would like to see being released? I'm >>> happy >>> > to >>> > > > > volunteer as a release manager for 1.13.1. Let us know! >>> > > > > >>> > > > > Kind regards, >>> > > > > Fokko >>> > > > > >>> > > > >>> > > >>> > >>> >>> >>> -- >>> Xinli Shang >>> >>
