1
0

[HUDI-3465] Add validation of column stats and bloom filters in HoodieMetadataTableValidator (#4878)

This commit is contained in:
Y Ethan Guo
2022-02-28 18:49:30 -08:00
committed by GitHub
parent 44b8ab6048
commit 257052a94d
4 changed files with 424 additions and 79 deletions

View File

@@ -105,7 +105,7 @@ public class HoodieColumnRangeMetadata<T> {
public String toString() {
return "HoodieColumnRangeMetadata{"
+ "filePath ='" + filePath + '\''
+ "columnName='" + columnName + '\''
+ ", columnName='" + columnName + '\''
+ ", minValue=" + minValue
+ ", maxValue=" + maxValue
+ ", nullCount=" + nullCount

View File

@@ -18,10 +18,6 @@
package org.apache.hudi.common.util;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieColumnRangeMetadata;
@@ -30,6 +26,11 @@ import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.MetadataNotFoundException;
import org.apache.hudi.keygen.BaseKeyGenerator;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.parquet.avro.AvroParquetReader;
@@ -46,6 +47,7 @@ import org.apache.parquet.schema.OriginalType;
import org.apache.parquet.schema.PrimitiveType;
import javax.annotation.Nonnull;
import java.io.IOException;
import java.math.BigDecimal;
import java.math.BigInteger;
@@ -80,6 +82,17 @@ public class ParquetUtils extends BaseFileUtils {
return filterParquetRowKeys(configuration, filePath, filter, HoodieAvroUtils.getRecordKeySchema());
}
public static ParquetMetadata readMetadata(Configuration conf, Path parquetFilePath) {
ParquetMetadata footer;
try {
// TODO(vc): Should we use the parallel reading version here?
footer = ParquetFileReader.readFooter(FSUtils.getFs(parquetFilePath.toString(), conf).getConf(), parquetFilePath);
} catch (IOException e) {
throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath, e);
}
return footer;
}
/**
* Read the rowKey list matching the given filter, from the given parquet file. If the filter is empty, then this will
* return all the rowkeys.
@@ -184,17 +197,6 @@ public class ParquetUtils extends BaseFileUtils {
}
}
public ParquetMetadata readMetadata(Configuration conf, Path parquetFilePath) {
ParquetMetadata footer;
try {
// TODO(vc): Should we use the parallel reading version here?
footer = ParquetFileReader.readFooter(FSUtils.getFs(parquetFilePath.toString(), conf).getConf(), parquetFilePath);
} catch (IOException e) {
throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath, e);
}
return footer;
}
/**
* Get the schema of the given parquet file.
*/