1
0

[HUDI-3664] Fixing Column Stats Index composition (#5181)

Co-authored-by: Sagar Sumit <sagarsumit09@gmail.com>
This commit is contained in:
Alexey Kudinkin
2022-04-02 17:15:52 -07:00
committed by GitHub
parent 74eb09be9b
commit cc3737be50
52 changed files with 1776 additions and 749 deletions

View File

@@ -1,281 +0,0 @@
/*
* 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.index.columnstats;
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.LongType$;
import org.apache.spark.sql.types.Metadata;
import org.apache.spark.sql.types.ShortType;
import org.apache.spark.sql.types.StringType;
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 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;
public class ColumnStatsIndexHelper {
private static final String COLUMN_STATS_INDEX_FILE_COLUMN_NAME = "file";
private static final String COLUMN_STATS_INDEX_MIN_VALUE_STAT_NAME = "minValue";
private static final String COLUMN_STATS_INDEX_MAX_VALUE_STAT_NAME = "maxValue";
private static final String COLUMN_STATS_INDEX_NUM_NULLS_STAT_NAME = "num_nulls";
public static String getMinColumnNameFor(String colName) {
return composeZIndexColName(colName, COLUMN_STATS_INDEX_MIN_VALUE_STAT_NAME);
}
public static String getMaxColumnNameFor(String colName) {
return composeZIndexColName(colName, COLUMN_STATS_INDEX_MAX_VALUE_STAT_NAME);
}
public static String getNumNullsColumnNameFor(String colName) {
return composeZIndexColName(colName, COLUMN_STATS_INDEX_NUM_NULLS_STAT_NAME);
}
/**
* @VisibleForTesting
*/
@Nonnull
public static StructType composeIndexSchema(@Nonnull List<StructField> zorderedColumnsSchemas) {
List<StructField> schema = new ArrayList<>();
schema.add(new StructField(COLUMN_STATS_INDEX_FILE_COLUMN_NAME, StringType$.MODULE$, true, Metadata.empty()));
zorderedColumnsSchemas.forEach(colSchema -> {
schema.add(composeColumnStatStructType(colSchema.name(), COLUMN_STATS_INDEX_MIN_VALUE_STAT_NAME, colSchema.dataType()));
schema.add(composeColumnStatStructType(colSchema.name(), COLUMN_STATS_INDEX_MAX_VALUE_STAT_NAME, colSchema.dataType()));
schema.add(composeColumnStatStructType(colSchema.name(), COLUMN_STATS_INDEX_NUM_NULLS_STAT_NAME, LongType$.MODULE$));
});
return StructType$.MODULE$.apply(schema);
}
private static StructField composeColumnStatStructType(String col, String statName, DataType dataType) {
return new StructField(composeZIndexColName(col, statName), dataType, true, Metadata.empty());
}
private static String composeZIndexColName(String col, String statName) {
// TODO add escaping for
return String.format("%s_%s", col, statName);
}
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);
SerializableConfiguration serializableConfiguration = new SerializableConfiguration(sc.hadoopConfiguration());
int numParallelism = (baseFilesPaths.size() / 3 + 1);
List<HoodieColumnRangeMetadata<Comparable>> colMinMaxInfos;
String previousJobDescription = sc.getLocalProperty("spark.job.description");
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),
orderedColumnSchemas.stream()
.map(StructField::name)
.collect(Collectors.toList())
)
.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 = composeIndexSchema(orderedColumnSchemas);
return sparkSession.createDataFrame(allMetaDataRDD, indexSchema);
}
}

View File

@@ -31,12 +31,11 @@ import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
import org.apache.hudi.keygen.{BaseKeyGenerator, CustomAvroKeyGenerator, CustomKeyGenerator, KeyGenerator}
import org.apache.spark.SPARK_VERSION
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.DataFrame
import org.apache.spark.sql.catalyst.encoders.RowEncoder
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, Literal}
import org.apache.spark.sql.execution.datasources.{FileStatusCache, InMemoryFileIndex}
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types.{StringType, StructField, StructType}
import org.apache.spark.sql.{DataFrame, SparkSession}
import java.util.Properties
import org.apache.hudi.internal.schema.InternalSchema

View File

@@ -318,7 +318,14 @@ public class TestClientRollback extends HoodieClientTestBase {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withRollbackUsingMarkers(false)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build())
.withMetadataConfig(
HoodieMetadataConfig.newBuilder()
// Column Stats Index is disabled, since these tests construct tables which are
// not valid (empty commit metadata, invalid parquet files)
.withMetadataIndexColumnStats(false)
.enable(enableMetadataTable)
.build()
)
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).build())
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();

