[ 
https://issues.apache.org/jira/browse/HUDI-7276?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17886556#comment-17886556
 ] 

Lin Liu edited comment on HUDI-7276 at 10/3/24 5:17 AM:
--------------------------------------------------------

After disable fg reader, the query works.


{code:java}
scala> sdf.write.format("hudi").
     |   option("hoodie.table.name", "test_table").
     |   option("hoodie.datasource.write.recordkey.field", "key").
     |   option("hoodie.datasource.write.table.type", "MERGE_ON_READ").
     |   option("hoodie.datasource.write.operation", "upsert").
     |   option("hoodie.file.group.reader.enabled", "false").
     |   mode("Append").
     |   save(basePath)
24/10/03 05:13:13 WARN ConfigUtils: The configuration key 
'hoodie.cleaner.policy.failed.writes' has been deprecated and may be removed in 
the future. Please use the new key 'hoodie.clean.failed.writes.policy' instead.
24/10/03 05:13:14 WARN ConfigUtils: The configuration key 
'hoodie.cleaner.policy.failed.writes' has been deprecated and may be removed in 
the future. Please use the new key 'hoodie.clean.failed.writes.policy' instead.
24/10/03 05:13:14 WARN ConfigUtils: The configuration key 
'hoodie.cleaner.policy.failed.writes' has been deprecated and may be removed in 
the future. Please use the new key 'hoodie.clean.failed.writes.policy' instead.
24/10/03 05:13:42 WARN ConfigUtils: The configuration key 
'hoodie.cleaner.policy.failed.writes' has been deprecated and may be removed in 
the future. Please use the new key 'hoodie.clean.failed.writes.policy' instead.
24/10/03 05:13:43 WARN ConfigUtils: The configuration key 
'hoodie.cleaner.policy.failed.writes' has been deprecated and may be removed in 
the future. Please use the new key 'hoodie.clean.failed.writes.policy' instead.
24/10/03 05:13:47 WARN ConfigUtils: The configuration key 
'hoodie.cleaner.policy.failed.writes' has been deprecated and may be removed in 
the future. Please use the new key 'hoodie.clean.failed.writes.policy' instead.
24/10/03 05:13:48 WARN HoodieSparkSqlWriterInternal: Closing write client

scala>

scala> val rsdf = spark.read.format("hudi").
     | option("hoodie.file.group.reader.enabled", "false").load(basePath)
rsdf: org.apache.spark.sql.DataFrame = [_hoodie_commit_time: string, 
_hoodie_commit_seqno: string ... 12 more fields]

scala> rsdf.createOrReplaceTempView("my_table")

scala> spark.sql("SELECT * FROM my_table LIMIT 1").show(false)
+-------------------+--------------------------+----------------------------------------+----------------------+--------------------------------------+----------------------------------------+----------+-------------+--------------------------------------------------------------------------------+------------+-------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----+
|_hoodie_commit_time|_hoodie_commit_seqno      |_hoodie_record_key              
        |_hoodie_partition_path|_hoodie_file_name                     |key      
                               |partition |ts           |textField              
                                                         
|decimalField|longField          |arrayField                                    
                                                                                
                                                                                
                                                                                
                        |mapField                                               
                                                                                
                                                                                
                                                 |round|
+-------------------+--------------------------+----------------------------------------+----------------------+--------------------------------------+----------------------------------------+----------+-------------+--------------------------------------------------------------------------------+------------+-------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----+
|20241003051313790  
|20241003051313790_1_632520|000-7e75df21-ad97-4208-accd-ff7702c28891|           
           
|6cee8ac1-21ab-4839-8244-b472fdcc3ebc-0|000-7e75df21-ad97-4208-accd-ff7702c28891|2024/02/12|1708030571930|abcdefghijklmnopqrstuvwxyz|abcdefghijklmnopqrstuvwxyz|abcdefghijklmnopqrstuvwxyz|0.05090767
  |8907629936308942339|[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 
16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 
36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 
56, 57, 58, 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 
76, 77, 78, 79]|{04fff22f-d275-4131-8b47-cce05f1aee0a -> -1571154779, 
5807913d-e19c-476e-92ac-9df1f886ca17 -> -2103664995, 
405e5bfb-bcf5-4e93-b14e-9447c3398175 -> -935547141, 
04b9a1cf-8a61-4b5c-9316-fa052d3544d9 -> -1902476851, 
55d66d36-6ba9-40e1-8eba-00b3fa37b923 -> -1834436483}|0    |
+-------------------+--------------------------+----------------------------------------+----------------------+--------------------------------------+----------------------------------------+----------+-------------+--------------------------------------------------------------------------------+------------+-------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----+

{code}



was (Author: JIRAUSER301185):
After disable fg reader, the query works.


{code:java}
scala> sdf.write.format("hudi").
     |   option("hoodie.table.name", "test_table").
     |   option("hoodie.datasource.write.recordkey.field", "key").
     |   option("hoodie.datasource.write.table.type", "MERGE_ON_READ").
     |   option("hoodie.datasource.write.operation", "upsert").
     |   option("hoodie.file.group.reader.enabled", "false").
     |   mode("Append").
     |   save(basePath)
