silencily opened a new issue, #7827:
URL: https://github.com/apache/hudi/issues/7827

   **Describe the problem you faced**
   
   I caught a exception when i query a hudi table by presto:Could not create 
HoodieRealtimeRecordReader on path hdfs://**/**.parquet
   Query sql: select count(1) from table_name
   Table queried are maintained by ddl statements.
   I discover that if the table contains deltalogs,it is ok. exception are 
thrown only when deltalogs not exist.
   
   **To Reproduce**
   
   Steps to reproduce the behavior:
   
   1.Create hudi table and maintain by ddl statement 
   hoodie.compact.inline=true
   hoodie.compact.inline.max.delta.commits=3
   
   create table test_case_10 (
    id int,
    name string,
    price double,
    ts long,
    dt string
   ) using hudi
    tblproperties(
    primaryKey = 'id',
    preCombineField = 'ts',
    type = 'mor'
   );
   insert into test_case_10 values
   (1,'hudi1',10,100,'2021-05-05'),
   (2,'hudi2',10,100,'2021-05-05');
     
   update test_case_10 set name = 'hudi1-1' where id=1;
   ALTER TABLE test_case_10 ADD COLUMNS(ext0 string);
   insert into test_case_10 values
   (3,'hudi3',10.88,100,'2021-05-05','added1'),
   (4,'hudi4',10,101,'2021-06-07','added2'),
   (5,'hudi5',20.5,101,'2021-06-07','added3');
     
   update test_case_10 set price=100.2 where id=4;
   insert into test_case_10 values (6,'hudi6',10.88,100,'2021-05-05','added6');
   
   2.Query by presto
   select count(1) from test_case_10_rt;
   
   3.Caught a exception
   Query 20230202_123728_00019_mc55h failed: Could not create 
HoodieRealtimeRecordReader on path 
hdfs://user/data/hudi_ddl/test_case_10/cd6b41da-9df0-4d08-af25-990b5951d4b6-0_0-40200-3525900_20230202092059311.parquet
   org.apache.hudi.exception.HoodieException: Could not create 
HoodieRealtimeRecordReader on path 
hdfs://user/data/hudi_ddl/test_case_10/cd6b41da-9df0-4d08-af25-990b5951d4b6-0_0-40200-3525900_20230202092059311.parquet
           at 
org.apache.hudi.hadoop.realtime.AbstractRealtimeRecordReader.<init>(AbstractRealtimeRecordReader.java:82)
           at 
org.apache.hudi.hadoop.realtime.RealtimeCompactedRecordReader.<init>(RealtimeCompactedRecordReader.java:63)
           at 
org.apache.hudi.hadoop.realtime.HoodieRealtimeRecordReader.constructRecordReader(HoodieRealtimeRecordReader.java:70)
           at 
org.apache.hudi.hadoop.realtime.HoodieRealtimeRecordReader.<init>(HoodieRealtimeRecordReader.java:47)
           at 
org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat.getRecordReader(HoodieParquetRealtimeInputFormat.java:74)
           at 
io.prestosql.plugin.hive.util.HiveUtil.createRecordReader(HiveUtil.java:289)
           at 
io.prestosql.plugin.hive.GenericHiveRecordCursorProvider.lambda$createRecordCursor$0(GenericHiveRecordCursorProvider.java:100)
           at 
io.prestosql.plugin.hive.authentication.UserGroupInformationUtils.lambda$executeActionInDoAs$0(UserGroupInformationUtils.java:29)
           at java.base/java.security.AccessController.doPrivileged(Native 
Method)
           at java.base/javax.security.auth.Subject.doAs(Subject.java:361)
           at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1710)
           at 
io.prestosql.plugin.hive.authentication.UserGroupInformationUtils.executeActionInDoAs(UserGroupInformationUtils.java:27)
           at 
io.prestosql.plugin.hive.authentication.ImpersonatingHdfsAuthentication.doAs(ImpersonatingHdfsAuthentication.java:39)
           at 
io.prestosql.plugin.hive.HdfsEnvironment.doAs(HdfsEnvironment.java:89)
           at 
io.prestosql.plugin.hive.GenericHiveRecordCursorProvider.createRecordCursor(GenericHiveRecordCursorProvider.java:99)
           at 
io.prestosql.plugin.hive.HivePageSourceProvider.createHivePageSource(HivePageSourceProvider.java:215)
           at 
io.prestosql.plugin.hive.HivePageSourceProvider.createPageSource(HivePageSourceProvider.java:106)
           at 
io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorPageSourceProvider.createPageSource(ClassLoaderSafeConnectorPageSourceProvider.java:57)
           at 
io.prestosql.split.PageSourceManager.createPageSource(PageSourceManager.java:64)
           at 
io.prestosql.operator.TableScanOperator.getOutput(TableScanOperator.java:299)
           at io.prestosql.operator.Driver.processInternal(Driver.java:379)
           at io.prestosql.operator.Driver.lambda$processFor$8(Driver.java:283)
           at io.prestosql.operator.Driver.tryWithLock(Driver.java:675)
           at io.prestosql.operator.Driver.processFor(Driver.java:276)
           at 
io.prestosql.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:1075)
           at 
io.prestosql.execution.executor.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:163)
           at 
io.prestosql.execution.executor.TaskExecutor$TaskRunner.run(TaskExecutor.java:484)
           at 
io.prestosql.$gen.Presto_333_3_2_gc8136e6_dirty____20230202_101641_2.run(Unknown
 Source)
           at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
           at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
           at java.base/java.lang.Thread.run(Thread.java:834)
   Caused by: java.lang.IllegalArgumentException: cannot prune col:  which not 
exisit in hudi table
           at 
org.apache.hudi.internal.schema.utils.InternalSchemaUtils.lambda$pruneInternalSchema$0(InternalSchemaUtils.java:60)
           at 
java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195)
           at 
java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1655)
           at 
java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484)
           at 
java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474)
           at 
java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913)
           at 
java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
           at 
java.base/java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578)
           at 
org.apache.hudi.internal.schema.utils.InternalSchemaUtils.pruneInternalSchema(InternalSchemaUtils.java:63)
           at 
org.apache.hudi.hadoop.SchemaEvolutionContext.doEvolutionForRealtimeInputFormat(SchemaEvolutionContext.java:136)
           at 
org.apache.hudi.hadoop.realtime.AbstractRealtimeRecordReader.<init>(AbstractRealtimeRecordReader.java:77)
           ... 30 more
   
   **Expected behavior**
   
   no errors
   
   **Environment Description**
   
   * Hudi version : 0.11.1
   
   * Spark version :3.2.1
   
   * Hive version :2.3.9
   
   * Hadoop version :2.7.3
   
   * Storage (HDFS/S3/GCS..) : HDFS
   
   * Presto : 333
   
   * Running on Docker? (yes/no) : no
   
   
   


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