codope commented on code in PR #11921: URL: https://github.com/apache/hudi/pull/11921#discussion_r1776588105
########## hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java: ########## @@ -914,6 +950,97 @@ private void validateAllColumnStats( validate(metadataBasedColStats, fsBasedColStats, partitionPath, "column stats"); } + private void validatePartitionStats( + HoodieMetadataValidationContext metadataTableBasedContext, + Set<String> baseDataFilesForCleaning, + List<String> allPartitions) throws Exception { + + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + HoodieData<HoodieMetadataColumnStats> partitionStatsUsingColStats = getPartitionStatsUsingColStats(metadataTableBasedContext, + baseDataFilesForCleaning, allPartitions, engineContext); + + TableSchemaResolver schemaResolver = new TableSchemaResolver(metaClient); + boolean enableMetadataTable = true; + HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() + .enable(enableMetadataTable) + .withMetadataIndexBloomFilter(enableMetadataTable) + .withMetadataIndexColumnStats(enableMetadataTable) + .withEnableRecordIndex(enableMetadataTable) + .build(); + PartitionStatsIndexSupport partitionStatsIndexSupport = + new PartitionStatsIndexSupport(engineContext.getSqlContext().sparkSession(), AvroConversionUtils.convertAvroSchemaToStructType(schemaResolver.getTableAvroSchema()), metadataConfig, metaClient, + false); + HoodieData<HoodieMetadataColumnStats> partitionStats = + partitionStatsIndexSupport.loadColumnStatsIndexRecords(JavaConverters.asScalaBufferConverter(metadataTableBasedContext.allColumnNameList).asScala().toSeq(), false); + JavaRDD<HoodieMetadataColumnStats> diffRDD = HoodieJavaRDD.getJavaRDD(partitionStats).subtract(HoodieJavaRDD.getJavaRDD(partitionStatsUsingColStats)); + if (!diffRDD.isEmpty()) { + List<HoodieMetadataColumnStats> diff = diffRDD.collect(); + Set<String> partitionPaths = diff.stream().map(HoodieMetadataColumnStats::getFileName).collect(Collectors.toSet()); + StringBuilder statDiffMsg = new StringBuilder(); + for (String partitionPath : partitionPaths) { + List<HoodieMetadataColumnStats> diffPartitionStatsColStats = partitionStatsUsingColStats.filter(stat -> stat.getFileName().equals(partitionPath)).collectAsList(); + List<HoodieMetadataColumnStats> diffPartitionStats = partitionStats.filter(stat -> stat.getFileName().equals(partitionPath)).collectAsList(); + statDiffMsg.append(String.format("Partition stats from MDT: %s from colstats: %s", Arrays.toString(diffPartitionStats.toArray()), Arrays.toString(diffPartitionStatsColStats.toArray()))); + } + throw new HoodieValidationException(String.format("Partition stats validation failed diff: %s", statDiffMsg)); + } + } + + private HoodieData<HoodieMetadataColumnStats> getPartitionStatsUsingColStats(HoodieMetadataValidationContext metadataTableBasedContext, Set<String> baseDataFilesForCleaning, + List<String> allPartitions, HoodieSparkEngineContext engineContext) { + return engineContext.parallelize(allPartitions).flatMap(partitionPath -> { + List<FileSlice> latestFileSlicesFromMetadataTable = filterFileSliceBasedOnInflightCleaning(metadataTableBasedContext.getSortedLatestFileSliceList(partitionPath), + baseDataFilesForCleaning); + List<String> latestFileNames = new ArrayList<>(); + latestFileSlicesFromMetadataTable.stream().filter(fs -> fs.getBaseFile().isPresent()).forEach(fs -> { + latestFileNames.add(fs.getBaseFile().get().getFileName()); + latestFileNames.addAll(fs.getLogFiles().map(HoodieLogFile::getFileName).collect(Collectors.toList())); + }); + List<HoodieColumnRangeMetadata<Comparable>> colStats = metadataTableBasedContext + .getSortedColumnStatsList(partitionPath, latestFileNames); + + TreeSet<HoodieColumnRangeMetadata<Comparable>> aggregatedColumnStats = aggregateColumnStats(partitionPath, colStats); + List<HoodieRecord> partitionStatRecords = HoodieMetadataPayload.createPartitionStatsRecords(partitionPath, new ArrayList<>(aggregatedColumnStats), false) + .collect(Collectors.toList()); + return partitionStatRecords.stream() + .map(record -> { + try { + return ((HoodieMetadataPayload) record.getData()).getInsertValue(null, null) + .map(metadataRecord -> ((HoodieMetadataRecord) metadataRecord).getColumnStatsMetadata()); + } catch (IOException e) { + throw new RuntimeException(e); + } + }) + .filter(Option::isPresent) + .map(Option::get) + .collect(Collectors.toList()) + .iterator(); + }); + } + + /** + * Generates aggregated column stats which also signify as partition stat for the particular partition + * path. + * + * @param partitionPath Provided partition path + * @param colStats Column stat records for the partition + */ + private static TreeSet<HoodieColumnRangeMetadata<Comparable>> aggregateColumnStats(String partitionPath, List<HoodieColumnRangeMetadata<Comparable>> colStats) { + TreeSet<HoodieColumnRangeMetadata<Comparable>> aggregatedColumnStats = new TreeSet<>(Comparator.comparing(HoodieColumnRangeMetadata::getColumnName)); + for (HoodieColumnRangeMetadata<Comparable> colStat : colStats) { Review Comment: let's UT this method. ########## hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java: ########## @@ -914,6 +950,97 @@ private void validateAllColumnStats( validate(metadataBasedColStats, fsBasedColStats, partitionPath, "column stats"); } + private void validatePartitionStats( + HoodieMetadataValidationContext metadataTableBasedContext, + Set<String> baseDataFilesForCleaning, + List<String> allPartitions) throws Exception { + + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + HoodieData<HoodieMetadataColumnStats> partitionStatsUsingColStats = getPartitionStatsUsingColStats(metadataTableBasedContext, + baseDataFilesForCleaning, allPartitions, engineContext); + + TableSchemaResolver schemaResolver = new TableSchemaResolver(metaClient); + boolean enableMetadataTable = true; + HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() + .enable(enableMetadataTable) + .withMetadataIndexBloomFilter(enableMetadataTable) + .withMetadataIndexColumnStats(enableMetadataTable) + .withEnableRecordIndex(enableMetadataTable) + .build(); + PartitionStatsIndexSupport partitionStatsIndexSupport = + new PartitionStatsIndexSupport(engineContext.getSqlContext().sparkSession(), AvroConversionUtils.convertAvroSchemaToStructType(schemaResolver.getTableAvroSchema()), metadataConfig, metaClient, + false); + HoodieData<HoodieMetadataColumnStats> partitionStats = + partitionStatsIndexSupport.loadColumnStatsIndexRecords(JavaConverters.asScalaBufferConverter(metadataTableBasedContext.allColumnNameList).asScala().toSeq(), false); + JavaRDD<HoodieMetadataColumnStats> diffRDD = HoodieJavaRDD.getJavaRDD(partitionStats).subtract(HoodieJavaRDD.getJavaRDD(partitionStatsUsingColStats)); + if (!diffRDD.isEmpty()) { + List<HoodieMetadataColumnStats> diff = diffRDD.collect(); + Set<String> partitionPaths = diff.stream().map(HoodieMetadataColumnStats::getFileName).collect(Collectors.toSet()); + StringBuilder statDiffMsg = new StringBuilder(); + for (String partitionPath : partitionPaths) { + List<HoodieMetadataColumnStats> diffPartitionStatsColStats = partitionStatsUsingColStats.filter(stat -> stat.getFileName().equals(partitionPath)).collectAsList(); + List<HoodieMetadataColumnStats> diffPartitionStats = partitionStats.filter(stat -> stat.getFileName().equals(partitionPath)).collectAsList(); + statDiffMsg.append(String.format("Partition stats from MDT: %s from colstats: %s", Arrays.toString(diffPartitionStats.toArray()), Arrays.toString(diffPartitionStatsColStats.toArray()))); + } + throw new HoodieValidationException(String.format("Partition stats validation failed diff: %s", statDiffMsg)); + } + } + + private HoodieData<HoodieMetadataColumnStats> getPartitionStatsUsingColStats(HoodieMetadataValidationContext metadataTableBasedContext, Set<String> baseDataFilesForCleaning, + List<String> allPartitions, HoodieSparkEngineContext engineContext) { + return engineContext.parallelize(allPartitions).flatMap(partitionPath -> { + List<FileSlice> latestFileSlicesFromMetadataTable = filterFileSliceBasedOnInflightCleaning(metadataTableBasedContext.getSortedLatestFileSliceList(partitionPath), + baseDataFilesForCleaning); + List<String> latestFileNames = new ArrayList<>(); + latestFileSlicesFromMetadataTable.stream().filter(fs -> fs.getBaseFile().isPresent()).forEach(fs -> { + latestFileNames.add(fs.getBaseFile().get().getFileName()); + latestFileNames.addAll(fs.getLogFiles().map(HoodieLogFile::getFileName).collect(Collectors.toList())); + }); + List<HoodieColumnRangeMetadata<Comparable>> colStats = metadataTableBasedContext + .getSortedColumnStatsList(partitionPath, latestFileNames); Review Comment: The argument is named as if only base files are being considered - https://github.com/apache/hudi/blob/30f077d0ded790cdd908a84247c265189683c743/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java#L1486 Can you check again that log files are considered? I think if colstats is present then it's fine otherwise it only considers base files. -- 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: commits-unsubscr...@hudi.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org