[ 
https://issues.apache.org/jira/browse/HUDI-7945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sagar Sumit updated HUDI-7945:
------------------------------
    Status: Patch Available  (was: In Progress)

> Fix partition pruning using PARTITION_STATS index in Spark
> ----------------------------------------------------------
>
>                 Key: HUDI-7945
>                 URL: https://issues.apache.org/jira/browse/HUDI-7945
>             Project: Apache Hudi
>          Issue Type: Improvement
>            Reporter: Ethan Guo
>            Assignee: Ethan Guo
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.0.0-beta2, 1.0.0
>
>
> The issue can be reproduced by 
> [https://github.com/apache/hudi/pull/11472#issuecomment-2199332859.]
> When there are more than one base files in a table partition, the 
> corresponding PARTITION_STATS index record in the metadata table contains 
> null as the file_path field in HoodieColumnRangeMetadata.
> {code:java}
> private static <T extends Comparable<T>> HoodieColumnRangeMetadata<T> 
> mergeRanges(HoodieColumnRangeMetadata<T> one,
>                                                                               
>     HoodieColumnRangeMetadata<T> another) {
>   
> ValidationUtils.checkArgument(one.getColumnName().equals(another.getColumnName()),
>       "Column names should be the same for merging column ranges");
>   final T minValue = getMinValueForColumnRanges(one, another);
>   final T maxValue = getMaxValueForColumnRanges(one, another);
>   return HoodieColumnRangeMetadata.create(
>       null, one.getColumnName(), minValue, maxValue,
>       one.getNullCount() + another.getNullCount(),
>       one.getValueCount() + another.getValueCount(),
>       one.getTotalSize() + another.getTotalSize(),
>       one.getTotalUncompressedSize() + another.getTotalUncompressedSize());
> } 
> {code}
> The null causes NPE when loading the column stats per partition from 
> PARTITION_STATS index.  Also, current implementation of 
> PartitionStatsIndexSupport assumes that the file_path field contains the 
> exact file name and it does not work if the the file path does not contain 
> null (even a list of file names stored does not work).  We have to 
> reimplement PartitionStatsIndexSupport so that it gives the pruned partitions 
> for further processing.
> {code:java}
> Caused by: java.lang.NullPointerException: element cannot be mapped to a null 
> key
>     at java.util.Objects.requireNonNull(Objects.java:228)
>     at java.util.stream.Collectors.lambda$groupingBy$45(Collectors.java:907)
>     at java.util.stream.ReduceOps$3ReducingSink.accept(ReduceOps.java:169)
>     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$2$1.accept(ReferencePipeline.java:175)
>     at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
>     at java.util.Iterator.forEachRemaining(Iterator.java:116)
>     at 
> java.util.Spliterators$IteratorSpliterator.forEachRemaining(Spliterators.java:1801)
>     at 
> java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:647)
>     at 
> java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:272)
>     at 
> java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1384)
>     at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
>     at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
>     at java.util.stream.ReduceOps$ReduceTask.doLeaf(ReduceOps.java:747)
>     at java.util.stream.ReduceOps$ReduceTask.doLeaf(ReduceOps.java:721)
>     at java.util.stream.AbstractTask.compute(AbstractTask.java:327)
>     at java.util.concurrent.CountedCompleter.exec(CountedCompleter.java:731)
>     at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289)
>     at java.util.concurrent.ForkJoinTask.doInvoke(ForkJoinTask.java:401)
>     at java.util.concurrent.ForkJoinTask.invoke(ForkJoinTask.java:734)
>     at 
> java.util.stream.ReduceOps$ReduceOp.evaluateParallel(ReduceOps.java:714)
>     at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:233)
>     at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:566)
>     at 
> org.apache.hudi.common.data.HoodieListPairData.groupByKey(HoodieListPairData.java:115)
>     at 
> org.apache.hudi.ColumnStatsIndexSupport.transpose(ColumnStatsIndexSupport.scala:253)
>     at 
> org.apache.hudi.ColumnStatsIndexSupport.$anonfun$loadTransposed$1(ColumnStatsIndexSupport.scala:149)
>     at 
> org.apache.hudi.HoodieCatalystUtils$.withPersistedData(HoodieCatalystUtils.scala:61)
>     at 
> org.apache.hudi.ColumnStatsIndexSupport.loadTransposed(ColumnStatsIndexSupport.scala:148)
>     at 
> org.apache.hudi.ColumnStatsIndexSupport.computeCandidateFileNames(ColumnStatsIndexSupport.scala:101)
>     at 
> org.apache.hudi.HoodieFileIndex.$anonfun$lookupCandidateFilesInMetadataTable$3(HoodieFileIndex.scala:354)
>     at 
> org.apache.hudi.HoodieFileIndex.$anonfun$lookupCandidateFilesInMetadataTable$3$adapted(HoodieFileIndex.scala:351)
>     at 
> scala.collection.TraversableLike$WithFilter.$anonfun$foreach$1(TraversableLike.scala:985)
>     at scala.collection.immutable.List.foreach(List.scala:431)
>     at 
> scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:984)
>     at 
> org.apache.hudi.HoodieFileIndex.$anonfun$lookupCandidateFilesInMetadataTable$1(HoodieFileIndex.scala:351)
>     at scala.util.Try$.apply(Try.scala:213)
>     at 
> org.apache.hudi.HoodieFileIndex.lookupCandidateFilesInMetadataTable(HoodieFileIndex.scala:338)
>     at 
> org.apache.hudi.HoodieFileIndex.filterFileSlices(HoodieFileIndex.scala:241)
>     ... 106 more {code}



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

Reply via email to