leesf commented on code in PR #11473:
URL: https://github.com/apache/hudi/pull/11473#discussion_r1682311733


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala:
##########
@@ -110,7 +110,9 @@ class 
HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState,
                                               hadoopConf: Configuration): 
PartitionedFile => Iterator[InternalRow] = {
     //dataSchema is not always right due to spark bugs
     val partitionColumns = partitionSchema.fieldNames
-    val dataSchema = 
StructType(tableSchema.structTypeSchema.fields.filterNot(f => 
partitionColumns.contains(f.name)))
+    val preCombineField = 
options.getOrElse(HoodieTableConfig.PRECOMBINE_FIELD.key, "")
+    val dataSchema = 
StructType(tableSchema.structTypeSchema.fields.filterNot(f => 
partitionColumns.contains(f.name)

Review Comment:
   > @leesf I can not reproduce exception from this code. What version of Spark 
do you use? Also, provide please full exception stack trace.
   
   @wombatu-kun I use spark 3.5.1 and hudi master branch, with the following 
exception
   ```
   Exception in thread "main" org.apache.spark.SparkException: Job aborted due 
to stage failure: Task 0 in stage 30.0 failed 4 times, most recent failure: 
Lost task 0.3 in stage 30.0 (TID 45) (21.9.11.158 executor 1): 
java.lang.IllegalArgumentException: Field: age does not exist in the table 
schema
   at 
org.apache.hudi.common.table.read.HoodieFileGroupReaderSchemaHandler.generateRequiredSchema(HoodieFileGroupReaderSchemaHandler.java:136)
   at 
org.apache.hudi.common.table.read.HoodieFileGroupReaderSchemaHandler.prepareRequiredSchema(HoodieFileGroupReaderSchemaHandler.java:151)
   at 
org.apache.hudi.common.table.read.HoodiePositionBasedSchemaHandler.prepareRequiredSchema(HoodiePositionBasedSchemaHandler.java:49)
   at 
org.apache.hudi.common.table.read.HoodieFileGroupReaderSchemaHandler.<init>(HoodieFileGroupReaderSchemaHandler.java:85)
   at 
org.apache.hudi.common.table.read.HoodiePositionBasedSchemaHandler.<init>(HoodiePositionBasedSchemaHandler.java:44)
   at 
org.apache.hudi.common.table.read.HoodieFileGroupReader.<init>(HoodieFileGroupReader.java:113)
   at 
org.apache.spark.sql.execution.datasources.parquet.HoodieFileGroupReaderBasedParquetFileFormat.$anonfun$buildReaderWithPartitionValues$3(HoodieFileGroupReaderBasedParquetFileFormat.scala:171)
   at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:217)
   at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:279)
   at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:129)
   at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
   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.WholeStageCodegenEvaluatorFactory$WholeStageCodegenPartitionEvaluator$$anon$1.hasNext(WholeStageCodegenEvaluatorFactory.scala:43)
   at scala.collection.Iterator.isEmpty(Iterator.scala:387)
   at scala.collection.Iterator.isEmpty$(Iterator.scala:387)
   at 
org.apache.spark.sql.execution.WholeStageCodegenEvaluatorFactory$WholeStageCodegenPartitionEvaluator$$anon$1.isEmpty(WholeStageCodegenEvaluatorFactory.scala:41)
   at 
org.apache.hudi.HoodieSparkUtils$.$anonfun$createRdd$2(HoodieSparkUtils.scala:112)
   at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:858)
   at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:858)
   at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:367)
   at org.apache.spark.rdd.RDD.iterator(RDD.scala:331)
   at 
org.apache.spark.sql.execution.SQLConfInjectingRDD.compute(SQLConfInjectingRDD.scala:58)
   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:367)
   at org.apache.spark.rdd.RDD.iterator(RDD.scala:331)
   at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:367)
   at org.apache.spark.rdd.RDD.$anonfun$getOrCompute$1(RDD.scala:381)
   at 
org.apache.spark.storage.BlockManager.$anonfun$getOrElseUpdate$1(BlockManager.scala:1372)
   at 
org.apache.spark.storage.BlockManager.$anonfun$doPutIterator$1(BlockManager.scala:1614)
   at 
org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$doPut(BlockManager.scala:1524)
   at 
org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1588)
   at 
org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:1389)
   at 
org.apache.spark.storage.BlockManager.getOrElseUpdateRDDBlock(BlockManager.scala:1343)
   at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:379)
   at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
   at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:367)
   at org.apache.spark.rdd.RDD.iterator(RDD.scala:331)
   at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:367)
   at org.apache.spark.rdd.RDD.iterator(RDD.scala:331)
   at 
org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
   at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:104)
   at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:54)
   at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:166)
   at org.apache.spark.scheduler.Task.run(Task.scala:141)
   at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:620)
   at 
org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64)
   at 
org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61)
   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:94)
   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:623)
   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:750)
   
   Driver stacktrace:
   at 
org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:2856)
   at 
org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2(DAGScheduler.scala:2792)
   at 
org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2$adapted(DAGScheduler.scala:2791)
   at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
   at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
   at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
   at 
org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:2791)
   at 
org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1(DAGScheduler.scala:1247)
   at 
org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1$adapted(DAGScheduler.scala:1247)
   at scala.Option.foreach(Option.scala:407)
   at 
org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:1247)
   at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:3060)
   at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2994)
   at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2983)
   at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
   at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:989)
   at org.apache.spark.SparkContext.runJob(SparkContext.scala:2398)
   at org.apache.spark.SparkContext.runJob(SparkContext.scala:2419)
   at org.apache.spark.SparkContext.runJob(SparkContext.scala:2438)
   at org.apache.spark.SparkContext.runJob(SparkContext.scala:2463)
   at org.apache.spark.rdd.RDD.$anonfun$collect$1(RDD.scala:1049)
   at 
org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
   at 
org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
   at org.apache.spark.rdd.RDD.withScope(RDD.scala:410)
   at org.apache.spark.rdd.RDD.collect(RDD.scala:1048)
   at 
org.apache.spark.rdd.PairRDDFunctions.$anonfun$countByKey$1(PairRDDFunctions.scala:367)
   at 
org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
   at 
org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
   at org.apache.spark.rdd.RDD.withScope(RDD.scala:410)
   at 
org.apache.spark.rdd.PairRDDFunctions.countByKey(PairRDDFunctions.scala:367)
   at org.apache.spark.api.java.JavaPairRDD.countByKey(JavaPairRDD.scala:314)
   at 
org.apache.hudi.data.HoodieJavaPairRDD.countByKey(HoodieJavaPairRDD.java:108)
   at 
org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor.buildProfile(BaseSparkCommitActionExecutor.java:193)
   at 
org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor.execute(BaseSparkCommitActionExecutor.java:164)
   at 
org.apache.hudi.table.action.deltacommit.SparkUpsertPreppedDeltaCommitActionExecutor.execute(SparkUpsertPreppedDeltaCommitActionExecutor.java:44)
   at 
org.apache.hudi.table.HoodieSparkMergeOnReadTable.upsertPrepped(HoodieSparkMergeOnReadTable.java:126)
   at 
org.apache.hudi.table.HoodieSparkMergeOnReadTable.upsertPrepped(HoodieSparkMergeOnReadTable.java:88)
   at 
org.apache.hudi.client.SparkRDDWriteClient.upsertPreppedRecords(SparkRDDWriteClient.java:156)
   at org.apache.hudi.DataSourceUtils.doWriteOperation(DataSourceUtils.java:222)
   at 
org.apache.hudi.HoodieSparkSqlWriterInternal.liftedTree1$1(HoodieSparkSqlWriter.scala:515)
   at 
org.apache.hudi.HoodieSparkSqlWriterInternal.writeInternal(HoodieSparkSqlWriter.scala:513)
   at 
org.apache.hudi.HoodieSparkSqlWriterInternal.$anonfun$write$1(HoodieSparkSqlWriter.scala:188)
   at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:125)
   at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:201)
   at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:108)
   at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:901)
   at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:66)
   at 
org.apache.spark.sql.adapter.BaseSpark3Adapter.sqlExecutionWithNewExecutionId(BaseSpark3Adapter.scala:107)
   at 
org.apache.hudi.HoodieSparkSqlWriterInternal.write(HoodieSparkSqlWriter.scala:210)
   at 
org.apache.hudi.HoodieSparkSqlWriter$.write(HoodieSparkSqlWriter.scala:125)
   at org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:162)
   at 
org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:48)
   at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:75)
   at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:73)
   at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:84)
   at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:107)
   at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:125)
   at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:201)
   at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:108)
   at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:901)
   at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:66)
   at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:107)
   at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:98)
   at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:461)
   at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:76)
   at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:461)
   at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:32)
   at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
   at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
   at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
   at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
   at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:437)
   at 
org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:98)
   at 
org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:85)
   at 
org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:83)
   at 
org.apache.spark.sql.execution.QueryExecution.assertCommandExecuted(QueryExecution.scala:142)
   at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:859)
   at 
org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:388)
   at 
org.apache.spark.sql.DataFrameWriter.saveInternal(DataFrameWriter.scala:361)
   at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:248)
   at 
org.apache.spark.sql.hudi.command.UpdateHoodieTableCommand.run(UpdateHoodieTableCommand.scala:79)
   at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:75)
   at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:73)
   at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:84)
   at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:107)
   at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:125)
   at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:201)
   at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:108)
   at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:901)
   at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:66)
   at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:107)
   at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:98)
   at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:461)
   at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:76)
   at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:461)
   at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:32)
   at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
   at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
   at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
   at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32)
   at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:437)
   at 
org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:98)
   at 
org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:85)
   at 
org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:83)
   at org.apache.spark.sql.Dataset.<init>(Dataset.scala:220)
   at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:100)
   at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:901)
   at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:97)
   at org.apache.spark.sql.SparkSession.$anonfun$sql$4(SparkSession.scala:692)
   at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:901)
   at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:683)
   at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:714)
   at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:745)
   at 
org.apache.spark.sql.HiveResponseProxy$.hiveResponseProxy(HiveResponseProxy.scala:11)
   at 
com.aliyun.adb.spark.sql.OfflineSqlTemplate$.$anonfun$main$1(OfflineSqlTemplate.scala:31)
   at 
com.aliyun.adb.spark.sql.OfflineSqlTemplate$.$anonfun$main$1$adapted(OfflineSqlTemplate.scala:25)
   at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36)
   at 
scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33)
   at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:198)
   at 
com.aliyun.adb.spark.sql.OfflineSqlTemplate$.main(OfflineSqlTemplate.scala:25)
   at com.aliyun.adb.spark.sql.OfflineSqlTemplate.main(OfflineSqlTemplate.scala)
   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   at java.lang.reflect.Method.invoke(Method.java:498)
   at 
org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
   at 
org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:1029)
   at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:194)
   at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:217)
   at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:91)
   at 
org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1120)
   at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1129)
   at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
   Caused by: java.lang.IllegalArgumentException: Field: age does not exist in 
the table schema
   at 
org.apache.hudi.common.table.read.HoodieFileGroupReaderSchemaHandler.generateRequiredSchema(HoodieFileGroupReaderSchemaHandler.java:136)
   at 
org.apache.hudi.common.table.read.HoodieFileGroupReaderSchemaHandler.prepareRequiredSchema(HoodieFileGroupReaderSchemaHandler.java:151)
   at 
org.apache.hudi.common.table.read.HoodiePositionBasedSchemaHandler.prepareRequiredSchema(HoodiePositionBasedSchemaHandler.java:49)
   at 
org.apache.hudi.common.table.read.HoodieFileGroupReaderSchemaHandler.<init>(HoodieFileGroupReaderSchemaHandler.java:85)
   at 
org.apache.hudi.common.table.read.HoodiePositionBasedSchemaHandler.<init>(HoodiePositionBasedSchemaHandler.java:44)
   at 
org.apache.hudi.common.table.read.HoodieFileGroupReader.<init>(HoodieFileGroupReader.java:113)
   at 
org.apache.spark.sql.execution.datasources.parquet.HoodieFileGroupReaderBasedParquetFileFormat.$anonfun$buildReaderWithPartitionValues$3(HoodieFileGroupReaderBasedParquetFileFormat.scala:171)
   at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:217)
   at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:279)
   at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:129)
   at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
   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.WholeStageCodegenEvaluatorFactory$WholeStageCodegenPartitionEvaluator$$anon$1.hasNext(WholeStageCodegenEvaluatorFactory.scala:43)
   at scala.collection.Iterator.isEmpty(Iterator.scala:387)
   at scala.collection.Iterator.isEmpty$(Iterator.scala:387)
   at 
org.apache.spark.sql.execution.WholeStageCodegenEvaluatorFactory$WholeStageCodegenPartitionEvaluator$$anon$1.isEmpty(WholeStageCodegenEvaluatorFactory.scala:41)
   at 
org.apache.hudi.HoodieSparkUtils$.$anonfun$createRdd$2(HoodieSparkUtils.scala:112)
   at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:858)
   at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2$adapted(RDD.scala:858)
   at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:367)
   at org.apache.spark.rdd.RDD.iterator(RDD.scala:331)
   at 
org.apache.spark.sql.execution.SQLConfInjectingRDD.compute(SQLConfInjectingRDD.scala:58)
   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:367)
   at org.apache.spark.rdd.RDD.iterator(RDD.scala:331)
   at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:367)
   at org.apache.spark.rdd.RDD.$anonfun$getOrCompute$1(RDD.scala:381)
   at 
org.apache.spark.storage.BlockManager.$anonfun$getOrElseUpdate$1(BlockManager.scala:1372)
   at 
org.apache.spark.storage.BlockManager.$anonfun$doPutIterator$1(BlockManager.scala:1614)
   at 
org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$doPut(BlockManager.scala:1524)
   at 
org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1588)
   at 
org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:1389)
   at 
org.apache.spark.storage.BlockManager.getOrElseUpdateRDDBlock(BlockManager.scala:1343)
   at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:379)
   at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
   at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:367)
   at org.apache.spark.rdd.RDD.iterator(RDD.scala:331)
   at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:367)
   at org.apache.spark.rdd.RDD.iterator(RDD.scala:331)
   at 
org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
   at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:104)
   at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:54)
   at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:166)
   at org.apache.spark.scheduler.Task.run(Task.scala:141)
   at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:620)
   at 
org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64)
   at 
org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61)
   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:94)
   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:623)
   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:750)
   ```



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

Reply via email to