guanziyue opened a new issue #2648:
URL: https://github.com/apache/hudi/issues/2648


   **Describe the problem you faced**
   
   Hello guys, I meet a NPE when I use spark dataSource API to read a MOR 
table.  The stacktrace is attached at the end of post.
   
   Then I tried to find suspicious code by online debugging. The result being 
observed is shown as below.
   
   Let's start from the method buildFileIndex in MergeOnReadSnapshotRelation.
   
[https://github.com/apache/hudi/blob/master/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala#L136](url)
   
   Firstly, file status of all **parquet** files is fetched from 
InMemoryFileIndex at line 137.
   Then all **parquet** files is fetched as base file from 
HoodieTableFileSystemView at line 145 as latestFiles.
   
   After that, logic goes into groupLogsByBaseFile in 
HoodieRealtimeInputFormatUtils.
   
[https://github.com/apache/hudi/blob/master/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java#L139](url)
   
   At line 158, **all fileSlices** are returned with some having a parquet base 
file while others not (not be compacted).
   At line 166, for every file slice, hudi try to get base file for it by look 
up the map which only contains parquet base file id.
   When a file slice has not have a parquet base file yet , such looking up 
will result in NPE.
   
   Could any one please kindly point out which step has an unexpected result?
   **To Reproduce**
   The code I used to query is quite simple.
   
   > `SparkSession spark = SparkSession.builder()
                   .appName("hudi-read_guanziyue")
                   .enableHiveSupport()
                   .config("spark.serializer", 
"org.apache.spark.serializer.KryoSerializer")
                   .config("spark.sql.hive.convertMetastoreParquet", "false")
                   .config("spark.driver.allowMultipleContexts", true)
                   .config("spark.dynamicAllocation.enabled", true)
                   .config("spark.executor.memory", "30g")
                   .config("spark.executor.cores", "4")
                   .getOrCreate();
           Dataset<Row> queryDF = spark
                   .read()
                   .format("hudi")
                   .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY(), 
DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL())
                   .load(warehousePath + "/*");
           queryDF.createOrReplaceTempView("table");
           queryDF.show();`
   
   
   **Environment Description**
   
   * Hudi version :0.6.0
   
   * Spark version :3.0.1
   
   **Stacktrace**
   
   ```org.apache.hudi.exception.HoodieException: Error obtaining data file/log 
file grouping: hdfs://mytablePath/20210308
        at 
org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils.lambda$groupLogsByBaseFile$16(HoodieRealtimeInputFormatUtils.java:162)
        at java.util.HashMap$KeySet.forEach(HashMap.java:932)
        at 
org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(HoodieRealtimeInputFormatUtils.java:131)
        at 
org.apache.hudi.MergeOnReadSnapshotRelation.buildFileIndex(MergeOnReadSnapshotRelation.scala:139)
        at 
org.apache.hudi.MergeOnReadSnapshotRelation.<init>(MergeOnReadSnapshotRelation.scala:73)
        at org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:98)
        at org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:53)
        at 
org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:342)
        at 
org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:279)
        at 
org.apache.spark.sql.DataFrameReader.$anonfun$load$2(DataFrameReader.scala:268)
        at scala.Option.getOrElse(Option.scala:189)
        at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:268)
        at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:214)
        at hudiReadExample.main(hudiReadExample.java:32)
        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.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:742)
   Caused by: java.lang.NullPointerException
        at 
org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils.lambda$null$15(HoodieRealtimeInputFormatUtils.java:151)
        at 
java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:184)
        at 
java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
        at 
java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
        at 
java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:175)
        at 
java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
        at 
java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1374)
        at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:481)
        at 
java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:471)
        at 
java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:151)
        at 
java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:174)
        at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at 
java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:418)
        at 
org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils.lambda$groupLogsByBaseFile$16(HoodieRealtimeInputFormatUtils.java:149)```
   
   


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

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to