1
0

[HUDI-2778] Optimize statistics collection related codes and add some docs for z-order add fix some bugs (#4013)

* [HUDI-2778] Optimize statistics collection related codes and add more docs for z-order.

* add test code for multi-thread parquet footer read
This commit is contained in:
xiarixiaoyao
2021-11-23 13:46:02 +08:00
committed by GitHub
parent c88c2af8bf
commit 9de9951348
6 changed files with 175 additions and 39 deletions

View File

@@ -18,8 +18,6 @@
package org.apache.hudi.common.model;
import org.apache.parquet.schema.PrimitiveStringifier;
import java.util.Objects;
/**
@@ -30,16 +28,28 @@ public class HoodieColumnRangeMetadata<T> {
private final String columnName;
private final T minValue;
private final T maxValue;
private final long numNulls;
private final PrimitiveStringifier stringifier;
private long numNulls;
// For Decimal Type/Date Type, minValue/maxValue cannot represent it's original value.
// eg: when parquet collects column information, the decimal type is collected as int/binary type.
// so we cannot use minValue and maxValue directly, use minValueAsString/maxValueAsString instead.
private final String minValueAsString;
private final String maxValueAsString;
public HoodieColumnRangeMetadata(final String filePath, final String columnName, final T minValue, final T maxValue, final long numNulls, final PrimitiveStringifier stringifier) {
public HoodieColumnRangeMetadata(
final String filePath,
final String columnName,
final T minValue,
final T maxValue,
long numNulls,
final String minValueAsString,
final String maxValueAsString) {
this.filePath = filePath;
this.columnName = columnName;
this.minValue = minValue;
this.maxValue = maxValue;
this.numNulls = numNulls;
this.stringifier = stringifier;
this.numNulls = numNulls == -1 ? 0 : numNulls;
this.minValueAsString = minValueAsString;
this.maxValueAsString = maxValueAsString;
}
public String getFilePath() {
@@ -58,8 +68,12 @@ public class HoodieColumnRangeMetadata<T> {
return this.maxValue;
}
public PrimitiveStringifier getStringifier() {
return stringifier;
public String getMaxValueAsString() {
return maxValueAsString;
}
public String getMinValueAsString() {
return minValueAsString;
}
public long getNumNulls() {
@@ -79,12 +93,14 @@ public class HoodieColumnRangeMetadata<T> {
&& Objects.equals(getColumnName(), that.getColumnName())
&& Objects.equals(getMinValue(), that.getMinValue())
&& Objects.equals(getMaxValue(), that.getMaxValue())
&& Objects.equals(getNumNulls(), that.getNumNulls());
&& Objects.equals(getNumNulls(), that.getNumNulls())
&& Objects.equals(getMinValueAsString(), that.getMinValueAsString())
&& Objects.equals(getMaxValueAsString(), that.getMaxValueAsString());
}
@Override
public int hashCode() {
return Objects.hash(getColumnName(), getMinValue(), getMaxValue(), getNumNulls());
return Objects.hash(getColumnName(), getMinValue(), getMaxValue(), getNumNulls(), getMinValueAsString(), getMaxValueAsString());
}
@Override
@@ -94,6 +110,8 @@ public class HoodieColumnRangeMetadata<T> {
+ "columnName='" + columnName + '\''
+ ", minValue=" + minValue
+ ", maxValue=" + maxValue
+ ", numNulls=" + numNulls + '}';
+ ", numNulls=" + numNulls
+ ", minValueAsString=" + minValueAsString
+ ", minValueAsString=" + maxValueAsString + '}';
}
}

View File

@@ -39,6 +39,7 @@ import org.apache.parquet.hadoop.ParquetReader;
import org.apache.parquet.hadoop.metadata.BlockMetaData;
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
import org.apache.parquet.schema.MessageType;
import org.apache.parquet.schema.OriginalType;
import java.io.IOException;
import java.util.ArrayList;
@@ -56,6 +57,8 @@ import java.util.stream.Collectors;
*/
public class ParquetUtils extends BaseFileUtils {
private static Object lock = new Object();
/**
* 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.
@@ -283,17 +286,38 @@ public class ParquetUtils extends BaseFileUtils {
/**
* Parse min/max statistics stored in parquet footers for all columns.
* ParquetRead.readFooter is not a thread safe method.
*
* @param conf hadoop conf.
* @param parquetFilePath file to be read.
* @param cols cols which need to collect statistics.
* @return a HoodieColumnRangeMetadata instance.
*/
public Collection<HoodieColumnRangeMetadata<Comparable>> readRangeFromParquetMetadata(Configuration conf, Path parquetFilePath, List<String> cols) {
public Collection<HoodieColumnRangeMetadata<Comparable>> readRangeFromParquetMetadata(
Configuration conf,
Path parquetFilePath,
List<String> cols) {
ParquetMetadata metadata = readMetadata(conf, parquetFilePath);
// collect stats from all parquet blocks
Map<String, List<HoodieColumnRangeMetadata<Comparable>>> columnToStatsListMap = metadata.getBlocks().stream().flatMap(blockMetaData -> {
return blockMetaData.getColumns().stream().filter(f -> cols.contains(f.getPath().toDotString())).map(columnChunkMetaData ->
new HoodieColumnRangeMetadata<>(parquetFilePath.getName(), columnChunkMetaData.getPath().toDotString(),
columnChunkMetaData.getStatistics().genericGetMin(),
columnChunkMetaData.getStatistics().genericGetMax(),
columnChunkMetaData.getStatistics().getNumNulls(),
columnChunkMetaData.getPrimitiveType().stringifier()));
return blockMetaData.getColumns().stream().filter(f -> cols.contains(f.getPath().toDotString())).map(columnChunkMetaData -> {
String minAsString;
String maxAsString;
if (columnChunkMetaData.getPrimitiveType().getOriginalType() == OriginalType.DATE) {
synchronized (lock) {
minAsString = columnChunkMetaData.getStatistics().minAsString();
maxAsString = columnChunkMetaData.getStatistics().maxAsString();
}
} else {
minAsString = columnChunkMetaData.getStatistics().minAsString();
maxAsString = columnChunkMetaData.getStatistics().maxAsString();
}
return new HoodieColumnRangeMetadata<>(parquetFilePath.getName(), columnChunkMetaData.getPath().toDotString(),
columnChunkMetaData.getStatistics().genericGetMin(),
columnChunkMetaData.getStatistics().genericGetMax(),
columnChunkMetaData.getStatistics().getNumNulls(),
minAsString, maxAsString);
});
}).collect(Collectors.groupingBy(e -> e.getColumnName()));
// we only intend to keep file level statistics.
@@ -316,23 +340,41 @@ public class ParquetUtils extends BaseFileUtils {
HoodieColumnRangeMetadata<Comparable> range2) {
final Comparable minValue;
final Comparable maxValue;
final String minValueAsString;
final String maxValueAsString;
if (range1.getMinValue() != null && range2.getMinValue() != null) {
minValue = range1.getMinValue().compareTo(range2.getMinValue()) < 0 ? range1.getMinValue() : range2.getMinValue();
if (range1.getMinValue().compareTo(range2.getMinValue()) < 0) {
minValue = range1.getMinValue();
minValueAsString = range1.getMinValueAsString();
} else {
minValue = range2.getMinValue();
minValueAsString = range2.getMinValueAsString();
}
} else if (range1.getMinValue() == null) {
minValue = range2.getMinValue();
minValueAsString = range2.getMinValueAsString();
} else {
minValue = range1.getMinValue();
minValueAsString = range1.getMinValueAsString();
}
if (range1.getMaxValue() != null && range2.getMaxValue() != null) {
maxValue = range1.getMaxValue().compareTo(range2.getMaxValue()) < 0 ? range2.getMaxValue() : range1.getMaxValue();
if (range1.getMaxValue().compareTo(range2.getMaxValue()) < 0) {
maxValue = range2.getMaxValue();
maxValueAsString = range2.getMaxValueAsString();
} else {
maxValue = range1.getMaxValue();
maxValueAsString = range1.getMaxValueAsString();
}
} else if (range1.getMaxValue() == null) {
maxValue = range2.getMaxValue();
maxValueAsString = range2.getMaxValueAsString();
} else {
maxValue = range1.getMaxValue();
maxValueAsString = range1.getMaxValueAsString();
}
return new HoodieColumnRangeMetadata<>(range1.getFilePath(),
range1.getColumnName(), minValue, maxValue, range1.getNumNulls() + range2.getNumNulls(), range1.getStringifier());
range1.getColumnName(), minValue, maxValue, range1.getNumNulls() + range2.getNumNulls(), minValueAsString, maxValueAsString);
}
}