[
https://issues.apache.org/jira/browse/BEAM-6697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16777332#comment-16777332
]
Igor Dvorzhak commented on BEAM-6697:
-------------------------------------
This happens is because `GoogleCloudStorageReadChannel` constructor doesn't
initialize GCS metadata (includes object size) - it's initialized lazily during
first read.
Metadata initialized eagerly only if `GoogleCloudStorageReadChannel` created
via `GoogleCloudStorage.open()` method.
It's fixed
[here|https://github.com/GoogleCloudPlatform/bigdata-interop/commit/8f6443bfd6ee821c5667dd2811cf3fe03167b755]
and will be release in GCS connector 1.9.16 in couple hours.
> ParquetIO Performance test is failing on (GCS filesystem)
> ---------------------------------------------------------
>
> Key: BEAM-6697
> URL: https://issues.apache.org/jira/browse/BEAM-6697
> Project: Beam
> Issue Type: New Feature
> Components: io-java-parquet, test-failures
> Reporter: Lukasz Gajowy
> Priority: Blocker
> Fix For: 2.11.0
>
>
> Relevant failure logs:
> {code:java}
> Caused by: java.lang.RuntimeException:
> org.apache.beam.sdk.io.parquet.ParquetIO$ReadFiles$BeamParquetInputFile@2de8303e
> is not a Parquet file (too small length: -1)
> at
> org.apache.parquet.hadoop.ParquetFileReader.readFooter(ParquetFileReader.java:514)
> at
> org.apache.parquet.hadoop.ParquetFileReader.<init>(ParquetFileReader.java:689)
> at
> org.apache.parquet.hadoop.ParquetFileReader.open(ParquetFileReader.java:595)
> at
> org.apache.parquet.hadoop.ParquetReader.initReader(ParquetReader.java:152)
> at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:135)
> at
> org.apache.beam.sdk.io.parquet.ParquetIO$ReadFiles$ReadFn.processElement(ParquetIO.java:221){code}
>
> Full logs can be found here:
> [https://builds.apache.org/view/A-D/view/Beam/view/PerformanceTests/job/beam_PerformanceTests_ParquetIOIT/|https://builds.apache.org/view/A-D/view/Beam/view/PerformanceTests/job/beam_PerformanceTests_ParquetIOIT/1096/console]
>
>
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)