vlad-lyutenko commented on PR #13070: URL: https://github.com/apache/iceberg/pull/13070#issuecomment-3265603117
> Still figuring out what the issue is, even after reverting back to the un-refactored code some of the spark 3.4 avro tests still fail with the following. Note, the same Spark 3.5 tests consistently pass. > > ``` > java.io.IOException: Block read partially, the data may be corrupt > org.apache.avro.AvroRuntimeException: java.io.IOException: Block read partially, the data may be corrupt > at app//org.apache.avro.file.DataFileStream.hasNext(DataFileStream.java:237) > at app//org.apache.iceberg.avro.AvroIterable$AvroRangeIterator.hasNext(AvroIterable.java:131) > at app//org.apache.iceberg.avro.AvroIterable$AvroReuseIterator.hasNext(AvroIterable.java:193) > at app//org.apache.iceberg.io.FilterIterator.advance(FilterIterator.java:64) > at app//org.apache.iceberg.io.FilterIterator.hasNext(FilterIterator.java:49) > at app//org.apache.iceberg.spark.source.BaseReader.next(BaseReader.java:131) > at app//org.apache.spark.sql.execution.datasources.v2.PartitionIterator.hasNext(DataSourceRDD.scala:120) > at app//org.apache.spark.sql.execution.datasources.v2.MetricsIterator.hasNext(DataSourceRDD.scala:158) > at app//org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.$anonfun$hasNext$1(DataSourceRDD.scala:63) > at app//org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.$anonfun$hasNext$1$adapted(DataSourceRDD.scala:63) > at app//scala.Option.exists(Option.scala:376) > at app//org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.hasNext(DataSourceRDD.scala:63) > at app//org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37) > at app//scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460) > at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source) > at app//org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) > at app//org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:760) > at app//org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:388) > at app//org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:891) > at app//org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:891) > at app//org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) > at app//org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:367) > at app//org.apache.spark.rdd.RDD.iterator(RDD.scala:331) > at app//org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:92) > at app//org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161) > at app//org.apache.spark.scheduler.Task.run(Task.scala:139) > at app//org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:554) > at app//org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1529) > at app//org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:557) > at [email protected]/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) > at [email protected]/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) > at [email protected]/java.lang.Thread.run(Thread.java:840) > Caused by: java.io.IOException: Block read partially, the data may be corrupt > at org.apache.avro.file.DataFileStream.hasNext(DataFileStream.java:222) > ... 31 more > ``` Hi @amogh-jahagirdar we have the same exception in our tests (in Trino) after we upgrade to iceberg containing this changes. And it looks like this: as far as I understand this change - you try to read row_id : ``` public static ValueReader<Long> rowIds(Long baseRowId, ValueReader<?> idReader) { if (baseRowId != null) { return new RowIdReader(baseRowId, (ValueReader<Long>) idReader); } else { return ValueReaders.constant(null); } } ``` Either as constant or by calculating it as first_row_id + position. But in our case it's failing for situation when _row_id is already presents in data file and we want just to read it as it is (without calculating it on the fly, because the value could actually be incorrect, for example after update). And in this case test fails, because in previous version - we just read the field with `LongReader` but now it's going to branch: ``` } else { return ValueReaders.constant(null); } ``` instead of pure reading. Maybe you can suggest some way how to use `AvroIterable` -> `ValueReaders` as pure reader, without this functionality. Big thanks in advance. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
