[HUDI-3664] Fixing Column Stats Index composition (#5181)
Co-authored-by: Sagar Sumit <sagarsumit09@gmail.com>
This commit is contained in:
@@ -0,0 +1,246 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.model.HoodieColumnRangeMetadata;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.util.BaseFileUtils;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.spark.SparkContext;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.Row$;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.apache.spark.sql.types.BinaryType;
|
||||
import org.apache.spark.sql.types.BooleanType;
|
||||
import org.apache.spark.sql.types.ByteType;
|
||||
import org.apache.spark.sql.types.DataType;
|
||||
import org.apache.spark.sql.types.DateType;
|
||||
import org.apache.spark.sql.types.DecimalType;
|
||||
import org.apache.spark.sql.types.DoubleType;
|
||||
import org.apache.spark.sql.types.FloatType;
|
||||
import org.apache.spark.sql.types.IntegerType;
|
||||
import org.apache.spark.sql.types.LongType;
|
||||
import org.apache.spark.sql.types.ShortType;
|
||||
import org.apache.spark.sql.types.StringType;
|
||||
import org.apache.spark.sql.types.StructField;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
import org.apache.spark.sql.types.StructType$;
|
||||
import org.apache.spark.sql.types.TimestampType;
|
||||
import org.apache.spark.util.SerializableConfiguration;
|
||||
import scala.collection.JavaConversions;
|
||||
import scala.collection.JavaConverters$;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import java.math.BigDecimal;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
// TODO merge w/ ColumnStatsIndexSupport
|
||||
public class ColumnStatsIndexHelper {
|
||||
|
||||
public static Pair<Object, Object>
|
||||
fetchMinMaxValues(
|
||||
@Nonnull DataType colType,
|
||||
@Nonnull HoodieColumnRangeMetadata<Comparable> colMetadata) {
|
||||
if (colType instanceof IntegerType) {
|
||||
return Pair.of(
|
||||
new Integer(colMetadata.getMinValue().toString()),
|
||||
new Integer(colMetadata.getMaxValue().toString())
|
||||
);
|
||||
} else if (colType instanceof DoubleType) {
|
||||
return Pair.of(
|
||||
new Double(colMetadata.getMinValue().toString()),
|
||||
new Double(colMetadata.getMaxValue().toString())
|
||||
);
|
||||
} else if (colType instanceof StringType) {
|
||||
return Pair.of(
|
||||
colMetadata.getMinValue().toString(),
|
||||
colMetadata.getMaxValue().toString());
|
||||
} else if (colType instanceof DecimalType) {
|
||||
return Pair.of(
|
||||
new BigDecimal(colMetadata.getMinValue().toString()),
|
||||
new BigDecimal(colMetadata.getMaxValue().toString()));
|
||||
} else if (colType instanceof DateType) {
|
||||
return Pair.of(
|
||||
java.sql.Date.valueOf(colMetadata.getMinValue().toString()),
|
||||
java.sql.Date.valueOf(colMetadata.getMaxValue().toString()));
|
||||
} else if (colType instanceof LongType) {
|
||||
return Pair.of(
|
||||
new Long(colMetadata.getMinValue().toString()),
|
||||
new Long(colMetadata.getMaxValue().toString()));
|
||||
} else if (colType instanceof ShortType) {
|
||||
return Pair.of(
|
||||
new Short(colMetadata.getMinValue().toString()),
|
||||
new Short(colMetadata.getMaxValue().toString()));
|
||||
} else if (colType instanceof FloatType) {
|
||||
return Pair.of(
|
||||
new Float(colMetadata.getMinValue().toString()),
|
||||
new Float(colMetadata.getMaxValue().toString()));
|
||||
} else if (colType instanceof BinaryType) {
|
||||
return Pair.of(
|
||||
((ByteBuffer) colMetadata.getMinValue()).array(),
|
||||
((ByteBuffer) colMetadata.getMaxValue()).array());
|
||||
} else if (colType instanceof BooleanType) {
|
||||
return Pair.of(
|
||||
Boolean.valueOf(colMetadata.getMinValue().toString()),
|
||||
Boolean.valueOf(colMetadata.getMaxValue().toString()));
|
||||
} else if (colType instanceof ByteType) {
|
||||
return Pair.of(
|
||||
Byte.valueOf(colMetadata.getMinValue().toString()),
|
||||
Byte.valueOf(colMetadata.getMaxValue().toString()));
|
||||
} else {
|
||||
throw new HoodieException(String.format("Not support type: %s", colType));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* NOTE: THIS IS ONLY USED IN TESTING CURRENTLY, SINCE DATA SKIPPING IS NOW RELYING ON
|
||||
* METADATA TABLE INDEX
|
||||
*
|
||||
* Parse min/max statistics from Parquet footers for provided columns and composes column-stats
|
||||
* index table in the following format with 3 statistics denominated for each
|
||||
* linear/Z-curve/Hilbert-curve-ordered column. For ex, if original table contained
|
||||
* column {@code A}:
|
||||
*
|
||||
* <pre>
|
||||
* +---------------------------+------------+------------+-------------+
|
||||
* | file | A_minValue | A_maxValue | A_num_nulls |
|
||||
* +---------------------------+------------+------------+-------------+
|
||||
* | one_base_file.parquet | 1 | 10 | 0 |
|
||||
* | another_base_file.parquet | -10 | 0 | 5 |
|
||||
* +---------------------------+------------+------------+-------------+
|
||||
* </pre>
|
||||
* <p>
|
||||
* NOTE: Currently {@link TimestampType} is not supported, since Parquet writer
|
||||
* does not support statistics for it.
|
||||
*
|
||||
* @VisibleForTestingOnly
|
||||
*
|
||||
* @param sparkSession encompassing Spark session
|
||||
* @param baseFilesPaths list of base-files paths to be sourced for column-stats index
|
||||
* @param orderedColumnSchemas target ordered columns
|
||||
* @return Spark's {@link Dataset} holding an index table
|
||||
* @VisibleForTesting
|
||||
*/
|
||||
@Nonnull
|
||||
public static Dataset<Row> buildColumnStatsTableFor(
|
||||
@Nonnull SparkSession sparkSession,
|
||||
@Nonnull List<String> baseFilesPaths,
|
||||
@Nonnull List<StructField> orderedColumnSchemas
|
||||
) {
|
||||
SparkContext sc = sparkSession.sparkContext();
|
||||
JavaSparkContext jsc = new JavaSparkContext(sc);
|
||||
|
||||
List<String> columnNames = orderedColumnSchemas.stream()
|
||||
.map(StructField::name)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
SerializableConfiguration serializableConfiguration = new SerializableConfiguration(sc.hadoopConfiguration());
|
||||
int numParallelism = (baseFilesPaths.size() / 3 + 1);
|
||||
|
||||
String previousJobDescription = sc.getLocalProperty("spark.job.description");
|
||||
|
||||
List<HoodieColumnRangeMetadata<Comparable>> colMinMaxInfos;
|
||||
try {
|
||||
jsc.setJobDescription("Listing parquet column statistics");
|
||||
colMinMaxInfos =
|
||||
jsc.parallelize(baseFilesPaths, numParallelism)
|
||||
.mapPartitions(paths -> {
|
||||
ParquetUtils utils = (ParquetUtils) BaseFileUtils.getInstance(HoodieFileFormat.PARQUET);
|
||||
Iterable<String> iterable = () -> paths;
|
||||
return StreamSupport.stream(iterable.spliterator(), false)
|
||||
.flatMap(path ->
|
||||
utils.readRangeFromParquetMetadata(
|
||||
serializableConfiguration.value(),
|
||||
new Path(path),
|
||||
columnNames
|
||||
)
|
||||
.stream()
|
||||
)
|
||||
.iterator();
|
||||
})
|
||||
.collect();
|
||||
} finally {
|
||||
jsc.setJobDescription(previousJobDescription);
|
||||
}
|
||||
|
||||
// Group column's metadata by file-paths of the files it belongs to
|
||||
Map<String, List<HoodieColumnRangeMetadata<Comparable>>> filePathToColumnMetadataMap =
|
||||
colMinMaxInfos.stream()
|
||||
.collect(Collectors.groupingBy(HoodieColumnRangeMetadata::getFilePath));
|
||||
|
||||
JavaRDD<Row> allMetaDataRDD =
|
||||
jsc.parallelize(new ArrayList<>(filePathToColumnMetadataMap.values()), 1)
|
||||
.map(fileColumnsMetadata -> {
|
||||
int colSize = fileColumnsMetadata.size();
|
||||
if (colSize == 0) {
|
||||
return null;
|
||||
}
|
||||
|
||||
String filePath = fileColumnsMetadata.get(0).getFilePath();
|
||||
|
||||
List<Object> indexRow = new ArrayList<>();
|
||||
|
||||
// First columns of the Z-index's row is target file-path
|
||||
indexRow.add(filePath);
|
||||
|
||||
// For each column
|
||||
orderedColumnSchemas.forEach(colSchema -> {
|
||||
String colName = colSchema.name();
|
||||
|
||||
HoodieColumnRangeMetadata<Comparable> colMetadata =
|
||||
fileColumnsMetadata.stream()
|
||||
.filter(s -> s.getColumnName().trim().equalsIgnoreCase(colName))
|
||||
.findFirst()
|
||||
.orElse(null);
|
||||
|
||||
DataType colType = colSchema.dataType();
|
||||
if (colMetadata == null || colType == null) {
|
||||
throw new HoodieException(String.format("Cannot collect min/max statistics for column (%s)", colSchema));
|
||||
}
|
||||
|
||||
Pair<Object, Object> minMaxValue = fetchMinMaxValues(colType, colMetadata);
|
||||
|
||||
indexRow.add(minMaxValue.getLeft()); // min
|
||||
indexRow.add(minMaxValue.getRight()); // max
|
||||
indexRow.add(colMetadata.getNullCount());
|
||||
});
|
||||
|
||||
return Row$.MODULE$.apply(JavaConversions.asScalaBuffer(indexRow));
|
||||
})
|
||||
.filter(Objects::nonNull);
|
||||
|
||||
StructType indexSchema = ColumnStatsIndexSupport$.MODULE$.composeIndexSchema(
|
||||
JavaConverters$.MODULE$.collectionAsScalaIterableConverter(columnNames).asScala().toSeq(),
|
||||
StructType$.MODULE$.apply(orderedColumnSchemas)
|
||||
);
|
||||
|
||||
return sparkSession.createDataFrame(allMetaDataRDD, indexSchema);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,4 @@
|
||||
{"c1_maxValue":769,"c1_minValue":309,"c1_num_nulls":0,"c2_maxValue":" 769sdc","c2_minValue":" 309sdc","c2_num_nulls":0,"c3_maxValue":919.769,"c3_minValue":76.430,"c3_num_nulls":0,"c4_maxValue":"2021-11-19T20:40:55.543-08:00","c4_minValue":"2021-11-19T20:40:55.521-08:00","c4_num_nulls":0,"c5_maxValue":78,"c5_minValue":32,"c5_num_nulls":0,"c6_maxValue":"2020-11-14","c6_minValue":"2020-01-08","c6_num_nulls":0,"c7_maxValue":"uQ==","c7_minValue":"AQ==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0}
|
||||
{"c1_maxValue":932,"c1_minValue":0,"c1_num_nulls":0,"c2_maxValue":" 932sdc","c2_minValue":" 0sdc","c2_num_nulls":0,"c3_maxValue":994.355,"c3_minValue":19.000,"c3_num_nulls":0,"c4_maxValue":"2021-11-19T20:40:55.549-08:00","c4_minValue":"2021-11-19T20:40:55.339-08:00","c4_num_nulls":0,"c5_maxValue":94,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-09-09","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"xw==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0}
|
||||
{"c1_maxValue":943,"c1_minValue":89,"c1_num_nulls":0,"c2_maxValue":" 943sdc","c2_minValue":" 200sdc","c2_num_nulls":0,"c3_maxValue":854.690,"c3_minValue":100.556,"c3_num_nulls":0,"c4_maxValue":"2021-11-19T20:40:55.549-08:00","c4_minValue":"2021-11-19T20:40:55.508-08:00","c4_num_nulls":0,"c5_maxValue":95,"c5_minValue":10,"c5_num_nulls":0,"c6_maxValue":"2020-10-10","c6_minValue":"2020-01-10","c6_num_nulls":0,"c7_maxValue":"yA==","c7_minValue":"LA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0}
|
||||
{"c1_maxValue":959,"c1_minValue":74,"c1_num_nulls":0,"c2_maxValue":" 959sdc","c2_minValue":" 181sdc","c2_num_nulls":0,"c3_maxValue":980.213,"c3_minValue":38.740,"c3_num_nulls":0,"c4_maxValue":"2021-11-19T20:40:55.550-08:00","c4_minValue":"2021-11-19T20:40:55.507-08:00","c4_num_nulls":0,"c5_maxValue":97,"c5_minValue":9,"c5_num_nulls":0,"c6_maxValue":"2020-11-22","c6_minValue":"2020-01-23","c6_num_nulls":0,"c7_maxValue":"1Q==","c7_minValue":"Kw==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0}
|
||||
@@ -0,0 +1,8 @@
|
||||
{"c1_maxValue":568,"c1_minValue":8,"c1_num_nulls":0,"c2_maxValue":" 8sdc","c2_minValue":" 111sdc","c2_num_nulls":0,"c3_maxValue":979.272,"c3_minValue":82.111,"c3_num_nulls":0,"c4_maxValue":"2021-11-18T23:34:44.193-08:00","c4_minValue":"2021-11-18T23:34:44.159-08:00","c4_num_nulls":0,"c5_maxValue":58,"c5_minValue":2,"c5_num_nulls":0,"c6_maxValue":"2020-11-08","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"9g==","c7_minValue":"Ag==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0}
|
||||
{"c1_maxValue":715,"c1_minValue":76,"c1_num_nulls":0,"c2_maxValue":" 76sdc","c2_minValue":" 224sdc","c2_num_nulls":0,"c3_maxValue":958.579,"c3_minValue":246.427,"c3_num_nulls":0,"c4_maxValue":"2021-11-18T23:34:44.199-08:00","c4_minValue":"2021-11-18T23:34:44.166-08:00","c4_num_nulls":0,"c5_maxValue":73,"c5_minValue":9,"c5_num_nulls":0,"c6_maxValue":"2020-11-21","c6_minValue":"2020-01-16","c6_num_nulls":0,"c7_maxValue":"+g==","c7_minValue":"LA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0}
|
||||
{"c1_maxValue":768,"c1_minValue":59,"c1_num_nulls":0,"c2_maxValue":" 768sdc","c2_minValue":" 118sdc","c2_num_nulls":0,"c3_maxValue":959.131,"c3_minValue":64.768,"c3_num_nulls":0,"c4_maxValue":"2021-11-18T23:34:44.201-08:00","c4_minValue":"2021-11-18T23:34:44.164-08:00","c4_num_nulls":0,"c5_maxValue":78,"c5_minValue":7,"c5_num_nulls":0,"c6_maxValue":"2020-11-20","c6_minValue":"2020-05-04","c6_num_nulls":0,"c7_maxValue":"zw==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0}
|
||||
{"c1_maxValue":769,"c1_minValue":309,"c1_num_nulls":0,"c2_maxValue":" 769sdc","c2_minValue":" 309sdc","c2_num_nulls":0,"c3_maxValue":919.769,"c3_minValue":76.430,"c3_num_nulls":0,"c4_maxValue":"2021-11-19T20:40:55.543-08:00","c4_minValue":"2021-11-19T20:40:55.521-08:00","c4_num_nulls":0,"c5_maxValue":78,"c5_minValue":32,"c5_num_nulls":0,"c6_maxValue":"2020-11-14","c6_minValue":"2020-01-08","c6_num_nulls":0,"c7_maxValue":"uQ==","c7_minValue":"AQ==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0}
|
||||
{"c1_maxValue":770,"c1_minValue":129,"c1_num_nulls":0,"c2_maxValue":" 770sdc","c2_minValue":" 129sdc","c2_num_nulls":0,"c3_maxValue":977.328,"c3_minValue":153.431,"c3_num_nulls":0,"c4_maxValue":"2021-11-18T23:34:44.201-08:00","c4_minValue":"2021-11-18T23:34:44.169-08:00","c4_num_nulls":0,"c5_maxValue":78,"c5_minValue":14,"c5_num_nulls":0,"c6_maxValue":"2020-10-21","c6_minValue":"2020-01-15","c6_num_nulls":0,"c7_maxValue":"rw==","c7_minValue":"Ag==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0}
|
||||
{"c1_maxValue":932,"c1_minValue":0,"c1_num_nulls":0,"c2_maxValue":" 932sdc","c2_minValue":" 0sdc","c2_num_nulls":0,"c3_maxValue":994.355,"c3_minValue":19.000,"c3_num_nulls":0,"c4_maxValue":"2021-11-19T20:40:55.549-08:00","c4_minValue":"2021-11-19T20:40:55.339-08:00","c4_num_nulls":0,"c5_maxValue":94,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-09-09","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"xw==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0}
|
||||
{"c1_maxValue":943,"c1_minValue":89,"c1_num_nulls":0,"c2_maxValue":" 943sdc","c2_minValue":" 200sdc","c2_num_nulls":0,"c3_maxValue":854.690,"c3_minValue":100.556,"c3_num_nulls":0,"c4_maxValue":"2021-11-19T20:40:55.549-08:00","c4_minValue":"2021-11-19T20:40:55.508-08:00","c4_num_nulls":0,"c5_maxValue":95,"c5_minValue":10,"c5_num_nulls":0,"c6_maxValue":"2020-10-10","c6_minValue":"2020-01-10","c6_num_nulls":0,"c7_maxValue":"yA==","c7_minValue":"LA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0}
|
||||
{"c1_maxValue":959,"c1_minValue":74,"c1_num_nulls":0,"c2_maxValue":" 959sdc","c2_minValue":" 181sdc","c2_num_nulls":0,"c3_maxValue":980.213,"c3_minValue":38.740,"c3_num_nulls":0,"c4_maxValue":"2021-11-19T20:40:55.550-08:00","c4_minValue":"2021-11-19T20:40:55.507-08:00","c4_num_nulls":0,"c5_maxValue":97,"c5_minValue":9,"c5_num_nulls":0,"c6_maxValue":"2020-11-22","c6_minValue":"2020-01-23","c6_num_nulls":0,"c7_maxValue":"1Q==","c7_minValue":"Kw==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0}
|
||||
@@ -1,8 +0,0 @@
|
||||
{"c1_maxValue":272,"c1_minValue":8,"c1_num_nulls":0,"c2_maxValue":" 8sdc","c2_minValue":" 129sdc","c2_num_nulls":0,"c3_maxValue":979.272,"c3_minValue":430.129,"c3_num_nulls":0,"c5_maxValue":28,"c5_minValue":2,"c5_num_nulls":0,"c6_maxValue":"2020-11-20","c6_minValue":"2020-03-23","c6_num_nulls":0,"c7_maxValue":"8A==","c7_minValue":"Ag==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00003-xxx-c000.snappy.parquet"}
|
||||
{"c1_maxValue":486,"c1_minValue":59,"c1_num_nulls":0,"c2_maxValue":" 79sdc","c2_minValue":" 111sdc","c2_num_nulls":0,"c3_maxValue":771.590,"c3_minValue":82.111,"c3_num_nulls":0,"c5_maxValue":50,"c5_minValue":7,"c5_num_nulls":0,"c6_maxValue":"2020-11-21","c6_minValue":"2020-01-22","c6_num_nulls":0,"c7_maxValue":"5g==","c7_minValue":"Ow==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00002-xxx-c000.snappy.parquet"}
|
||||
{"c1_maxValue":559,"c1_minValue":74,"c1_num_nulls":0,"c2_maxValue":" 74sdc","c2_minValue":" 181sdc","c2_num_nulls":0,"c3_maxValue":980.213,"c3_minValue":38.740,"c3_num_nulls":0,"c5_maxValue":57,"c5_minValue":9,"c5_num_nulls":0,"c6_maxValue":"2020-11-09","c6_minValue":"2020-01-08","c6_num_nulls":0,"c7_maxValue":"1Q==","c7_minValue":"Gw==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00001-xxx-c000.snappy.parquet"}
|
||||
{"c1_maxValue":639,"c1_minValue":323,"c1_num_nulls":0,"c2_maxValue":" 639sdc","c2_minValue":" 323sdc","c2_num_nulls":0,"c3_maxValue":811.638,"c3_minValue":100.556,"c3_num_nulls":0,"c5_maxValue":65,"c5_minValue":33,"c5_num_nulls":0,"c6_maxValue":"2020-09-09","c6_minValue":"2020-01-23","c6_num_nulls":0,"c7_maxValue":"fw==","c7_minValue":"Kw==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00000-xxx-c000.snappy.parquet"}
|
||||
{"c1_maxValue":719,"c1_minValue":125,"c1_num_nulls":0,"c2_maxValue":" 719sdc","c2_minValue":" 125sdc","c2_num_nulls":0,"c3_maxValue":958.579,"c3_minValue":153.125,"c3_num_nulls":0,"c5_maxValue":73,"c5_minValue":14,"c5_num_nulls":0,"c6_maxValue":"2020-09-27","c6_minValue":"2020-01-16","c6_num_nulls":0,"c7_maxValue":"+g==","c7_minValue":"OA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00001-xxx-c000.snappy.parquet"}
|
||||
{"c1_maxValue":770,"c1_minValue":300,"c1_num_nulls":0,"c2_maxValue":" 770sdc","c2_minValue":" 300sdc","c2_num_nulls":0,"c3_maxValue":977.328,"c3_minValue":64.768,"c3_num_nulls":0,"c5_maxValue":78,"c5_minValue":31,"c5_num_nulls":0,"c6_maxValue":"2020-10-21","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"rw==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00000-xxx-c000.snappy.parquet"}
|
||||
{"c1_maxValue":945,"c1_minValue":355,"c1_num_nulls":0,"c2_maxValue":" 945sdc","c2_minValue":" 355sdc","c2_num_nulls":0,"c3_maxValue":994.355,"c3_minValue":374.882,"c3_num_nulls":0,"c5_maxValue":96,"c5_minValue":37,"c5_num_nulls":0,"c6_maxValue":"2020-11-22","c6_minValue":"2020-02-25","c6_num_nulls":0,"c7_maxValue":"sQ==","c7_minValue":"AQ==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00002-xxx-c000.snappy.parquet"}
|
||||
{"c1_maxValue":959,"c1_minValue":0,"c1_num_nulls":0,"c2_maxValue":" 959sdc","c2_minValue":" 0sdc","c2_num_nulls":0,"c3_maxValue":916.697,"c3_minValue":19.000,"c3_num_nulls":0,"c5_maxValue":97,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-22","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"yA==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00003-xxx-c000.snappy.parquet"}
|
||||
@@ -1,4 +0,0 @@
|
||||
{"c1_maxValue":559,"c1_minValue":74,"c1_num_nulls":0,"c2_maxValue":" 74sdc","c2_minValue":" 181sdc","c2_num_nulls":0,"c3_maxValue":980.213,"c3_minValue":38.740,"c3_num_nulls":0,"c5_maxValue":57,"c5_minValue":9,"c5_num_nulls":0,"c6_maxValue":"2020-11-09","c6_minValue":"2020-01-08","c6_num_nulls":0,"c7_maxValue":"1Q==","c7_minValue":"Gw==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00001-xxx-c000.snappy.parquet"}
|
||||
{"c1_maxValue":639,"c1_minValue":323,"c1_num_nulls":0,"c2_maxValue":" 639sdc","c2_minValue":" 323sdc","c2_num_nulls":0,"c3_maxValue":811.638,"c3_minValue":100.556,"c3_num_nulls":0,"c5_maxValue":65,"c5_minValue":33,"c5_num_nulls":0,"c6_maxValue":"2020-09-09","c6_minValue":"2020-01-23","c6_num_nulls":0,"c7_maxValue":"fw==","c7_minValue":"Kw==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00000-xxx-c000.snappy.parquet"}
|
||||
{"c1_maxValue":945,"c1_minValue":355,"c1_num_nulls":0,"c2_maxValue":" 945sdc","c2_minValue":" 355sdc","c2_num_nulls":0,"c3_maxValue":994.355,"c3_minValue":374.882,"c3_num_nulls":0,"c5_maxValue":96,"c5_minValue":37,"c5_num_nulls":0,"c6_maxValue":"2020-11-22","c6_minValue":"2020-02-25","c6_num_nulls":0,"c7_maxValue":"sQ==","c7_minValue":"AQ==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00002-xxx-c000.snappy.parquet"}
|
||||
{"c1_maxValue":959,"c1_minValue":0,"c1_num_nulls":0,"c2_maxValue":" 959sdc","c2_minValue":" 0sdc","c2_num_nulls":0,"c3_maxValue":916.697,"c3_minValue":19.000,"c3_num_nulls":0,"c5_maxValue":97,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-22","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"yA==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00003-xxx-c000.snappy.parquet"}
|
||||
@@ -17,7 +17,7 @@
|
||||
|
||||
package org.apache.hudi
|
||||
|
||||
import org.apache.hudi.index.columnstats.ColumnStatsIndexHelper
|
||||
import org.apache.hudi.ColumnStatsIndexSupport.composeIndexSchema
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.spark.sql.catalyst.expressions.{Expression, Not}
|
||||
import org.apache.spark.sql.functions.{col, lower}
|
||||
@@ -35,24 +35,22 @@ import java.sql.Timestamp
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
// NOTE: Only A, B columns are indexed
|
||||
case class IndexRow(
|
||||
file: String,
|
||||
case class IndexRow(fileName: String,
|
||||
|
||||
// Corresponding A column is LongType
|
||||
A_minValue: Long = -1,
|
||||
A_maxValue: Long = -1,
|
||||
A_num_nulls: Long = -1,
|
||||
// Corresponding A column is LongType
|
||||
A_minValue: Long = -1,
|
||||
A_maxValue: Long = -1,
|
||||
A_num_nulls: Long = -1,
|
||||
|
||||
// Corresponding B column is StringType
|
||||
B_minValue: String = null,
|
||||
B_maxValue: String = null,
|
||||
B_num_nulls: Long = -1,
|
||||
// Corresponding B column is StringType
|
||||
B_minValue: String = null,
|
||||
B_maxValue: String = null,
|
||||
B_num_nulls: Long = -1,
|
||||
|
||||
// Corresponding B column is TimestampType
|
||||
C_minValue: Timestamp = null,
|
||||
C_maxValue: Timestamp = null,
|
||||
C_num_nulls: Long = -1
|
||||
) {
|
||||
// Corresponding B column is TimestampType
|
||||
C_minValue: Timestamp = null,
|
||||
C_maxValue: Timestamp = null,
|
||||
C_num_nulls: Long = -1) {
|
||||
def toRow: Row = Row(productIterator.toSeq: _*)
|
||||
}
|
||||
|
||||
@@ -79,29 +77,27 @@ class TestDataSkippingUtils extends HoodieClientTestBase with SparkAdapterSuppor
|
||||
)
|
||||
)
|
||||
|
||||
val indexSchema: StructType =
|
||||
ColumnStatsIndexHelper.composeIndexSchema(
|
||||
sourceTableSchema.fields.toSeq
|
||||
.filter(f => indexedCols.contains(f.name))
|
||||
.asJava
|
||||
)
|
||||
val indexSchema: StructType = composeIndexSchema(indexedCols, sourceTableSchema)
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource(
|
||||
Array(
|
||||
"testBasicLookupFilterExpressionsSource",
|
||||
"testAdvancedLookupFilterExpressionsSource",
|
||||
"testCompositeFilterExpressionsSource"
|
||||
"testBasicLookupFilterExpressionsSource",
|
||||
"testAdvancedLookupFilterExpressionsSource",
|
||||
"testCompositeFilterExpressionsSource"
|
||||
))
|
||||
def testLookupFilterExpressions(sourceExpr: String, input: Seq[IndexRow], output: Seq[String]): Unit = {
|
||||
// We have to fix the timezone to make sure all date-bound utilities output
|
||||
// is consistent with the fixtures
|
||||
spark.sqlContext.setConf(SESSION_LOCAL_TIMEZONE.key, "UTC")
|
||||
|
||||
val resolvedExpr: Expression = exprUtils.resolveExpr(spark, sourceExpr, sourceTableSchema)
|
||||
val lookupFilter = DataSkippingUtils.translateIntoColumnStatsIndexFilterExpr(resolvedExpr, indexSchema)
|
||||
|
||||
val indexDf = spark.createDataFrame(input.map(_.toRow).asJava, indexSchema)
|
||||
|
||||
val rows = indexDf.where(new Column(lookupFilter))
|
||||
.select("file")
|
||||
.select("fileName")
|
||||
.collect()
|
||||
.map(_.getString(0))
|
||||
.toSeq
|
||||
@@ -121,7 +117,7 @@ class TestDataSkippingUtils extends HoodieClientTestBase with SparkAdapterSuppor
|
||||
val indexDf = spark.createDataset(input)
|
||||
|
||||
val rows = indexDf.where(new Column(lookupFilter))
|
||||
.select("file")
|
||||
.select("fileName")
|
||||
.collect()
|
||||
.map(_.getString(0))
|
||||
.toSeq
|
||||
@@ -340,7 +336,7 @@ object TestDataSkippingUtils {
|
||||
|
||||
arguments(
|
||||
// Filter out all rows that contain A = 0 AND B = 'abc'
|
||||
"A != 0 OR B != 'abc'",
|
||||
"A != 0 OR B != 'abc'",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
|
||||
@@ -20,25 +20,28 @@ package org.apache.hudi.functional
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.{FileSystem, LocatedFileStatus, Path}
|
||||
import org.apache.hudi.ColumnStatsIndexSupport.composeIndexSchema
|
||||
import org.apache.hudi.DataSourceWriteOptions.{PRECOMBINE_FIELD, RECORDKEY_FIELD}
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
||||
import org.apache.hudi.common.util.ParquetUtils
|
||||
import org.apache.hudi.index.columnstats.ColumnStatsIndexHelper
|
||||
import org.apache.hudi.config.{HoodieStorageConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.metadata.HoodieTableMetadata
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.hudi.{ColumnStatsIndexSupport, DataSourceWriteOptions}
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.expressions.UserDefinedFunction
|
||||
import org.apache.spark.sql.functions.typedLit
|
||||
import org.apache.spark.sql.types._
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, Tag, Test}
|
||||
import org.junit.jupiter.api._
|
||||
|
||||
import java.math.BigInteger
|
||||
import java.sql.{Date, Timestamp}
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.util.Random
|
||||
|
||||
// TODO repurpose to test Column Stats in Metadata Table
|
||||
@Disabled
|
||||
@Tag("functional")
|
||||
class TestColumnStatsIndex extends HoodieClientTestBase {
|
||||
class TestColumnStatsIndex extends HoodieClientTestBase with ColumnStatsIndexSupport {
|
||||
var spark: SparkSession = _
|
||||
|
||||
val sourceTableSchema =
|
||||
@@ -67,140 +70,98 @@ class TestColumnStatsIndex extends HoodieClientTestBase {
|
||||
}
|
||||
|
||||
@Test
|
||||
def testZIndexTableComposition(): Unit = {
|
||||
val targetParquetTablePath = tempDir.resolve("index/zorder/input-table").toAbsolutePath.toString
|
||||
def testMetadataColumnStatsIndex(): Unit = {
|
||||
val opts = Map(
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
|
||||
RECORDKEY_FIELD.key -> "c1",
|
||||
PRECOMBINE_FIELD.key -> "c1",
|
||||
HoodieMetadataConfig.ENABLE.key -> "true",
|
||||
HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true",
|
||||
HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS_FOR_ALL_COLUMNS.key -> "true",
|
||||
HoodieTableConfig.POPULATE_META_FIELDS.key -> "true"
|
||||
)
|
||||
|
||||
setTableName("hoodie_test")
|
||||
initMetaClient()
|
||||
|
||||
val sourceJSONTablePath = getClass.getClassLoader.getResource("index/zorder/input-table-json").toString
|
||||
|
||||
bootstrapParquetInputTableFromJSON(sourceJSONTablePath, targetParquetTablePath)
|
||||
|
||||
val inputDf =
|
||||
// NOTE: Schema here is provided for validation that the input date is in the appropriate format
|
||||
val inputDF = spark.read.schema(sourceTableSchema).json(sourceJSONTablePath)
|
||||
|
||||
inputDF
|
||||
.sort("c1")
|
||||
.repartition(4, new Column("c1"))
|
||||
.write
|
||||
.format("hudi")
|
||||
.options(opts)
|
||||
.option(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.key, 10 * 1024)
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient)
|
||||
|
||||
val metadataTablePath = HoodieTableMetadata.getMetadataTableBasePath(basePath)
|
||||
|
||||
val colStatsDF = readColumnStatsIndex(spark, metadataTablePath)
|
||||
val transposedColStatsDF = transposeColumnStatsIndex(spark, colStatsDF, sourceTableSchema.fieldNames, sourceTableSchema)
|
||||
|
||||
val expectedColStatsSchema = composeIndexSchema(sourceTableSchema.fieldNames, sourceTableSchema)
|
||||
|
||||
// Match against expected column stats table
|
||||
val expectedColStatsIndexTableDf =
|
||||
spark.read
|
||||
.schema(sourceTableSchema)
|
||||
.parquet(targetParquetTablePath)
|
||||
.schema(expectedColStatsSchema)
|
||||
.json(getClass.getClassLoader.getResource("index/zorder/column-stats-index-table.json").toString)
|
||||
|
||||
val zorderedCols = Seq("c1", "c2", "c3", "c5", "c6", "c7", "c8")
|
||||
val zorderedColsSchemaFields = inputDf.schema.fields.filter(f => zorderedCols.contains(f.name)).toSeq
|
||||
assertEquals(expectedColStatsIndexTableDf.schema, transposedColStatsDF.schema)
|
||||
// NOTE: We have to drop the `fileName` column as it contains semi-random components
|
||||
// that we can't control in this test. Nevertheless, since we manually verify composition of the
|
||||
// ColStats Index by reading Parquet footers from individual Parquet files, this is not an issue
|
||||
assertEquals(asJson(sort(expectedColStatsIndexTableDf)), asJson(sort(transposedColStatsDF.drop("fileName"))))
|
||||
|
||||
// {@link TimestampType} is not supported, and will throw -- hence skipping "c4"
|
||||
val newZIndexTableDf = null
|
||||
// ColumnStatsIndexHelper.buildColumnStatsTableFor(
|
||||
// inputDf.sparkSession,
|
||||
// inputDf.inputFiles.toSeq.asJava,
|
||||
// zorderedColsSchemaFields.asJava
|
||||
// )
|
||||
// Collect Column Stats manually (reading individual Parquet files)
|
||||
val manualColStatsTableDF =
|
||||
buildColumnStatsTableManually(basePath, sourceTableSchema.fieldNames, expectedColStatsSchema)
|
||||
|
||||
val indexSchema =
|
||||
ColumnStatsIndexHelper.composeIndexSchema(
|
||||
sourceTableSchema.fields.filter(f => zorderedCols.contains(f.name)).toSeq.asJava
|
||||
)
|
||||
assertEquals(asJson(sort(manualColStatsTableDF)), asJson(sort(transposedColStatsDF)))
|
||||
|
||||
// Collect Z-index stats manually (reading individual Parquet files)
|
||||
val manualZIndexTableDf =
|
||||
buildColumnStatsTableManually(targetParquetTablePath, zorderedCols, indexSchema)
|
||||
// do an upsert and validate
|
||||
val updateJSONTablePath = getClass.getClassLoader.getResource("index/zorder/another-input-table-json").toString
|
||||
val updateDF = spark.read
|
||||
.schema(sourceTableSchema)
|
||||
.json(updateJSONTablePath)
|
||||
|
||||
// NOTE: Z-index is built against stats collected w/in Parquet footers, which will be
|
||||
// represented w/ corresponding Parquet schema (INT, INT64, INT96, etc).
|
||||
//
|
||||
// When stats are collected manually, produced Z-index table is inherently coerced into the
|
||||
// schema of the original source Parquet base-file and therefore we have to similarly coerce newly
|
||||
// built Z-index table (built off Parquet footers) into the canonical index schema (built off the
|
||||
// original source file schema)
|
||||
assertEquals(asJson(sort(manualZIndexTableDf)), asJson(sort(newZIndexTableDf)))
|
||||
updateDF.repartition(4)
|
||||
.write
|
||||
.format("hudi")
|
||||
.options(opts)
|
||||
.option(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.key, 10 * 1024)
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
// Match against expected Z-index table
|
||||
val expectedZIndexTableDf =
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient)
|
||||
|
||||
val updatedColStatsDF = readColumnStatsIndex(spark, metadataTablePath)
|
||||
val transposedUpdatedColStatsDF = transposeColumnStatsIndex(spark, updatedColStatsDF, sourceTableSchema.fieldNames, sourceTableSchema)
|
||||
|
||||
val expectedColStatsIndexUpdatedDF =
|
||||
spark.read
|
||||
.schema(indexSchema)
|
||||
.json(getClass.getClassLoader.getResource("index/zorder/z-index-table.json").toString)
|
||||
.schema(expectedColStatsSchema)
|
||||
.json(getClass.getClassLoader.getResource("index/zorder/updated-column-stats-index-table.json").toString)
|
||||
|
||||
assertEquals(asJson(sort(expectedZIndexTableDf)), asJson(sort(replace(newZIndexTableDf))))
|
||||
}
|
||||
assertEquals(expectedColStatsIndexUpdatedDF.schema, transposedUpdatedColStatsDF.schema)
|
||||
assertEquals(asJson(sort(expectedColStatsIndexUpdatedDF)), asJson(sort(transposedUpdatedColStatsDF.drop("fileName"))))
|
||||
|
||||
@Test
|
||||
def testZIndexTableMerge(): Unit = {
|
||||
val testZIndexPath = new Path(basePath, "zindex")
|
||||
// Collect Column Stats manually (reading individual Parquet files)
|
||||
val manualUpdatedColStatsTableDF =
|
||||
buildColumnStatsTableManually(basePath, sourceTableSchema.fieldNames, expectedColStatsSchema)
|
||||
|
||||
val firstParquetTablePath = tempDir.resolve("index/zorder/input-table").toAbsolutePath.toString
|
||||
val firstJSONTablePath = getClass.getClassLoader.getResource("index/zorder/input-table-json").toString
|
||||
|
||||
// Bootstrap FIRST source Parquet table
|
||||
bootstrapParquetInputTableFromJSON(firstJSONTablePath, firstParquetTablePath)
|
||||
|
||||
val zorderedCols = Seq("c1", "c2", "c3", "c5", "c6", "c7", "c8")
|
||||
val indexSchema =
|
||||
ColumnStatsIndexHelper.composeIndexSchema(
|
||||
sourceTableSchema.fields.filter(f => zorderedCols.contains(f.name)).toSeq.asJava
|
||||
)
|
||||
|
||||
//
|
||||
// Bootstrap Z-index table
|
||||
//
|
||||
|
||||
val firstCommitInstance = "0"
|
||||
val firstInputDf = spark.read.parquet(firstParquetTablePath)
|
||||
|
||||
// ColumnStatsIndexHelper.updateColumnStatsIndexFor(
|
||||
// firstInputDf.sparkSession,
|
||||
// sourceTableSchema,
|
||||
// firstInputDf.inputFiles.toSeq.asJava,
|
||||
// zorderedCols.asJava,
|
||||
// testZIndexPath.toString,
|
||||
// firstCommitInstance,
|
||||
// Seq().asJava
|
||||
// )
|
||||
|
||||
// NOTE: We don't need to provide schema upon reading from Parquet, since Spark will be able
|
||||
// to reliably retrieve it
|
||||
val initialZIndexTable =
|
||||
spark.read
|
||||
.parquet(new Path(testZIndexPath, firstCommitInstance).toString)
|
||||
|
||||
val expectedInitialZIndexTableDf =
|
||||
spark.read
|
||||
.schema(indexSchema)
|
||||
.json(getClass.getClassLoader.getResource("index/zorder/z-index-table.json").toString)
|
||||
|
||||
assertEquals(asJson(sort(expectedInitialZIndexTableDf)), asJson(sort(replace(initialZIndexTable))))
|
||||
|
||||
// Bootstrap SECOND source Parquet table
|
||||
val secondParquetTablePath = tempDir.resolve("index/zorder/another-input-table").toAbsolutePath.toString
|
||||
val secondJSONTablePath = getClass.getClassLoader.getResource("index/zorder/another-input-table-json").toString
|
||||
|
||||
bootstrapParquetInputTableFromJSON(secondJSONTablePath, secondParquetTablePath)
|
||||
|
||||
val secondCommitInstance = "1"
|
||||
val secondInputDf =
|
||||
spark.read
|
||||
.schema(sourceTableSchema)
|
||||
.parquet(secondParquetTablePath)
|
||||
|
||||
//
|
||||
// Update Column Stats table
|
||||
//
|
||||
|
||||
// ColumnStatsIndexHelper.updateColumnStatsIndexFor(
|
||||
// secondInputDf.sparkSession,
|
||||
// sourceTableSchema,
|
||||
// secondInputDf.inputFiles.toSeq.asJava,
|
||||
// zorderedCols.asJava,
|
||||
// testZIndexPath.toString,
|
||||
// secondCommitInstance,
|
||||
// Seq(firstCommitInstance).asJava
|
||||
// )
|
||||
|
||||
// NOTE: We don't need to provide schema upon reading from Parquet, since Spark will be able
|
||||
// to reliably retrieve it
|
||||
val mergedZIndexTable =
|
||||
spark.read
|
||||
.parquet(new Path(testZIndexPath, secondCommitInstance).toString)
|
||||
|
||||
val expectedMergedZIndexTableDf =
|
||||
spark.read
|
||||
.schema(indexSchema)
|
||||
.json(getClass.getClassLoader.getResource("index/zorder/z-index-table-merged.json").toString)
|
||||
|
||||
assertEquals(asJson(sort(expectedMergedZIndexTableDf)), asJson(sort(replace(mergedZIndexTable))))
|
||||
assertEquals(asJson(sort(manualUpdatedColStatsTableDF)), asJson(sort(transposedUpdatedColStatsDF)))
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -249,7 +210,7 @@ class TestColumnStatsIndex extends HoodieClientTestBase {
|
||||
while (it.hasNext) {
|
||||
seq = seq :+ it.next()
|
||||
}
|
||||
seq
|
||||
seq.filter(fs => fs.getPath.getName.endsWith(".parquet"))
|
||||
}
|
||||
|
||||
spark.createDataFrame(
|
||||
@@ -296,23 +257,6 @@ class TestColumnStatsIndex extends HoodieClientTestBase {
|
||||
fs.delete(new Path(targetParquetTablePath, "_SUCCESS"), false)
|
||||
}
|
||||
|
||||
def replace(ds: Dataset[Row]): DataFrame = {
|
||||
val uuidRegexp = "[a-z0-9]{8}-[a-z0-9]{4}-[a-z0-9]{4}-[a-z0-9]{4}-[a-z0-9]{12}"
|
||||
|
||||
val uuids =
|
||||
ds.selectExpr(s"regexp_extract(file, '(${uuidRegexp})')")
|
||||
.distinct()
|
||||
.collect()
|
||||
.map(_.getString(0))
|
||||
|
||||
val uuidToIdx: UserDefinedFunction = functions.udf((fileName: String) => {
|
||||
val uuid = uuids.find(uuid => fileName.contains(uuid)).get
|
||||
fileName.replace(uuid, "xxx")
|
||||
})
|
||||
|
||||
ds.withColumn("file", uuidToIdx(ds("file")))
|
||||
}
|
||||
|
||||
private def generateRandomDataFrame(spark: SparkSession): DataFrame = {
|
||||
val sourceTableSchema =
|
||||
new StructType()
|
||||
|
||||
@@ -51,6 +51,8 @@ class TestMetadataTableWithSparkDataSource extends SparkClientFunctionalTestHarn
|
||||
|
||||
val opts: Map[String, String] = commonOpts ++ Map(
|
||||
HoodieMetadataConfig.ENABLE.key -> "true",
|
||||
HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true",
|
||||
HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS_FOR_ALL_COLUMNS.key -> "true",
|
||||
HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key -> "1"
|
||||
)
|
||||
|
||||
@@ -74,22 +76,29 @@ class TestMetadataTableWithSparkDataSource extends SparkClientFunctionalTestHarn
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val metadataDF = spark.read.format(hudi).load(s"$basePath/.hoodie/metadata")
|
||||
// Files partition of MT
|
||||
val filesPartitionDF = spark.read.format(hudi).load(s"$basePath/.hoodie/metadata/files")
|
||||
|
||||
// Smoke test
|
||||
metadataDF.show()
|
||||
filesPartitionDF.show()
|
||||
|
||||
// Query w/ 0 requested columns should be working fine
|
||||
assertEquals(4, metadataDF.count())
|
||||
assertEquals(4, filesPartitionDF.count())
|
||||
|
||||
val expectedKeys = Seq("2015/03/16", "2015/03/17", "2016/03/15", "__all_partitions__")
|
||||
val keys = metadataDF.select("key")
|
||||
val keys = filesPartitionDF.select("key")
|
||||
.collect()
|
||||
.map(_.getString(0))
|
||||
.toSeq
|
||||
.sorted
|
||||
|
||||
assertEquals(expectedKeys, keys)
|
||||
|
||||
// Column Stats Index partition of MT
|
||||
val colStatsDF = spark.read.format(hudi).load(s"$basePath/.hoodie/metadata/column_stats")
|
||||
|
||||
// Smoke test
|
||||
colStatsDF.show()
|
||||
}
|
||||
|
||||
private def parseRecords(records: Seq[String]) = {
|
||||
|
||||
@@ -27,10 +27,9 @@ import org.apache.hudi.keygen.NonpartitionedKeyGenerator
|
||||
import org.apache.hudi.testutils.SparkClientFunctionalTestHarness
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, DefaultSource, HoodieBaseRelation, HoodieSparkUtils, HoodieUnsafeRDD}
|
||||
import org.apache.parquet.hadoop.util.counters.BenchmarkCounter
|
||||
import org.apache.spark.HoodieUnsafeRDDUtils
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.{Dataset, Row, SaveMode}
|
||||
import org.apache.spark.sql.{Dataset, HoodieUnsafeRDDUtils, Row, SaveMode}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, fail}
|
||||
import org.junit.jupiter.api.{Tag, Test}
|
||||
|
||||
|
||||
@@ -19,8 +19,8 @@
|
||||
package org.apache.spark.sql.execution.benchmark
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.ColumnStatsIndexHelper.buildColumnStatsTableFor
|
||||
import org.apache.hudi.config.HoodieClusteringConfig.LayoutOptimizationStrategy
|
||||
import org.apache.hudi.index.columnstats.ColumnStatsIndexHelper
|
||||
import org.apache.hudi.sort.SpaceCurveSortingHelper
|
||||
import org.apache.spark.sql.DataFrame
|
||||
import org.apache.spark.sql.hudi.TestHoodieSqlBase
|
||||
@@ -38,7 +38,7 @@ object SpaceCurveOptimizeBenchmark extends TestHoodieSqlBase {
|
||||
|
||||
val orderedColsTypes = Seq(StructField(co1, IntegerType), StructField(co2, IntegerType))
|
||||
val colStatsIndexTable =
|
||||
ColumnStatsIndexHelper.buildColumnStatsTableFor(spark, sourceTableDF.inputFiles.toSeq, orderedColsTypes)
|
||||
buildColumnStatsTableFor(spark, sourceTableDF.inputFiles.toSeq, orderedColsTypes)
|
||||
.collect()
|
||||
.map(f => (f.getInt(1), f.getInt(2), f.getInt(4), f.getInt(5)))
|
||||
|
||||
|
||||
Reference in New Issue
Block a user