[
https://issues.apache.org/jira/browse/FLINK-8186?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16276027#comment-16276027
]
Stephan Ewen commented on FLINK-8186:
-------------------------------------
Hotfix: You should be able to make it work by setting
{{classloader.resolve-order: parent-first}} in the {{flink-conf.yaml}}.
The problem seems to be caused by the inverted classloading in Flink 1.4.
Inverted classloading means that user code may load a different copy of its
dependencies, compared to those in the classpath, and helps circumvent many
dependency conflicts, as well as other issues like class cast conflicts in
Avro's schema and data caching.
Apparently, for some reason, the {{AvroInputFormat}} (or at least its Avro
dependency) gets loaded from the parent (classpath, i.e., from Avro in
{{flink-dist}} or in the Hadoop jars) and the user-supplied parameters get
properly loaded from the user-code jar. Hence the clash that {{GenericRecord !=
GenericRecord}}.
One fix would be to remove the {{flink-avro}} dependency from {{flink-dist}}
(and removing the exclusion in the quickstarts), meaning that it always has to
go through the user code classloader - no dependency duplication.
But I think we should better understand why this happens in the first place.
The {{AvroInputFormat}} and the Avro classes it references should also come
from the user code jar, and we should know why it does not come from there in
this scenario.
[~aljoscha] - because this looks like a classloading issue, what is your take?
> AvroInputFormat regression: fails to deserialize GenericRecords on standalone
> cluster with hadoop27 compat
> ----------------------------------------------------------------------------------------------------------
>
> Key: FLINK-8186
> URL: https://issues.apache.org/jira/browse/FLINK-8186
> Project: Flink
> Issue Type: Bug
> Affects Versions: 1.4.0
> Reporter: Sebastian Klemke
> Priority: Minor
> Attachments: GenericRecordCount.java, pom.xml
>
>
> The following job runs fine on a Flink 1.3.2 cluster, but fails on a Flink
> 1.4.0 RC2 standalone cluster, "hadoop27" flavour:
> {code}
> public class GenericRecordCount {
> public static void main(String[] args) throws Exception {
> String input = ParameterTool.fromArgs(args).getRequired("input");
> ExecutionEnvironment env =
> ExecutionEnvironment.getExecutionEnvironment();
> long count = env.readFile(new AvroInputFormat<>(new Path(input),
> GenericRecord.class), input)
> .count();
> System.out.printf("Counted %d records\n", count);
> }
> }
> {code}
> Runs fine in LocalExecutionEnvironment and also on no-hadoop flavour
> standalone cluster, though. Exception thrown in Flink 1.4.0 hadoop27:
> {code}
> 12/01/2017 13:22:09 DataSource (at
> readFile(ExecutionEnvironment.java:514)
> (org.apache.flink.formats.avro.AvroInputFormat))(4/4) switched to FAILED
> java.lang.RuntimeException: java.lang.NoSuchMethodException:
> org.apache.avro.generic.GenericRecord.<init>()
> at
> org.apache.avro.specific.SpecificData.newInstance(SpecificData.java:353)
> at
> org.apache.avro.specific.SpecificData.newRecord(SpecificData.java:369)
> at org.apache.avro.reflect.ReflectData.newRecord(ReflectData.java:901)
> at
> org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:212)
> at
> org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:175)
> at
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:153)
> at
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:145)
> at org.apache.avro.file.DataFileStream.next(DataFileStream.java:233)
> at
> org.apache.flink.formats.avro.AvroInputFormat.nextRecord(AvroInputFormat.java:165)
> at
> org.apache.flink.runtime.operators.DataSourceTask.invoke(DataSourceTask.java:167)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:718)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.NoSuchMethodException:
> org.apache.avro.generic.GenericRecord.<init>()
> at java.lang.Class.getConstructor0(Class.java:3082)
> at java.lang.Class.getDeclaredConstructor(Class.java:2178)
> at
> org.apache.avro.specific.SpecificData.newInstance(SpecificData.java:347)
> ... 11 more
> {code}
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)