Thanks, Josh! I guess it makes sense that without the base class you can't
load the Parquet class.

We'll have to watch out for Hadoop/Flink issues. I think we hit one as well
where not having Configuration in the Flink classpath could prevent loading
modules correctly.

Ryan

On Sun, Sep 26, 2021 at 2:49 AM Joshua Fan <joshuafat...@gmail.com> wrote:

> Hi
> I found the reason why  this exception 'java.lang.NoClassDefFoundError:
> org/apache/iceberg/shaded/org/apache/parquet/hadoop/ParquetInputFormat'
> was raised. Actually, it was because of the absence of class
> 'org/apache/hadoop/mapreduce/lib/input/FileInputFormat'.  After I put
> the hadoop-mapreduce-client-core-2.7.3.jar to the classpath. It goes well.
> Thanks to Openinx.
>
> Josh
>
> Joshua Fan <joshuafat...@gmail.com> 于2021年9月26日周日 下午3:40写道:
>
>> Hi openinx
>>
>> I do not get you. what do you mean by 'Looks like the line 112 in
>> HadoopReadOptions is not the first line accessing the variables in
>> ParquetInputFormat.'?
>> The parquet file I want to read was wrote by iceberg table without any
>> explicit specified, no file format and no parquet version was specified.
>> I just want to read the parquet file by iceberg, when read, there was
>> also no explicit file format and parquet version.
>>
>> OpenInx <open...@gmail.com> 于2021年9月23日周四 下午12:34写道:
>>
>>> Hi Joshua
>>>
>>> Can you check what's the parquet version you are using ?   Looks like
>>> the line 112 in HadoopReadOptions is not the first line accessing the
>>> variables in ParquetInputFormat.
>>>
>>> [image: image.png]
>>>
>>> On Wed, Sep 22, 2021 at 11:07 PM Joshua Fan <joshuafat...@gmail.com>
>>> wrote:
>>>
>>>> Hi
>>>> I am glad to use iceberg as table source in flink sql, flink version is
>>>> 1.13.2, and iceberg version is 0.12.0.
>>>>
>>>> After changed the flink version from 1.12 to 1.13, and changed some
>>>> codes in FlinkCatalogFactory, the project can be build successfully.
>>>>
>>>> First, I tried to write data into iceberg by flink sql, and it seems go
>>>> well. And then I want to verify the data, so I want to read from iceberg
>>>> table, I wrote a
>>>> simple sql, like "select * from
>>>> iceberg_catalog.catalog_database.catalog_table", the sql can be submitted,
>>>> but the flink job kept restarting by 'java.lang.NoClassDefFoundError:
>>>> org/apache/iceberg/shaded/org/apache/parquet/hadoop/ParquetInputFormat'.
>>>> But, actually, ParquetInputFormat was in the 
>>>> iceberg-flink-runtime-0.12.0.jar.
>>>> Has no idea why this can happen.
>>>> The full stack trace is below:
>>>> java.lang.NoClassDefFoundError:
>>>> org/apache/iceberg/shaded/org/apache/parquet/hadoop/ParquetInputFormat
>>>>     at
>>>> org.apache.iceberg.shaded.org.apache.parquet.HadoopReadOptions$Builder.<init>(HadoopReadOptions.java:112)
>>>> ~[iceberg-flink-runtime-0.12.0-qihoo.jar:?]
>>>>     at
>>>> org.apache.iceberg.shaded.org.apache.parquet.HadoopReadOptions$Builder.<init>(HadoopReadOptions.java:97)
>>>> ~[iceberg-flink-runtime-0.12.0-qihoo.jar:?]
>>>>     at
>>>> org.apache.iceberg.shaded.org.apache.parquet.HadoopReadOptions.builder(HadoopReadOptions.java:85)
>>>> ~[iceberg-flink-runtime-0.12.0-qihoo.jar:?]
>>>>     at
>>>> org.apache.iceberg.parquet.Parquet$ReadBuilder.build(Parquet.java:793)
>>>> ~[iceberg-flink-runtime-0.12.0-qihoo.jar:?]
>>>>     at
>>>> org.apache.iceberg.flink.source.RowDataIterator.newParquetIterable(RowDataIterator.java:135)
>>>> ~[iceberg-flink-runtime-0.12.0-qihoo.jar:?]
>>>>     at
>>>> org.apache.iceberg.flink.source.RowDataIterator.newIterable(RowDataIterator.java:86)
>>>> ~[iceberg-flink-runtime-0.12.0-qihoo.jar:?]
>>>>     at
>>>> org.apache.iceberg.flink.source.RowDataIterator.openTaskIterator(RowDataIterator.java:74)
>>>> ~[iceberg-flink-runtime-0.12.0-qihoo.jar:?]
>>>>     at
>>>> org.apache.iceberg.flink.source.DataIterator.updateCurrentIterator(DataIterator.java:102)
>>>> ~[iceberg-flink-runtime-0.12.0-qihoo.jar:?]
>>>>     at
>>>> org.apache.iceberg.flink.source.DataIterator.hasNext(DataIterator.java:84)
>>>> ~[iceberg-flink-runtime-0.12.0-qihoo.jar:?]
>>>>     at
>>>> org.apache.iceberg.flink.source.FlinkInputFormat.reachedEnd(FlinkInputFormat.java:104)
>>>> ~[iceberg-flink-runtime-0.12.0-qihoo.jar:?]
>>>>     at
>>>> org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction.run(InputFormatSourceFunction.java:89)
>>>> ~[flink-dist_2.11-1.13.2.jar:1.13.2]
>>>>     at
>>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:110)
>>>> ~[flink-dist_2.11-1.13.2.jar:1.13.2]
>>>>     at
>>>> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:66)
>>>> ~[flink-dist_2.11-1.13.2.jar:1.13.2]
>>>>     at
>>>> org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:269)
>>>> ~[flink-dist_2.11-1.13.2.jar:1.13.2]
>>>> You can see that the HadoopReadOptions can be found.
>>>>
>>>> Any help will be appricated. Thank you.
>>>>
>>>> Yours sincerely
>>>>
>>>> Josh
>>>>
>>>

-- 
Ryan Blue
Tabular

Reply via email to