[HUDI-2814] Addressing issues w/ Z-order Layout Optimization (#4060)
* `ZCurveOptimizeHelper` > `ZOrderingIndexHelper`; Moved Z-index helper under `hudi.index.zorder` package * Tidying up `ZOrderingIndexHelper` * Fixing compilation * Fixed index new/original table merging sequence to always prefer values from new index; Cleaned up `HoodieSparkUtils` * Added test for `mergeIndexSql` * Abstracted Z-index name composition w/in `ZOrderingIndexHelper`; * Fixed `DataSkippingUtils` to interrupt prunning in case data filter contains non-indexed column reference * Properly handle exceptions origination during pruning in `HoodieFileIndex` * Make sure no errors are logged upon encountering `AnalysisException` * Cleaned up Z-index updating sequence; Tidying up comments, java-docs; * Fixed Z-index to properly handle changes of the list of clustered columns * Tidying up * `lint` * Suppressing `JavaDocStyle` first sentence check * Fixed compilation * Fixing incorrect `DecimalType` conversion * Refactored test `TestTableLayoutOptimization` - Added Z-index table composition test (against fixtures) - Separated out GC test; Tidying up * Fixed tests re-shuffling column order for Z-Index table `DataFrame` to align w/ the one by one loaded from JSON * Scaffolded `DataTypeUtils` to do basic checks of Spark types; Added proper compatibility checking b/w old/new index-tables * Added test for Z-index tables merging * Fixed import being shaded by creating internal `hudi.util` package * Fixed packaging for `TestOptimizeTable` * Revised `updateMetadataIndex` seq to provide Z-index updating process w/ source table schema * Make sure existing Z-index table schema is sync'd to source table's one * Fixed shaded refs * Fixed tests * Fixed type conversion of Parquet provided metadata values into Spark expected schemas * Fixed `composeIndexSchema` utility to propose proper schema * Added more tests for Z-index: - Checking that Z-index table is built correctly - Checking that Z-index tables are merged correctly (during update) * Fixing source table * Fixing tests to read from Parquet w/ proper schema * Refactored `ParquetUtils` utility reading stats from Parquet footers * Fixed incorrect handling of Decimals extracted from Parquet footers * Worked around issues in javac failign to compile stream's collection * Fixed handling of `Date` type * Fixed handling of `DateType` to be parsed as `LocalDate` * Updated fixture; Make sure test loads Z-index fixture using proper schema * Removed superfluous scheme adjusting when reading from Parquet, since Spark is actually able to perfectly restore schema (given Parquet was previously written by Spark as well) * Fixing race-condition in Parquet's `DateStringifier` trying to share `SimpleDataFormat` object which is inherently not thread-safe * Tidying up * Make sure schema is used upon reading to validate input files are in the appropriate format; Tidying up; * Worked around javac (1.8) inability to infer expression type properly * Updated fixtures; Tidying up * Fixing compilation after rebase * Assert clustering have in Z-order layout optimization testing * Tidying up exception messages * XXX * Added test validating Z-index lookup filter correctness * Added more test-cases; Tidying up * Added tests for string expressions * Fixed incorrect Z-index filter lookup translations * Added more test-cases * Added proper handling on complex negations of AND/OR expressions by pushing NOT operator down into inner expressions for appropriate handling * Added `-target:jvm-1.8` for `hudi-spark` module * Adding more tests * Added tests for non-indexed columns * Properly handle non-indexed columns by falling back to a re-write of containing expression as `TrueLiteral` instead * Fixed tests * Removing the parquet test files and disabling corresponding tests Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
@@ -18,6 +18,8 @@
|
||||
|
||||
package org.apache.hudi.common.model;
|
||||
|
||||
import org.apache.parquet.schema.PrimitiveStringifier;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
@@ -28,28 +30,16 @@ public class HoodieColumnRangeMetadata<T> {
|
||||
private final String columnName;
|
||||
private final T minValue;
|
||||
private final T maxValue;
|
||||
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;
|
||||
private final long numNulls;
|
||||
private 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) {
|
||||
public HoodieColumnRangeMetadata(final String filePath, final String columnName, final T minValue, final T maxValue, final long numNulls, final PrimitiveStringifier stringifier) {
|
||||
this.filePath = filePath;
|
||||
this.columnName = columnName;
|
||||
this.minValue = minValue;
|
||||
this.maxValue = maxValue;
|
||||
this.numNulls = numNulls == -1 ? 0 : numNulls;
|
||||
this.minValueAsString = minValueAsString;
|
||||
this.maxValueAsString = maxValueAsString;
|
||||
this.numNulls = numNulls;
|
||||
this.stringifier = stringifier;
|
||||
}
|
||||
|
||||
public String getFilePath() {
|
||||
@@ -68,12 +58,8 @@ public class HoodieColumnRangeMetadata<T> {
|
||||
return this.maxValue;
|
||||
}
|
||||
|
||||
public String getMaxValueAsString() {
|
||||
return maxValueAsString;
|
||||
}
|
||||
|
||||
public String getMinValueAsString() {
|
||||
return minValueAsString;
|
||||
public PrimitiveStringifier getStringifier() {
|
||||
return stringifier;
|
||||
}
|
||||
|
||||
public long getNumNulls() {
|
||||
@@ -93,14 +79,12 @@ 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(getMinValueAsString(), that.getMinValueAsString())
|
||||
&& Objects.equals(getMaxValueAsString(), that.getMaxValueAsString());
|
||||
&& Objects.equals(getNumNulls(), that.getNumNulls());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(getColumnName(), getMinValue(), getMaxValue(), getNumNulls(), getMinValueAsString(), getMaxValueAsString());
|
||||
return Objects.hash(getColumnName(), getMinValue(), getMaxValue(), getNumNulls());
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -110,8 +94,6 @@ public class HoodieColumnRangeMetadata<T> {
|
||||
+ "columnName='" + columnName + '\''
|
||||
+ ", minValue=" + minValue
|
||||
+ ", maxValue=" + maxValue
|
||||
+ ", numNulls=" + numNulls
|
||||
+ ", minValueAsString=" + minValueAsString
|
||||
+ ", minValueAsString=" + maxValueAsString + '}';
|
||||
+ ", numNulls=" + numNulls + '}';
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,6 +18,10 @@
|
||||
|
||||
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;
|
||||
@@ -26,11 +30,6 @@ 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.parquet.avro.AvroParquetReader;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
@@ -38,12 +37,15 @@ import org.apache.parquet.hadoop.ParquetFileReader;
|
||||
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.DecimalMetadata;
|
||||
import org.apache.parquet.schema.MessageType;
|
||||
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.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
@@ -51,14 +53,13 @@ import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Utility functions involving with parquet.
|
||||
*/
|
||||
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.
|
||||
@@ -286,95 +287,97 @@ 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 List<HoodieColumnRangeMetadata<Comparable>> readRangeFromParquetMetadata(
|
||||
@Nonnull Configuration conf,
|
||||
@Nonnull Path parquetFilePath,
|
||||
@Nonnull 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 -> {
|
||||
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()));
|
||||
// Collect stats from all individual Parquet blocks
|
||||
Map<String, List<HoodieColumnRangeMetadata<Comparable>>> columnToStatsListMap = metadata.getBlocks().stream().sequential()
|
||||
.flatMap(blockMetaData -> blockMetaData.getColumns().stream()
|
||||
.filter(f -> cols.contains(f.getPath().toDotString()))
|
||||
.map(columnChunkMetaData ->
|
||||
new HoodieColumnRangeMetadata<Comparable>(
|
||||
parquetFilePath.getName(),
|
||||
columnChunkMetaData.getPath().toDotString(),
|
||||
convertToNativeJavaType(
|
||||
columnChunkMetaData.getPrimitiveType(),
|
||||
columnChunkMetaData.getStatistics().genericGetMin()),
|
||||
convertToNativeJavaType(
|
||||
columnChunkMetaData.getPrimitiveType(),
|
||||
columnChunkMetaData.getStatistics().genericGetMax()),
|
||||
columnChunkMetaData.getStatistics().getNumNulls(),
|
||||
columnChunkMetaData.getPrimitiveType().stringifier()))
|
||||
).collect(Collectors.groupingBy(HoodieColumnRangeMetadata::getColumnName));
|
||||
|
||||
// we only intend to keep file level statistics.
|
||||
return new ArrayList<>(columnToStatsListMap.values().stream()
|
||||
.map(blocks -> getColumnRangeInFile(blocks))
|
||||
.collect(Collectors.toList()));
|
||||
// Combine those into file-level statistics
|
||||
// NOTE: Inlining this var makes javac (1.8) upset (due to its inability to infer
|
||||
// expression type correctly)
|
||||
Stream<HoodieColumnRangeMetadata<Comparable>> stream = columnToStatsListMap.values()
|
||||
.stream()
|
||||
.map(this::getColumnRangeInFile);
|
||||
|
||||
return stream.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private HoodieColumnRangeMetadata<Comparable> getColumnRangeInFile(final List<HoodieColumnRangeMetadata<Comparable>> blockRanges) {
|
||||
private <T extends Comparable<T>> HoodieColumnRangeMetadata<T> getColumnRangeInFile(
|
||||
@Nonnull List<HoodieColumnRangeMetadata<T>> blockRanges
|
||||
) {
|
||||
if (blockRanges.size() == 1) {
|
||||
// only one block in parquet file. we can just return that range.
|
||||
return blockRanges.get(0);
|
||||
} else {
|
||||
// there are multiple blocks. Compute min(block_mins) and max(block_maxs)
|
||||
return blockRanges.stream().reduce((b1, b2) -> combineRanges(b1, b2)).get();
|
||||
}
|
||||
|
||||
// there are multiple blocks. Compute min(block_mins) and max(block_maxs)
|
||||
return blockRanges.stream()
|
||||
.sequential()
|
||||
.reduce(this::combineRanges).get();
|
||||
}
|
||||
|
||||
private HoodieColumnRangeMetadata<Comparable> combineRanges(HoodieColumnRangeMetadata<Comparable> range1,
|
||||
HoodieColumnRangeMetadata<Comparable> range2) {
|
||||
final Comparable minValue;
|
||||
final Comparable maxValue;
|
||||
final String minValueAsString;
|
||||
final String maxValueAsString;
|
||||
if (range1.getMinValue() != null && range2.getMinValue() != null) {
|
||||
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();
|
||||
private <T extends Comparable<T>> HoodieColumnRangeMetadata<T> combineRanges(
|
||||
HoodieColumnRangeMetadata<T> one,
|
||||
HoodieColumnRangeMetadata<T> another
|
||||
) {
|
||||
final T minValue;
|
||||
final T maxValue;
|
||||
if (one.getMinValue() != null && another.getMinValue() != null) {
|
||||
minValue = one.getMinValue().compareTo(another.getMinValue()) < 0 ? one.getMinValue() : another.getMinValue();
|
||||
} else if (one.getMinValue() == null) {
|
||||
minValue = another.getMinValue();
|
||||
} else {
|
||||
minValue = range1.getMinValue();
|
||||
minValueAsString = range1.getMinValueAsString();
|
||||
minValue = one.getMinValue();
|
||||
}
|
||||
|
||||
if (range1.getMaxValue() != null && range2.getMaxValue() != null) {
|
||||
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();
|
||||
if (one.getMaxValue() != null && another.getMaxValue() != null) {
|
||||
maxValue = one.getMaxValue().compareTo(another.getMaxValue()) < 0 ? another.getMaxValue() : one.getMaxValue();
|
||||
} else if (one.getMaxValue() == null) {
|
||||
maxValue = another.getMaxValue();
|
||||
} else {
|
||||
maxValue = range1.getMaxValue();
|
||||
maxValueAsString = range1.getMaxValueAsString();
|
||||
maxValue = one.getMaxValue();
|
||||
}
|
||||
|
||||
return new HoodieColumnRangeMetadata<>(range1.getFilePath(),
|
||||
range1.getColumnName(), minValue, maxValue, range1.getNumNulls() + range2.getNumNulls(), minValueAsString, maxValueAsString);
|
||||
return new HoodieColumnRangeMetadata<T>(
|
||||
one.getFilePath(),
|
||||
one.getColumnName(), minValue, maxValue, one.getNumNulls() + another.getNumNulls(), one.getStringifier());
|
||||
}
|
||||
|
||||
private static Comparable<?> convertToNativeJavaType(PrimitiveType primitiveType, Comparable val) {
|
||||
if (primitiveType.getOriginalType() == OriginalType.DECIMAL) {
|
||||
DecimalMetadata decimalMetadata = primitiveType.getDecimalMetadata();
|
||||
return BigDecimal.valueOf((Integer) val, decimalMetadata.getScale());
|
||||
} else if (primitiveType.getOriginalType() == OriginalType.DATE) {
|
||||
// NOTE: This is a workaround to address race-condition in using
|
||||
// {@code SimpleDataFormat} concurrently (w/in {@code DateStringifier})
|
||||
// TODO cleanup after Parquet upgrade to 1.12
|
||||
synchronized (primitiveType.stringifier()) {
|
||||
return java.sql.Date.valueOf(
|
||||
primitiveType.stringifier().stringify((Integer) val)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
return val;
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user