24/10/03 05:13:13 WARN ConfigUtils: The configuration key 
'hoodie.cleaner.policy.failed.writes' has been deprecated and may be removed in 
the future. Please use the new key 'hoodie.clean.failed.writes.policy' instead.
24/10/03 05:13:14 WARN ConfigUtils: The configuration key 
'hoodie.cleaner.policy.failed.writes' has been deprecated and may be removed in 
the future. Please use the new key 'hoodie.clean.failed.writes.policy' instead.
24/10/03 05:13:14 WARN ConfigUtils: The configuration key 
'hoodie.cleaner.policy.failed.writes' has been deprecated and may be removed in 
the future. Please use the new key 'hoodie.clean.failed.writes.policy' instead.
24/10/03 05:13:42 WARN ConfigUtils: The configuration key 
'hoodie.cleaner.policy.failed.writes' has been deprecated and may be removed in 
the future. Please use the new key 'hoodie.clean.failed.writes.policy' instead.
24/10/03 05:13:43 WARN ConfigUtils: The configuration key 
'hoodie.cleaner.policy.failed.writes' has been deprecated and may be removed in 
the future. Please use the new key 'hoodie.clean.failed.writes.policy' instead.
24/10/03 05:13:47 WARN ConfigUtils: The configuration key 
'hoodie.cleaner.policy.failed.writes' has been deprecated and may be removed in 
the future. Please use the new key 'hoodie.clean.failed.writes.policy' instead.
24/10/03 05:13:48 WARN HoodieSparkSqlWriterInternal: Closing write client

scala>

scala> val rsdf = spark.read.format("hudi").
     | option("hoodie.file.group.reader.enabled", "false").load(basePath)
rsdf: org.apache.spark.sql.DataFrame = [_hoodie_commit_time: string, 
_hoodie_commit_seqno: string ... 12 more fields]

scala> rsdf.createOrReplaceTempView("my_table")

scala> spark.sql("SELECT * FROM my_table LIMIT 1").show(false)
{code}


> Fix IOException on the File group reader path
> ---------------------------------------------
>
>                 Key: HUDI-7276
>                 URL: https://issues.apache.org/jira/browse/HUDI-7276
>             Project: Apache Hudi
>          Issue Type: Bug
>          Components: spark
>            Reporter: xy
>            Assignee: Lin Liu
>            Priority: Blocker
>              Labels: hudi-1.0.0-beta2, pull-request-available
>             Fix For: 1.0.0
>
>
> FILE_GROUP_READER_ENABLED should be disable for query
>  
> java.io.IOException: com.esotericsoftware.kryo.KryoException: 
> java.lang.NullPointerException
> Serialization trace:
> props (org.apache.avro.Schema$LongSchema)
> types (org.apache.avro.Schema$UnionSchema)
> schema (org.apache.avro.Schema$Field)
> fieldMap (org.apache.avro.Schema$RecordSchema)
> at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1453)
> at 
> org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:226)
> at 
> org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:103)
> at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70)
> at 
> org.apache.spark.sql.execution.datasources.parquet.HoodieFileGroupReaderBasedParquetFileFormat.$anonfun$buildReaderWithPartitionValues$3(HoodieFileGroupReaderBasedParquetFileFormat.scala:149)
> at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:117)
> at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:165)
> at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:94)
> at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
> at 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.agg_doAggregateWithoutKey_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 scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
> at 
> org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:140)
> at 
> org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
> at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
> at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
> 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:1480)
> at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: com.esotericsoftware.kryo.KryoException: 
> java.lang.NullPointerException
> Serialization trace:
> props (org.apache.avro.Schema$LongSchema)
> types (org.apache.avro.Schema$UnionSchema)
> schema (org.apache.avro.Schema$Field)
> fieldMap (org.apache.avro.Schema$RecordSchema)
> at 
> com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:144)
> at 
> com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:543)
> at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:813)
> at 
> com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:134)
> at 
> com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:40)
> at com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:731)
> at 
> com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:125)
> at 
> com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:543)
> at com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:731)
> at 
> com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:125)
> at 
> com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:543)
> at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:813)
> at 
> com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:161)
> at 
> com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:39)
> at com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:731)
> at 
> com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:125)
> at 
> com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:543)
> at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:813)
> at 
> org.apache.spark.serializer.KryoDeserializationStream.readObject(KryoSerializer.scala:306)
> at 
> org.apache.spark.broadcast.TorrentBroadcast$.$anonfun$unBlockifyObject$4(TorrentBroadcast.scala:336)
> at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1480)
> at 
> org.apache.spark.broadcast.TorrentBroadcast$.unBlockifyObject(TorrentBroadcast.scala:338)
> at 
> org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$4(TorrentBroadcast.scala:257)
> at scala.Option.getOrElse(Option.scala:189)
> at 
> org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$2(TorrentBroadcast.scala:231)
> at org.apache.spark.util.KeyLock.withLock(KeyLock.scala:64)
> at 
> org.apache.spark.broadcast.TorrentBroadcast.$anonfun$readBroadcastBlock$1(TorrentBroadcast.scala:226)
> at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1446)
> ... 24 more
> Caused by: java.lang.NullPointerException
> at org.apache.avro.JsonProperties$2.putIfAbsent(JsonProperties.java:159)
> at org.apache.avro.JsonProperties$2.put(JsonProperties.java:166)
> at org.apache.avro.JsonProperties$2.put(JsonProperties.java:151)
> at 
> com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:162)
> at 
> com.esotericsoftware.kryo.serializers.MapSerializer.read(MapSerializer.java:39)
> at com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:731)
> at 
> com.esotericsoftware.kryo.serializers.ObjectField.read(ObjectField.java:125)
> ... 51 more
>  
> Driver stacktrace:
>  
>  
> display route: spark bulk insert,then merge into,at last query in sparksql
>  
> spark version: 3.2.0
> Hudi version:1.0(master)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to