View File

@@ -154,6 +154,7 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
@Tag("functional")
public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
@@ -859,30 +860,31 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
}
Schema writerSchema = new AvroSchemaConverter().convert(writerSchemaMsg);
HoodieLogFormat.Reader logFileReader = HoodieLogFormat.newReader(fs, new HoodieLogFile(fsStatus[0].getPath()), writerSchema);
while (logFileReader.hasNext()) {
HoodieLogBlock logBlock = logFileReader.next();
if (logBlock instanceof HoodieDataBlock) {
try (ClosableIterator<IndexedRecord> recordItr = ((HoodieDataBlock) logBlock).getRecordItr()) {
recordItr.forEachRemaining(indexRecord -> {
final GenericRecord record = (GenericRecord) indexRecord;
if (enableMetaFields) {
// Metadata table records should have meta fields!
assertNotNull(record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
assertNotNull(record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD));
} else {
// Metadata table records should not have meta fields!
assertNull(record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
assertNull(record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD));
}
try (HoodieLogFormat.Reader logFileReader = HoodieLogFormat.newReader(fs, new HoodieLogFile(fsStatus[0].getPath()), writerSchema)) {
while (logFileReader.hasNext()) {
HoodieLogBlock logBlock = logFileReader.next();
if (logBlock instanceof HoodieDataBlock) {
try (ClosableIterator<IndexedRecord> recordItr = ((HoodieDataBlock) logBlock).getRecordItr()) {
recordItr.forEachRemaining(indexRecord -> {
final GenericRecord record = (GenericRecord) indexRecord;
if (enableMetaFields) {
// Metadata table records should have meta fields!
assertNotNull(record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
assertNotNull(record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD));
} else {
// Metadata table records should not have meta fields!
assertNull(record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
assertNull(record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD));
}
final String key = String.valueOf(record.get(HoodieMetadataPayload.KEY_FIELD_NAME));
assertFalse(key.isEmpty());
if (enableMetaFields) {
assertTrue(key.equals(String.valueOf(record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD))));
}
});
final String key = String.valueOf(record.get(HoodieMetadataPayload.KEY_FIELD_NAME));
assertFalse(key.isEmpty());
if (enableMetaFields) {
assertTrue(key.equals(String.valueOf(record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD))));
}
});
}
}
}
}
@@ -2214,11 +2216,57 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
assertTrue(latestSlices.size()
<= (numFileVersions * metadataEnabledPartitionTypes.get(partition).getFileGroupCount()), "Should limit file slice to "
+ numFileVersions + " per file group, but was " + latestSlices.size());
List<HoodieLogFile> logFiles = latestSlices.get(0).getLogFiles().collect(Collectors.toList());
try {
if (MetadataPartitionType.FILES.getPartitionPath().equals(partition)) {
verifyMetadataRawRecords(table, logFiles, false);
}
if (MetadataPartitionType.COLUMN_STATS.getPartitionPath().equals(partition)) {
verifyMetadataColumnStatsRecords(logFiles);
}
} catch (IOException e) {
LOG.error("Metadata record validation failed", e);
fail("Metadata record validation failed");
}
});
LOG.info("Validation time=" + timer.endTimer());
}
private void verifyMetadataColumnStatsRecords(List<HoodieLogFile> logFiles) throws IOException {
for (HoodieLogFile logFile : logFiles) {
FileStatus[] fsStatus = fs.listStatus(logFile.getPath());
MessageType writerSchemaMsg = TableSchemaResolver.readSchemaFromLogFile(fs, logFile.getPath());
if (writerSchemaMsg == null) {
// not a data block
continue;
}
Schema writerSchema = new AvroSchemaConverter().convert(writerSchemaMsg);
try (HoodieLogFormat.Reader logFileReader = HoodieLogFormat.newReader(fs, new HoodieLogFile(fsStatus[0].getPath()), writerSchema)) {
while (logFileReader.hasNext()) {
HoodieLogBlock logBlock = logFileReader.next();
if (logBlock instanceof HoodieDataBlock) {
try (ClosableIterator<IndexedRecord> recordItr = ((HoodieDataBlock) logBlock).getRecordItr()) {
recordItr.forEachRemaining(indexRecord -> {
final GenericRecord record = (GenericRecord) indexRecord;
final GenericRecord colStatsRecord = (GenericRecord) record.get(HoodieMetadataPayload.SCHEMA_FIELD_ID_COLUMN_STATS);
assertNotNull(colStatsRecord);
assertNotNull(colStatsRecord.get(HoodieMetadataPayload.COLUMN_STATS_FIELD_COLUMN_NAME));
assertNotNull(colStatsRecord.get(HoodieMetadataPayload.COLUMN_STATS_FIELD_NULL_COUNT));
/**
* TODO: some types of field may have null min/max as these statistics are only supported for primitive types
* assertNotNull(colStatsRecord.get(HoodieMetadataPayload.COLUMN_STATS_FIELD_MAX_VALUE));
* assertNotNull(colStatsRecord.get(HoodieMetadataPayload.COLUMN_STATS_FIELD_MIN_VALUE));
*/
});
}
}
}
}
}
}
/**
* Returns the list of all files in the dataset by iterating over the metadata table.
*

View File

@@ -288,19 +288,19 @@ public class TestHoodieBackedTableMetadata extends TestHoodieMetadataBase {
}
Schema writerSchema = new AvroSchemaConverter().convert(writerSchemaMsg);
HoodieLogFormat.Reader logFileReader = HoodieLogFormat.newReader(fs, new HoodieLogFile(fsStatus[0].getPath()), writerSchema);
while (logFileReader.hasNext()) {
HoodieLogBlock logBlock = logFileReader.next();
if (logBlock instanceof HoodieDataBlock) {
try (ClosableIterator<IndexedRecord> recordItr = ((HoodieDataBlock) logBlock).getRecordItr()) {
recordItr.forEachRemaining(indexRecord -> {
final GenericRecord record = (GenericRecord) indexRecord;
assertNull(record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
assertNull(record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD));
final String key = String.valueOf(record.get(HoodieMetadataPayload.KEY_FIELD_NAME));
assertFalse(key.isEmpty());
});
try (HoodieLogFormat.Reader logFileReader = HoodieLogFormat.newReader(fs, new HoodieLogFile(fsStatus[0].getPath()), writerSchema)) {
while (logFileReader.hasNext()) {
HoodieLogBlock logBlock = logFileReader.next();
if (logBlock instanceof HoodieDataBlock) {
try (ClosableIterator<IndexedRecord> recordItr = ((HoodieDataBlock) logBlock).getRecordItr()) {
recordItr.forEachRemaining(indexRecord -> {
final GenericRecord record = (GenericRecord) indexRecord;
assertNull(record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
assertNull(record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD));
final String key = String.valueOf(record.get(HoodieMetadataPayload.KEY_FIELD_NAME));
assertFalse(key.isEmpty());
});
}
}
}
}

View File

@@ -1509,6 +1509,7 @@ public class TestCleaner extends HoodieClientTestBase {
protected static HoodieCommitMetadata generateCommitMetadata(
String instantTime, Map<String, List<String>> partitionToFilePaths) {
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, HoodieTestTable.PHONY_TABLE_SCHEMA);
partitionToFilePaths.forEach((partitionPath, fileList) -> fileList.forEach(f -> {
HoodieWriteStat writeStat = new HoodieWriteStat();
writeStat.setPartitionPath(partitionPath);

View File

@@ -119,7 +119,7 @@ public class TestInlineCompaction extends CompactionTestBase {
@Test
public void testSuccessfulCompactionBasedOnNumOrTime() throws Exception {
// Given: make three commits
HoodieWriteConfig cfg = getConfigForInlineCompaction(3, 20, CompactionTriggerStrategy.NUM_OR_TIME);
HoodieWriteConfig cfg = getConfigForInlineCompaction(3, 60, CompactionTriggerStrategy.NUM_OR_TIME);
try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
List<HoodieRecord> records = dataGen.generateInserts(HoodieActiveTimeline.createNewInstantTime(), 10);
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
@@ -134,7 +134,7 @@ public class TestInlineCompaction extends CompactionTestBase {
assertEquals(4, metaClient.getActiveTimeline().getWriteTimeline().countInstants());
// 4th commit, that will trigger compaction because reach the time elapsed
metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build();
finalInstant = HoodieActiveTimeline.createNewInstantTime(20000);
finalInstant = HoodieActiveTimeline.createNewInstantTime(60000);
createNextDeltaCommit(finalInstant, dataGen.generateUpdates(finalInstant, 10), writeClient, metaClient, cfg, false);
metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(cfg.getBasePath()).build();

View File

@@ -111,7 +111,10 @@ public class TestCleanPlanExecutor extends TestCleaner {
boolean simulateFailureRetry, boolean simulateMetadataFailure,
boolean enableIncrementalClean, boolean enableBootstrapSourceClean) throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withMetadataConfig(
HoodieMetadataConfig.newBuilder()
.withAssumeDatePartitioning(true)
.build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withIncrementalCleaningMode(enableIncrementalClean)
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER)
@@ -384,7 +387,13 @@ public class TestCleanPlanExecutor extends TestCleaner {
HoodieWriteConfig config =
HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withMetadataConfig(
HoodieMetadataConfig.newBuilder()
.withAssumeDatePartitioning(true)
// Column Stats Index is disabled, since these tests construct tables which are
// not valid (empty commit metadata, invalid parquet files)
.withMetadataIndexColumnStats(false)
.build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build())
.build();
@@ -422,7 +431,13 @@ public class TestCleanPlanExecutor extends TestCleaner {
HoodieWriteConfig config =
HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withMetadataConfig(
HoodieMetadataConfig.newBuilder()
.withAssumeDatePartitioning(true)
// Column Stats Index is disabled, since these tests construct tables which are
// not valid (empty commit metadata, invalid parquet files)
.withMetadataIndexColumnStats(false)
.build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1).build())
.build();

View File

@@ -17,6 +17,13 @@
package org.apache.hudi.testutils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hudi.avro.model.HoodieActionInstant;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanerPlan;
@@ -25,6 +32,7 @@ import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.SparkTaskContextSupplier;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.HoodieCleanStat;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.FileSlice;
@@ -59,19 +67,12 @@ import org.apache.hudi.metadata.FileSystemBackedTableMetadata;
import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter;
import org.apache.hudi.metadata.HoodieTableMetadata;
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
import org.apache.hudi.metadata.MetadataPartitionType;
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.WorkloadStat;
import org.apache.hudi.timeline.service.TimelineService;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
@@ -82,6 +83,7 @@ import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.TestInfo;
import scala.Tuple2;
import java.io.IOException;
import java.io.Serializable;
@@ -91,14 +93,14 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Properties;
import java.util.Random;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.function.Function;
import java.util.stream.Collectors;
import scala.Tuple2;
import static org.apache.hudi.common.util.CleanerUtils.convertCleanMetadata;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -571,7 +573,7 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
}
});
if (doFullValidation) {
runFullValidation(writeConfig, metadataTableBasePath, engineContext);
runFullValidation(table.getConfig().getMetadataConfig(), writeConfig, metadataTableBasePath, engineContext);
}
LOG.info("Validation time=" + timer.endTimer());
@@ -644,7 +646,10 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
assertEquals(metadataFilenames.size(), numFiles);
}
private void runFullValidation(HoodieWriteConfig writeConfig, String metadataTableBasePath, HoodieSparkEngineContext engineContext) {
private void runFullValidation(HoodieMetadataConfig metadataConfig,
HoodieWriteConfig writeConfig,
String metadataTableBasePath,
HoodieSparkEngineContext engineContext) {
HoodieBackedTableMetadataWriter metadataWriter = metadataWriter(writeConfig);
assertNotNull(metadataWriter, "MetadataWriter should have been initialized");
@@ -666,16 +671,25 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
// in the .hoodie folder.
List<String> metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, HoodieTableMetadata.getMetadataTableBasePath(basePath),
false, false);
Assertions.assertEquals(metadataWriter.getEnabledPartitionTypes().size(), metadataTablePartitions.size());
List<MetadataPartitionType> enabledPartitionTypes = metadataWriter.getEnabledPartitionTypes();
Assertions.assertEquals(enabledPartitionTypes.size(), metadataTablePartitions.size());
Map<String, MetadataPartitionType> partitionTypeMap = enabledPartitionTypes.stream()
.collect(Collectors.toMap(MetadataPartitionType::getPartitionPath, Function.identity()));
// Metadata table should automatically compact and clean
// versions are +1 as autoClean / compaction happens end of commits
int numFileVersions = metadataWriteConfig.getCleanerFileVersionsRetained() + 1;
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metadataMetaClient, metadataMetaClient.getActiveTimeline());
metadataTablePartitions.forEach(partition -> {
MetadataPartitionType partitionType = partitionTypeMap.get(partition);
List<FileSlice> latestSlices = fsView.getLatestFileSlices(partition).collect(Collectors.toList());
assertTrue(latestSlices.stream().map(FileSlice::getBaseFile).count() <= 1, "Should have a single latest base file");
assertTrue(latestSlices.size() <= 1, "Should have a single latest file slice");
assertTrue(latestSlices.stream().map(FileSlice::getBaseFile).filter(Objects::nonNull).count() <= partitionType.getFileGroupCount(), "Should have a single latest base file");
assertTrue(latestSlices.size() <= partitionType.getFileGroupCount(), "Should have a single latest file slice");
assertTrue(latestSlices.size() <= numFileVersions, "Should limit file slice to "
+ numFileVersions + " but was " + latestSlices.size());
});