tprelle opened a new issue, #4604:
URL: https://github.com/apache/iceberg/issues/4604

   When you you add a inner struct field as partition field without any 
transformation on an Iceberg orc table, you can not select anymore only this 
field because the as is a partition, iceberg do not read it from the file as 
it's inside a struct OrcValueReaders it's not able to reconstruct a constant 
struct.
   
   It's working when you select also another field from the struct because as 
OrcValueReaders is able to read from the file using the another field reader
   
   ```
   java.lang.IndexOutOfBoundsException: Index: 0, Size: 0
        at java.util.ArrayList.rangeCheck(ArrayList.java:659)
        at java.util.ArrayList.get(ArrayList.java:435)
        at 
org.apache.iceberg.orc.OrcValueReaders$StructReader.<init>(OrcValueReaders.java:161)
        at 
org.apache.iceberg.spark.data.SparkOrcValueReaders$StructReader.<init>(SparkOrcValueReaders.java:143)
        at 
org.apache.iceberg.spark.data.SparkOrcValueReaders.struct(SparkOrcValueReaders.java:70)
        at 
org.apache.iceberg.spark.data.SparkOrcReader$ReadBuilder.record(SparkOrcReader.java:75)
        at 
org.apache.iceberg.spark.data.SparkOrcReader$ReadBuilder.record(SparkOrcReader.java:65)
        at 
org.apache.iceberg.orc.OrcSchemaWithTypeVisitor.visitRecord(OrcSchemaWithTypeVisitor.java:71)
        at 
org.apache.iceberg.orc.OrcSchemaWithTypeVisitor.visit(OrcSchemaWithTypeVisitor.java:38)
        at 
org.apache.iceberg.orc.OrcSchemaWithTypeVisitor.visit(OrcSchemaWithTypeVisitor.java:32)
        at 
org.apache.iceberg.spark.data.SparkOrcReader.<init>(SparkOrcReader.java:52)
        at 
org.apache.iceberg.spark.source.RowDataReader.lambda$newOrcIterable$2(RowDataReader.java:163)
        at org.apache.iceberg.orc.OrcIterable.iterator(OrcIterable.java:108)
        at org.apache.iceberg.orc.OrcIterable.iterator(OrcIterable.java:45)
        at org.apache.iceberg.util.Filter.lambda$filter$0(Filter.java:35)
        at 
org.apache.iceberg.io.CloseableIterable$2.iterator(CloseableIterable.java:73)
        at 
org.apache.iceberg.spark.source.RowDataReader.open(RowDataReader.java:77)
        at 
org.apache.iceberg.spark.source.BaseDataReader.next(BaseDataReader.java:107)
        at 
org.apache.spark.sql.execution.datasources.v2.PartitionIterator.hasNext(DataSourceRDD.scala:93)
        at 
org.apache.spark.sql.execution.datasources.v2.MetricsIterator.hasNext(DataSourceRDD.scala:130)
        at 
org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
        at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.agg_doAggregateWithKeys_1$(Unknown
 Source)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.agg_doAggregateWithKeys_0$(Unknown
 Source)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.sort_addToSorter_0$(Unknown
 Source)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
 Source)
        at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
        at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:759)
        at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:349)
        at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
        at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:131)
        at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462)
   ```
   
   To reproduce
   ```
   CREATE TABLE testTable (id INT, st struct<id:INT, data:string>) USING 
iceberg TBLPROPERTIES ('write.format.default'='orc');
   INSERT INTO TABLE testTable VALUES (1, named_struct('id', 10, 'data', 
'v1')),(2, named_struct('id', 20, 'data', 'v2'));
   ALTER TABLE testTable ADD PARTITION FIELD st.data as data;
   INSERT INTO TABLE %s VALUES (1, named_struct('id', 11, 'data', 'v1'));
   select st.id, st.data from testTable; //OK
   select distinct st.data from testTable; //KO
   ```
   


-- 
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]

Reply via email to