Hi Aleksandr,
you could try the next configuration option:
classloader.parent-first-patterns.additional: "org.apache.hadoop"
to force Flink to load Hadoop classes on the parent ClassLoader.
No guarantees, but maybe it will solve your problem.
On Mon, 2025-05-12 at 11:26 +0100, Aleksandr Pilipenko wrote:
Hi all,
After updating one of our Flink jobs from 1.18 to 1.20 we started to see a
classloading issue when using file source with Parquet Avro format, which looks
like a regression:
java.lang.NoClassDefFoundError: org/apache/hadoop/conf/Configuration
at
org.apache.flink.formats.parquet.avro.AvroParquetRecordFormat.createReader(AvroParquetRecordFormat.java:86)
at
org.apache.flink.connector.file.src.impl.StreamFormatAdapter.lambda$createReader$0(StreamFormatAdapter.java:77)
...
Caused by: java.lang.ClassNotFoundException:
org.apache.hadoop.conf.Configuration
at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(Unknown
Source)
Further digging has shown that this issue was caused by changes to
AvroParquetRecordFormat from FLINK-35015[1][2] - even though class mentioned in
the exception is present in child classloader, exception is thrown when attempt
to access HadoopUtils.getHadoopConfiguration during creation of the reader.
One path around this is to include hadoop distribution into the image as
mentioned in docs [3], however this leads to significant increase in image size
compared to having necessary dependencies in the application jar.
1 - https://issues.apache.org/jira/browse/FLINK-35015
2 -
https://github.com/apache/flink/blob/release-1.20/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/avro/AvroParquetRecordFormat.java#L86
3 -
https://nightlies.apache.org/flink/flink-docs-release-1.20/docs/dev/configuration/advanced/#hadoop-dependencies