[
https://issues.apache.org/jira/browse/PARQUET-2276?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17718203#comment-17718203
]
Steve Loughran commented on PARQUET-2276:
-----------------------------------------
[~a2l] really? hadoop 2.8? why haven't they upgraded yet? that is a long way
behind on any form of security updates, doesn't come with any guarantees of
java8+ support etc. Even hadoop 2.9.x only gets CVE updates for hadoop's own
code so that those people running their own clusters with private hadoop-2
forks know what to pick up.
The PARQUET-2134 patch did not break Hadoop 2 compatibility; it used APIs it's
which were in the version of Hadoop that parquet compiled against. What it did
do was "explicitly break compatibility with a version of hadoop older than the
one parquet was built against" That patch may have been the one to show the
problem but the reality is there are many other places where incompatibilities
could've surfaced.
If you actually want to support hadoop-2.8.5 then the pom needs to be
downgraded before anything else.
You also need to worry about Java8/7 compatibility. We're already in a problem
where some of the java.nio classes in the java8 SDKs you can get have added
more overridden bytebuffer methods then were in the original Oracle Java8, and
https algorithms I've been another moving target. so even within "java8" there
is "original java8" and the openjdk/corretto/azul versions. Well you can get
away with building a modern library with a recent open JDK build, if you really
are planning on supporting hadoop 2.8 suddenly all these issues surface. [I
know this as when i have to go near the hadoop-2 line i have to use a docker
image with java7/, and since moving to macbook m1 I can't do that any more.
> ParquetReader reads do not work with Hadoop version 2.8.5
> ---------------------------------------------------------
>
> Key: PARQUET-2276
> URL: https://issues.apache.org/jira/browse/PARQUET-2276
> Project: Parquet
> Issue Type: Bug
> Components: parquet-mr
> Affects Versions: 1.13.0
> Reporter: Atul Mohan
> Assignee: Fokko Driesprong
> Priority: Major
> Fix For: 1.14.0, 1.13.1
>
>
> {{ParquetReader.read() fails with the following exception on parquet-mr
> version 1.13.0 when using hadoop version 2.8.5:}}
> {code:java}
> java.lang.NoSuchMethodError: 'boolean
> org.apache.hadoop.fs.FSDataInputStream.hasCapability(java.lang.String)'
> at
> org.apache.parquet.hadoop.util.HadoopStreams.isWrappedStreamByteBufferReadable(HadoopStreams.java:74)
>
> at org.apache.parquet.hadoop.util.HadoopStreams.wrap(HadoopStreams.java:49)
> at
> org.apache.parquet.hadoop.util.HadoopInputFile.newStream(HadoopInputFile.java:69)
>
> at
> org.apache.parquet.hadoop.ParquetFileReader.<init>(ParquetFileReader.java:787)
>
> at
> org.apache.parquet.hadoop.ParquetFileReader.open(ParquetFileReader.java:657)
> at org.apache.parquet.hadoop.ParquetReader.initReader(ParquetReader.java:162)
> org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:135)
> {code}
>
>
>
> From an initial investigation, it looks like HadoopStreams has started using
> [FSDataInputStream.hasCapability|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L74]
> but _FSDataInputStream_ does not have the _hasCapability_ API in [hadoop
> 2.8.x|https://hadoop.apache.org/docs/r2.8.3/api/org/apache/hadoop/fs/FSDataInputStream.html].
--
This message was sent by Atlassian Jira
(v8.20.10#820010)