diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java index 4e9340664e209..3f249e9f7b5a3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java @@ -37,6 +37,7 @@ import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndexUtils; import org.apache.hudi.io.HoodieRangeInfoHandle; +import org.apache.hudi.stats.ValueMetadata; import org.apache.hudi.table.HoodieTable; import org.slf4j.Logger; @@ -52,7 +53,6 @@ import static java.util.stream.Collectors.groupingBy; import static java.util.stream.Collectors.mapping; import static java.util.stream.Collectors.toList; -import static org.apache.hudi.avro.HoodieAvroWrapperUtils.unwrapAvroValueWrapper; import static org.apache.hudi.common.util.CollectionUtils.isNullOrEmpty; import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPartitions; import static org.apache.hudi.metadata.MetadataPartitionType.COLUMN_STATS; @@ -231,12 +231,13 @@ protected List> loadColumnRangesFromMetaIndex( List> result = new ArrayList<>(fileToColumnStatsMap.size()); for (Map.Entry, HoodieMetadataColumnStats> entry : fileToColumnStatsMap.entrySet()) { + ValueMetadata valueMetadata = ValueMetadata.getValueMetadata(entry.getValue().getValueType()); result.add(Pair.of(entry.getKey().getLeft(), new BloomIndexFileInfo( partitionAndFileNameToFileId.get(entry.getKey()), // NOTE: Here we assume that the type of the primary key field is string - unwrapAvroValueWrapper(entry.getValue().getMinValue()).toString(), - unwrapAvroValueWrapper(entry.getValue().getMaxValue()).toString() + valueMetadata.unwrapValue(entry.getValue().getMinValue()).toString(), + valueMetadata.unwrapValue(entry.getValue().getMaxValue()).toString() ))); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index 92b07fc39d162..9da898db8b310 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -28,7 +28,6 @@ import org.apache.hudi.common.model.DeleteRecord; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieDeltaWriteStat; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; @@ -59,7 +58,9 @@ import org.apache.hudi.exception.HoodieAppendException; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieUpsertException; +import org.apache.hudi.metadata.HoodieIndexVersion; import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.util.CommonClientUtils; @@ -83,6 +84,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.collectColumnRangeMetadata; /** @@ -432,15 +434,17 @@ protected void processAppendResult(AppendResult result, Option d updateWriteStatus(result, stat); if (config.isMetadataColumnStatsIndexEnabled()) { + HoodieIndexVersion indexVersion = HoodieTableMetadataUtil.existingIndexVersionOrDefault(PARTITION_NAME_COLUMN_STATS, hoodieTable.getMetaClient()); Set columnsToIndexSet = new HashSet<>(HoodieTableMetadataUtil .getColumnsToIndex(hoodieTable.getMetaClient().getTableConfig(), config.getMetadataConfig(), Lazy.eagerly(Option.of(writeSchemaWithMetaFields)), - Option.of(this.recordMerger.getRecordType())).keySet()); + Option.of(this.recordMerger.getRecordType()), indexVersion).keySet()); final List> fieldsToIndex = columnsToIndexSet.stream() .map(fieldName -> HoodieAvroUtils.getSchemaForField(writeSchemaWithMetaFields, fieldName)).collect(Collectors.toList()); try { Map> columnRangeMetadataMap = - collectColumnRangeMetadata(recordList.iterator(), fieldsToIndex, stat.getPath(), writeSchemaWithMetaFields, storage.getConf()); + collectColumnRangeMetadata(recordList.iterator(), fieldsToIndex, stat.getPath(), writeSchemaWithMetaFields, storage.getConf(), + indexVersion); stat.putRecordsStats(columnRangeMetadataMap); } catch (HoodieException e) { throw new HoodieAppendException("Failed to extract append result", e); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBinaryCopyHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBinaryCopyHandle.java index 47bd7b0aae42d..a39139d8d0920 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBinaryCopyHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBinaryCopyHandle.java @@ -35,7 +35,6 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hadoop.conf.Configuration; -import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.parquet.schema.MessageType; import org.slf4j.Logger; @@ -45,6 +44,8 @@ import java.util.Collections; import java.util.List; +import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; + /** * Compared to other Write Handles, HoodieBinaryCopyHandle merge multiple inputFiles into a single outputFile without performing * extra operations like data serialization/deserialization or compression/decompression. @@ -75,12 +76,12 @@ private MessageType getWriteSchema(HoodieWriteConfig config, List i return fileSchema; } catch (Exception e) { LOG.error("Failed to read schema from input file", e); - throw new HoodieIOException("Failed to read schema from input file when schema evolution is disabled: " + inputFiles.get(0), + throw new HoodieIOException("Failed to read schema from input file when schema evolution is disabled: " + inputFiles.get(0), e instanceof IOException ? (IOException) e : new IOException(e)); } } else { // Default behavior: use the table's write schema for evolution - return new AvroSchemaConverter(conf).convert(writeSchemaWithMetaFields); + return getAvroSchemaConverter(conf).convert(writeSchemaWithMetaFields); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 88180ac73b0d3..4cae3b7833f82 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -123,6 +123,7 @@ import static org.apache.hudi.metadata.HoodieMetadataWriteUtils.createMetadataWriteConfig; import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX; import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_SECONDARY_INDEX_PREFIX; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.createRecordIndexDefinition; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.existingIndexVersionOrDefault; @@ -578,10 +579,11 @@ private Pair, Pair>> initializeCo if (partitionIdToAllFilesMap.isEmpty()) { return Pair.of(Collections.emptyList(), Pair.of(fileGroupCount, engineContext.emptyHoodieData())); } + HoodieIndexVersion columnStatsIndexVersion = existingIndexVersionOrDefault(PARTITION_NAME_COLUMN_STATS, dataMetaClient); // Find the columns to index final List columnsToIndex = new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(dataMetaClient.getTableConfig(), dataWriteConfig.getMetadataConfig(), tableSchema, true, - Option.of(dataWriteConfig.getRecordMerger().getRecordType())).keySet()); + Option.of(dataWriteConfig.getRecordMerger().getRecordType()), columnStatsIndexVersion).keySet()); if (columnsToIndex.isEmpty()) { // this can only happen if meta fields are disabled and cols to index is not explicitly overridden. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/NineToEightDowngradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/NineToEightDowngradeHandler.java index d7fcdf8aaa1f0..aa62e1a20a167 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/NineToEightDowngradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/NineToEightDowngradeHandler.java @@ -83,8 +83,8 @@ public UpgradeDowngrade.TableConfigChangeSet downgrade(HoodieWriteConfig config, && isComplexKeyGeneratorWithSingleRecordKeyField(metaClient.getTableConfig())) { throw new HoodieUpgradeDowngradeException(getComplexKeygenErrorMessage("downgrade")); } - // Handle secondary index. - UpgradeDowngradeUtils.dropNonV1SecondaryIndexPartitions( + // Handle index Changes + UpgradeDowngradeUtils.dropNonV1IndexPartitions( config, context, table, upgradeDowngradeHelper, "downgrading from table version nine to eight"); // Update table properties. Set propertiesToRemove = new HashSet<>(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngradeUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngradeUtils.java index 8428059ad7134..673c3a2118039 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngradeUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngradeUtils.java @@ -56,7 +56,6 @@ import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataUtil; import org.apache.hudi.metadata.HoodieIndexVersion; -import org.apache.hudi.metadata.MetadataPartitionType; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -285,23 +284,15 @@ static boolean isMetadataTableBehindDataTable(HoodieWriteConfig config, * @param table Hoodie table * @param operationType Type of operation (upgrade/downgrade) */ - public static void dropNonV1SecondaryIndexPartitions(HoodieWriteConfig config, HoodieEngineContext context, - HoodieTable table, SupportsUpgradeDowngrade upgradeDowngradeHelper, String operationType) { + public static void dropNonV1IndexPartitions(HoodieWriteConfig config, HoodieEngineContext context, + HoodieTable table, SupportsUpgradeDowngrade upgradeDowngradeHelper, String operationType) { HoodieTableMetaClient metaClient = table.getMetaClient(); try (BaseHoodieWriteClient writeClient = upgradeDowngradeHelper.getWriteClient(config, context)) { List mdtPartitions = metaClient.getTableConfig().getMetadataPartitions() .stream() - .filter(partition -> { - // Only drop secondary indexes that are not V1 - return metaClient.getIndexForMetadataPartition(partition) - .map(indexDef -> { - if (MetadataPartitionType.fromPartitionPath(indexDef.getIndexName()).equals(MetadataPartitionType.SECONDARY_INDEX)) { - return HoodieIndexVersion.V1.lowerThan(indexDef.getVersion()); - } - return false; - }) - .orElse(false); - }) + .filter(partition -> metaClient.getIndexForMetadataPartition(partition) + .map(indexDef -> HoodieIndexVersion.V1.lowerThan(indexDef.getVersion())) + .orElse(false)) .collect(Collectors.toList()); LOG.info("Dropping from MDT partitions for {}: {}", operationType, mdtPartitions); if (!mdtPartitions.isEmpty()) { diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestNineToEightDowngradeHandler.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestNineToEightDowngradeHandler.java index 6f84dcab0bd97..4dc4123a9d904 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestNineToEightDowngradeHandler.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestNineToEightDowngradeHandler.java @@ -266,7 +266,7 @@ void testDowngradeDropsOnlyV2OrAboveIndexes() { )).thenAnswer(invocation -> null); // Do nothing // Mock the dropNonV1SecondaryIndexPartitions to simulate dropping V2 indexes - mockedUtils.when(() -> UpgradeDowngradeUtils.dropNonV1SecondaryIndexPartitions( + mockedUtils.when(() -> UpgradeDowngradeUtils.dropNonV1IndexPartitions( eq(config), eq(context), eq(table), diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/log/block/HoodieFlinkParquetDataBlock.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/log/block/HoodieFlinkParquetDataBlock.java index bebe3f807afa8..521dd3e3e206d 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/log/block/HoodieFlinkParquetDataBlock.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/log/block/HoodieFlinkParquetDataBlock.java @@ -19,7 +19,6 @@ package org.apache.hudi.io.log.block; import org.apache.hudi.avro.AvroSchemaCache; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock; @@ -29,6 +28,8 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.io.storage.ColumnRangeMetadataProvider; import org.apache.hudi.io.storage.HoodieIOFactory; +import org.apache.hudi.metadata.HoodieIndexVersion; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; import org.apache.hudi.storage.HoodieStorage; import org.apache.avro.Schema; @@ -97,10 +98,10 @@ public ByteArrayOutputStream getContentBytes(HoodieStorage storage) throws IOExc } @Override - public Map> getColumnRangeMeta(String filePath) { + public Map> getColumnRangeMeta(String filePath, HoodieIndexVersion indexVersion) { ValidationUtils.checkArgument(parquetMetadata != null, "parquetMetadata should not be null."); ParquetUtils parquetUtils = new ParquetUtils(); - List> columnMetaList = parquetUtils.readColumnStatsFromMetadata(parquetMetadata, filePath, Option.empty()); + List> columnMetaList = parquetUtils.readColumnStatsFromMetadata(parquetMetadata, filePath, Option.empty(), indexVersion); return columnMetaList.stream().collect(Collectors.toMap(HoodieColumnRangeMetadata::getColumnName, colMeta -> colMeta)); } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/v2/RowDataLogWriteHandle.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/v2/RowDataLogWriteHandle.java index 9908500ffc80c..c7f00da98d3f9 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/v2/RowDataLogWriteHandle.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/v2/RowDataLogWriteHandle.java @@ -19,7 +19,6 @@ package org.apache.hudi.io.v2; import org.apache.hudi.common.engine.TaskContextSupplier; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieDeltaWriteStat; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.log.AppendResult; @@ -36,7 +35,9 @@ import org.apache.hudi.io.log.block.HoodieFlinkAvroDataBlock; import org.apache.hudi.io.log.block.HoodieFlinkParquetDataBlock; import org.apache.hudi.io.storage.ColumnRangeMetadataProvider; +import org.apache.hudi.metadata.HoodieIndexVersion; import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.commit.BucketType; import org.apache.hudi.util.Lazy; @@ -55,6 +56,7 @@ import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME; import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION; import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS; /** * A write handle that supports creating a log file and writing records based on record Iterator. @@ -107,24 +109,22 @@ protected void processAppendResult(AppendResult result, Option d // for parquet data block, we can get column stats from parquet footer directly. if (config.isMetadataColumnStatsIndexEnabled()) { + HoodieIndexVersion indexVersion = HoodieTableMetadataUtil.existingIndexVersionOrDefault(PARTITION_NAME_COLUMN_STATS, hoodieTable.getMetaClient()); Set columnsToIndexSet = new HashSet<>(HoodieTableMetadataUtil .getColumnsToIndex(hoodieTable.getMetaClient().getTableConfig(), config.getMetadataConfig(), Lazy.eagerly(Option.of(writeSchemaWithMetaFields)), - Option.of(HoodieRecord.HoodieRecordType.FLINK)).keySet()); + Option.of(HoodieRecord.HoodieRecordType.FLINK), indexVersion).keySet()); Map> columnRangeMetadata; if (dataBlock.isEmpty()) { // only delete block exists columnRangeMetadata = new HashMap<>(); - for (String col: columnsToIndexSet) { - columnRangeMetadata.put(col, HoodieColumnRangeMetadata.create( - stat.getPath(), col, null, null, 0L, 0L, 0L, 0L)); - } + columnsToIndexSet.forEach(col -> columnRangeMetadata.put(col, HoodieColumnRangeMetadata.createEmpty(stat.getPath(), col, indexVersion))); } else { ValidationUtils.checkArgument(dataBlock.get() instanceof ColumnRangeMetadataProvider, "Log block for Flink ingestion should always be an instance of ColumnRangeMetadataProvider for collecting column stats efficiently."); columnRangeMetadata = - ((ColumnRangeMetadataProvider) dataBlock.get()).getColumnRangeMeta(stat.getPath()).entrySet().stream() + ((ColumnRangeMetadataProvider) dataBlock.get()).getColumnRangeMeta(stat.getPath(), indexVersion).entrySet().stream() .filter(e -> columnsToIndexSet.contains(e.getKey())) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkInternalSchemaConverter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkInternalSchemaConverter.java index 294e29a65fb1d..93fd328bcc78c 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkInternalSchemaConverter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkInternalSchemaConverter.java @@ -57,6 +57,7 @@ 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.TimestampNTZType$; import org.apache.spark.sql.types.TimestampType; import org.apache.spark.sql.types.TimestampType$; import org.apache.spark.sql.types.UserDefinedType; @@ -267,10 +268,14 @@ private static DataType constructSparkSchemaFromType(Type type) { case DATE: return DateType$.MODULE$; case TIME: + case TIME_MILLIS: throw new UnsupportedOperationException(String.format("cannot convert %s type to Spark", type)); case TIMESTAMP: - // todo support TimeStampNTZ + case TIMESTAMP_MILLIS: return TimestampType$.MODULE$; + case LOCAL_TIMESTAMP_MILLIS: + case LOCAL_TIMESTAMP_MICROS: + return TimestampNTZType$.MODULE$; case STRING: return StringType$.MODULE$; case UUID: @@ -280,7 +285,9 @@ private static DataType constructSparkSchemaFromType(Type type) { case BINARY: return BinaryType$.MODULE$; case DECIMAL: - Types.DecimalType decimal = (Types.DecimalType) type; + case DECIMAL_BYTES: + case DECIMAL_FIXED: + Types.DecimalBase decimal = (Types.DecimalBase) type; return DecimalType$.MODULE$.apply(decimal.precision(), decimal.scale()); default: throw new UnsupportedOperationException(String.format("cannot convert unknown type: %s to Spark", type)); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java index a4de0a7cbdce1..72613b93c3879 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java @@ -37,7 +37,6 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.function.SerializableFunction; import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieIndexDefinition; import org.apache.hudi.common.model.HoodieLogFile; @@ -60,8 +59,12 @@ import org.apache.hudi.index.expression.HoodieSparkExpressionIndex; import org.apache.hudi.index.expression.HoodieSparkExpressionIndex.ExpressionIndexComputationMetadata; import org.apache.hudi.io.storage.HoodieFileWriterFactory; +import org.apache.hudi.metadata.HoodieIndexVersion; import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; +import org.apache.hudi.stats.SparkValueMetadataUtils; +import org.apache.hudi.stats.ValueMetadata; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.util.JavaScalaConverters; @@ -75,6 +78,7 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.functions; +import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; @@ -151,7 +155,8 @@ public static ClosableIterator getRowsWithExpressionIndexMetadata(ClosableI @SuppressWarnings("checkstyle:LineLength") public static ExpressionIndexComputationMetadata getExpressionIndexRecordsUsingColumnStats(Dataset dataset, HoodieExpressionIndex expressionIndex, String columnToIndex, - Option>, HoodieData>> partitionRecordsFunctionOpt) { + Option>, HoodieData>> partitionRecordsFunctionOpt, + HoodieIndexVersion indexVersion) { // Aggregate col stats related data for the column to index Dataset columnRangeMetadataDataset = dataset .select(columnToIndex, SparkMetadataWriterUtils.getExpressionIndexColumnNames()) @@ -160,14 +165,16 @@ public static ExpressionIndexComputationMetadata getExpressionIndexRecordsUsingC functions.min(columnToIndex).alias(COLUMN_STATS_FIELD_MIN_VALUE), functions.max(columnToIndex).alias(COLUMN_STATS_FIELD_MAX_VALUE), functions.count(columnToIndex).alias(COLUMN_STATS_FIELD_VALUE_COUNT)); + // Generate column stat records using the aggregated data + ValueMetadata valueMetadata = getValueMetadataFromColumnRangeDatasetSchema(columnRangeMetadataDataset.schema(), indexVersion); HoodiePairData> rangeMetadataHoodieJavaRDD = HoodieJavaRDD.of(columnRangeMetadataDataset.javaRDD()) .flatMapToPair((SerializableFunction>>>) row -> { int baseAggregatePosition = SparkMetadataWriterUtils.getExpressionIndexColumnNames().length; long nullCount = row.getLong(baseAggregatePosition); - Comparable minValue = (Comparable) row.get(baseAggregatePosition + 1); - Comparable maxValue = (Comparable) row.get(baseAggregatePosition + 2); + Comparable minValue = SparkValueMetadataUtils.convertSparkToJava(valueMetadata, row.get(baseAggregatePosition + 1)); + Comparable maxValue = SparkValueMetadataUtils.convertSparkToJava(valueMetadata, row.get(baseAggregatePosition + 2)); long valueCount = row.getLong(baseAggregatePosition + 3); String partitionName = row.getString(0); @@ -184,7 +191,8 @@ public static ExpressionIndexComputationMetadata getExpressionIndexRecordsUsingC nullCount, valueCount, totalFileSize, - totalUncompressedSize + totalUncompressedSize, + valueMetadata ); return Collections.singletonList(Pair.of(partitionName, rangeMetadata)).iterator(); }); @@ -207,6 +215,16 @@ public static ExpressionIndexComputationMetadata getExpressionIndexRecordsUsingC : new ExpressionIndexComputationMetadata(colStatRecords); } + private static ValueMetadata getValueMetadataFromColumnRangeDatasetSchema(StructType datasetSchema, HoodieIndexVersion indexVersion) { + int baseAggregatePosition = SparkMetadataWriterUtils.getExpressionIndexColumnNames().length; + DataType minDataType = datasetSchema.apply(baseAggregatePosition + 1).dataType(); + DataType maxDataType = datasetSchema.apply(baseAggregatePosition + 2).dataType(); + if (minDataType != maxDataType) { + throw new HoodieException(String.format("Column stats data types do not match for min (%s) and max (%s)", minDataType, maxDataType)); + } + return SparkValueMetadataUtils.getValueMetadata(minDataType, indexVersion); + } + public static ExpressionIndexComputationMetadata getExpressionIndexRecordsUsingBloomFilter( Dataset dataset, String columnToIndex, HoodieStorageConfig storageConfig, String instantTime, HoodieIndexDefinition indexDefinition) { @@ -292,7 +310,7 @@ public static ExpressionIndexComputationMetadata getExprIndexRecords( // Generate expression index records if (indexDefinition.getIndexType().equalsIgnoreCase(PARTITION_NAME_COLUMN_STATS)) { - return getExpressionIndexRecordsUsingColumnStats(rowDataset, expressionIndex, columnToIndex, partitionRecordsFunctionOpt); + return getExpressionIndexRecordsUsingColumnStats(rowDataset, expressionIndex, columnToIndex, partitionRecordsFunctionOpt, indexDefinition.getVersion()); } else if (indexDefinition.getIndexType().equalsIgnoreCase(PARTITION_NAME_BLOOM_FILTERS)) { return getExpressionIndexRecordsUsingBloomFilter( rowDataset, columnToIndex, dataWriteConfig.getStorageConfig(), instantTime, indexDefinition); @@ -360,6 +378,7 @@ public static HoodiePairData> // We fetch stored Expression index records for these latest files and return HoodiePairData of partition name and list of column range metadata of these files // Step 1: Validate that partition stats is supported for the column data type + HoodieIndexVersion indexVersion = HoodieTableMetadataUtil.existingIndexVersionOrDefault(indexPartition, dataMetaClient); HoodieIndexDefinition indexDefinition = HoodieTableMetadataUtil.getHoodieIndexDefinition(indexPartition, dataMetaClient); List columnsToIndex = Collections.singletonList(indexDefinition.getSourceFields().get(0)); try { @@ -378,7 +397,7 @@ public static HoodiePairData> // filter for supported types final List validColumnsToIndex = columnsToIndexSchemaMap.stream() .filter(colSchemaPair -> HoodieTableMetadataUtil.SUPPORTED_META_FIELDS_PARTITION_STATS.contains(colSchemaPair.getKey()) - || HoodieTableMetadataUtil.isColumnTypeSupported(colSchemaPair.getValue(), recordTypeOpt)) + || HoodieTableMetadataUtil.isColumnTypeSupported(colSchemaPair.getValue(), recordTypeOpt, indexVersion)) .map(entry -> entry.getKey()) .collect(Collectors.toList()); if (validColumnsToIndex.isEmpty()) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/common/model/HoodieSparkRecord.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/common/model/HoodieSparkRecord.java index 32f7f4e57e5b7..e79b19aaea987 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/common/model/HoodieSparkRecord.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/common/model/HoodieSparkRecord.java @@ -63,7 +63,7 @@ import scala.Function1; -import static org.apache.hudi.BaseSparkInternalRecordContext.getFieldValueFromInternalRow; +import static org.apache.hudi.BaseSparkInternalRecordContext.getFieldValueFromInternalRowAsJava; import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS; import static org.apache.spark.sql.HoodieInternalRowUtils.getCachedUnsafeProjection; import static org.apache.spark.sql.types.DataTypes.StringType; @@ -205,7 +205,7 @@ public Object[] getColumnValues(Schema recordSchema, String[] columns, boolean c @Override public Object getColumnValueAsJava(Schema recordSchema, String column, Properties props) { - return getFieldValueFromInternalRow(data, recordSchema, column); + return getFieldValueFromInternalRowAsJava(data, recordSchema, column); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java index 27dbb32571bbd..594587c616168 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java @@ -461,7 +461,7 @@ private Type convertField(Schema avroFieldSchema, StructField structField, Type. .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS)).named(structField.name()); } else if (logicalType.getName().equals(LogicalTypes.localTimestampMillis().getName())) { return Types.primitive(INT64, repetition) - .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS)).named(structField.name()); + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)).named(structField.name()); } else { throw new UnsupportedOperationException("Unsupported timestamp type: " + logicalType); } @@ -485,7 +485,7 @@ private Type convertField(Schema avroFieldSchema, StructField structField, Type. } else if (dataType instanceof ArrayType) { ArrayType arrayType = (ArrayType) dataType; DataType elementType = arrayType.elementType(); - Schema avroElementSchema = resolvedSchema.getElementType(); + Schema avroElementSchema = resolvedSchema == null ? null : resolvedSchema.getElementType(); if (!writeLegacyListFormat) { return Types .buildGroup(repetition).as(LogicalTypeAnnotation.listType()) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java index 909d86b4d2c01..21d0689f2cf5d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java @@ -30,7 +30,6 @@ import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.metrics.Registry; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileGroupId; @@ -50,6 +49,7 @@ import org.apache.hudi.index.expression.HoodieSparkExpressionIndex.ExpressionIndexComputationMetadata; import org.apache.hudi.metrics.DistributedRegistry; import org.apache.hudi.metrics.MetricsReporterType; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; import org.apache.hudi.storage.StorageConfiguration; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.table.BulkInsertPartitioner; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/stats/SparkValueMetadataUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/stats/SparkValueMetadataUtils.java new file mode 100644 index 0000000000000..0a689a7e31b0a --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/stats/SparkValueMetadataUtils.java @@ -0,0 +1,157 @@ +/* + * 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.stats; + +import org.apache.hudi.SparkAdapterSupport$; +import org.apache.hudi.metadata.HoodieIndexVersion; + +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.CharType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DateType; +import org.apache.spark.sql.types.Decimal; +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.NullType; +import org.apache.spark.sql.types.ShortType; +import org.apache.spark.sql.types.StringType; +import org.apache.spark.sql.types.TimestampType; +import org.apache.spark.sql.types.VarcharType; + +import java.sql.Date; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDate; + +public class SparkValueMetadataUtils { + + public static ValueMetadata getValueMetadata(DataType dataType, HoodieIndexVersion indexVersion) { + if (indexVersion.lowerThan(HoodieIndexVersion.V2)) { + return ValueMetadata.V1EmptyMetadata.get(); + } + if (dataType == null) { + return ValueMetadata.NULL_METADATA; + } + + ValueType valueType = fromDataType(dataType); + if (valueType == ValueType.DECIMAL) { + return ValueMetadata.DecimalMetadata.create(((DecimalType) dataType).precision(), ((DecimalType) dataType).scale()); + } else { + return new ValueMetadata(valueType); + } + } + + private static ValueType fromDataType(DataType dataType) { + if (dataType instanceof NullType) { + return ValueType.NULL; + } else if (dataType instanceof BooleanType) { + return ValueType.BOOLEAN; + } else if (dataType instanceof IntegerType || dataType instanceof ShortType || dataType instanceof ByteType) { + return ValueType.INT; + } else if (dataType instanceof LongType) { + return ValueType.LONG; + } else if (dataType instanceof FloatType) { + return ValueType.FLOAT; + } else if (dataType instanceof DoubleType) { + return ValueType.DOUBLE; + } else if (dataType instanceof StringType || dataType instanceof CharType || dataType instanceof VarcharType) { + return ValueType.STRING; + } else if (dataType instanceof TimestampType) { + return ValueType.TIMESTAMP_MICROS; + } else if (dataType instanceof DecimalType) { + return ValueType.DECIMAL; + } else if (dataType instanceof DateType) { + return ValueType.DATE; + } else if (dataType instanceof BinaryType) { + return ValueType.BYTES; + } else if (SparkAdapterSupport$.MODULE$.sparkAdapter().isTimestampNTZType(dataType)) { + return ValueType.LOCAL_TIMESTAMP_MICROS; + } else { + throw new IllegalArgumentException("Unsupported data type: " + dataType); + } + } + + public static Comparable convertSparkToJava(ValueMetadata valueMetadata, Object value) { + if (value == null) { + return null; + } + + switch (valueMetadata.getValueType()) { + case V1: + return (Comparable) value; + case NULL: + return null; + case BOOLEAN: + return (Boolean) value; + case INT: + return (Integer) value; + case LONG: + return (Long) value; + case FLOAT: + return (Float) value; + case DOUBLE: + return (Double) value; + case STRING: + return (String) value; + case BYTES: + return ValueType.castToBytes(value); + case DECIMAL: + return ((Decimal) value).toJavaBigDecimal(); + case DATE: + return ValueType.castToDate(value, valueMetadata); + case TIMESTAMP_MICROS: + return ValueType.castToTimestampMicros(value, valueMetadata); + case LOCAL_TIMESTAMP_MICROS: + return ValueType.castToLocalTimestampMicros(value, valueMetadata); + case FIXED: + case UUID: + case TIME_MILLIS: + case TIME_MICROS: + case TIMESTAMP_MILLIS: + case TIMESTAMP_NANOS: + case LOCAL_TIMESTAMP_MILLIS: + case LOCAL_TIMESTAMP_NANOS: + default: + throw new IllegalStateException("Spark value metadata for expression index should never be " + valueMetadata.getValueType().name()); + } + } + + /** + * useJava8api is based of the spark config `spark.sql.datetime.java8API.enabled` + * If true, we need to return java.time.Instant and java.time.LocalDate for timestamp and date, otherwise + * we need to return java.sql.Timestamp and java.sql.Date + * + */ + public static Object convertJavaTypeToSparkType(Comparable javaVal, boolean useJava8api) { + if (!useJava8api) { + if (javaVal instanceof Instant) { + return Timestamp.from((Instant) javaVal); + } else if (javaVal instanceof LocalDate) { + return Date.valueOf((LocalDate) javaVal); + } + } + return javaVal; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java index e1d3d803ed068..7b1f48cd1affa 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java @@ -41,7 +41,6 @@ import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.format.converter.ParquetMetadataConverter; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.metadata.ParquetMetadata; @@ -58,6 +57,7 @@ import static org.apache.hudi.io.HoodieBootstrapHandle.METADATA_BOOTSTRAP_RECORD_SCHEMA; import static org.apache.hudi.io.storage.HoodieSparkIOFactory.getHoodieSparkIOFactory; +import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; class ParquetBootstrapMetadataHandler extends BaseBootstrapMetadataHandler { @@ -71,7 +71,7 @@ Schema getAvroSchema(StoragePath sourceFilePath) throws IOException { (Configuration) table.getStorageConf().unwrap(), new Path(sourceFilePath.toUri()), ParquetMetadataConverter.NO_FILTER); MessageType parquetSchema = readFooter.getFileMetaData().getSchema(); - return new AvroSchemaConverter().convert(parquetSchema); + return getAvroSchemaConverter((Configuration) table.getStorageConf().unwrap()).convert(parquetSchema); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRecordContext.java b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRecordContext.java index 1aca577da4315..d39d90de2798c 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRecordContext.java +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRecordContext.java @@ -35,9 +35,12 @@ import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; import org.apache.spark.sql.catalyst.expressions.UnsafeProjection; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; +import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; +import java.nio.ByteBuffer; import java.util.Collections; import java.util.List; import java.util.Map; @@ -58,17 +61,45 @@ protected BaseSparkInternalRecordContext() { } public static Object getFieldValueFromInternalRow(InternalRow row, Schema recordSchema, String fieldName) { + return getFieldValueFromInternalRowInternal(row, recordSchema, fieldName, false); + } + + public static Object getFieldValueFromInternalRowAsJava(InternalRow row, Schema recordSchema, String fieldName) { + return getFieldValueFromInternalRowInternal(row, recordSchema, fieldName, true); + } + + private static Object getFieldValueFromInternalRowInternal(InternalRow row, Schema recordSchema, String fieldName, boolean convertToJavaType) { StructType structType = getCachedSchema(recordSchema); scala.Option cachedNestedFieldPath = HoodieInternalRowUtils.getCachedPosList(structType, fieldName); if (cachedNestedFieldPath.isDefined()) { HoodieUnsafeRowUtils.NestedFieldPath nestedFieldPath = cachedNestedFieldPath.get(); - return HoodieUnsafeRowUtils.getNestedInternalRowValue(row, nestedFieldPath); + Object value = HoodieUnsafeRowUtils.getNestedInternalRowValue(row, nestedFieldPath); + return convertToJavaType ? sparkTypeToJavaType(value) : value; } else { return null; } } + private static Object sparkTypeToJavaType(Object value) { + if (value == null) { + return null; + } else if (value instanceof UTF8String) { + return ((UTF8String) value).toString(); + } else if (value instanceof Decimal) { + return ((Decimal) value).toJavaBigDecimal(); + } else if (value instanceof byte[]) { + return ByteBuffer.wrap((byte[]) value); + } else if (value instanceof CalendarInterval + || value instanceof InternalRow + || value instanceof org.apache.spark.sql.catalyst.util.MapData + || value instanceof org.apache.spark.sql.catalyst.util.ArrayData) { + throw new UnsupportedOperationException(String.format("Unsupported value type (%s)", value.getClass().getName())); + } else { + return value; + } + } + @Override public Object getValue(InternalRow row, Schema schema, String fieldName) { return getFieldValueFromInternalRow(row, schema, fieldName); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestWriteStatus.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestWriteStatus.java index a98821dc0f2c4..77c367b50d9c0 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestWriteStatus.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestWriteStatus.java @@ -19,12 +19,13 @@ package org.apache.hudi.client; import org.apache.hudi.common.model.DefaultHoodieRecordPayload; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordDelegate; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.util.Option; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; +import org.apache.hudi.stats.ValueMetadata; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -211,7 +212,7 @@ public void testRemoveMetadataStats() { WriteStatus status = new WriteStatus(true, 0.1); status.markSuccess(HoodieRecordDelegate.create(new HoodieKey("key", "partition")), Option.empty()); Map> stats = new HashMap<>(); - stats.put("field1", HoodieColumnRangeMetadata.create("f1", "field1", 1, 2, 0, 2, 5, 10)); + stats.put("field1", HoodieColumnRangeMetadata.create("f1", "field1", 1, 2, 0, 2, 5, 10, ValueMetadata.V1EmptyMetadata.get())); status.setStat(new HoodieWriteStat()); status.getStat().putRecordsStats(stats); assertEquals(1, status.getIndexStats().getWrittenRecordDelegates().size()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestExternalPathHandling.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestExternalPathHandling.java index 46202454d4d53..b348a1c335890 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestExternalPathHandling.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestExternalPathHandling.java @@ -31,7 +31,6 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCleaningPolicy; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieDeltaWriteStat; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.WriteOperationType; @@ -49,6 +48,8 @@ import org.apache.hudi.metadata.HoodieBackedTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; +import org.apache.hudi.stats.ValueMetadata; import org.apache.hudi.table.action.clean.CleanPlanner; import org.apache.hudi.testutils.HoodieClientTestBase; @@ -291,8 +292,8 @@ private WriteStatus createWriteStatus(String commitTime, String partitionPath, S writeStat.setTotalCorruptLogBlock(0); writeStat.setTotalRollbackBlocks(0); Map> stats = new HashMap<>(); - stats.put(FIELD_1, HoodieColumnRangeMetadata.create(filePath, FIELD_1, 1, 2, 0, 2, 5, 10)); - stats.put(FIELD_2, HoodieColumnRangeMetadata.create(filePath, FIELD_2, "a", "b", 1, 3, 10, 20)); + stats.put(FIELD_1, HoodieColumnRangeMetadata.create(filePath, FIELD_1, 1, 2, 0, 2, 5, 10, ValueMetadata.V1EmptyMetadata.get())); + stats.put(FIELD_2, HoodieColumnRangeMetadata.create(filePath, FIELD_2, "a", "b", 1, 3, 10, 20, ValueMetadata.V1EmptyMetadata.get())); writeStat.putRecordsStats(stats); writeStatus.setStat(writeStat); return writeStatus; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java index f872685e4bdee..4684ebebf5d6b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java @@ -64,7 +64,8 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.CsvSource; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -86,6 +87,8 @@ import static java.util.Arrays.asList; import static java.util.Collections.emptyList; +import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE; +import static org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ; import static org.apache.hudi.common.model.WriteOperationType.BULK_INSERT; import static org.apache.hudi.common.model.WriteOperationType.COMPACT; import static org.apache.hudi.common.model.WriteOperationType.INSERT; @@ -105,10 +108,21 @@ public class TestHoodieBackedTableMetadata extends TestHoodieMetadataBase { private static final Logger LOG = LoggerFactory.getLogger(TestHoodieBackedTableMetadata.class); + public static List testTableOperationsArgs() { + return Arrays.asList( + Arguments.of(true, 6), + Arguments.of(true, 8), + Arguments.of(true, HoodieTableVersion.current().versionCode()), + Arguments.of(false, 6), + Arguments.of(false, 8), + Arguments.of(false, HoodieTableVersion.current().versionCode()) + ); + } + @ParameterizedTest - @CsvSource({"true,6", "true,8", "false,6", "false,8"}) + @MethodSource("testTableOperationsArgs") public void testTableOperations(boolean reuseReaders, int tableVersion) throws Exception { - HoodieTableType tableType = HoodieTableType.COPY_ON_WRITE; + HoodieTableType tableType = COPY_ON_WRITE; initPath(); HoodieWriteConfig config = getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, true, false) .build(); @@ -131,10 +145,10 @@ public void testTableOperations(boolean reuseReaders, int tableVersion) throws E * @throws Exception */ @ParameterizedTest - @CsvSource({"true,6", "true,8", "false,6", "false,8"}) + @MethodSource("testTableOperationsArgs") public void testMultiReaderForHoodieBackedTableMetadata(boolean reuse, int tableVersion) throws Exception { final int taskNumber = 18; - HoodieTableType tableType = HoodieTableType.COPY_ON_WRITE; + HoodieTableType tableType = COPY_ON_WRITE; initPath(); HoodieWriteConfig config = getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, true, false) .build(); @@ -223,12 +237,23 @@ private void verifyBaseMetadataTable(boolean reuseMetadataReaders) throws IOExce }); } + public static List testMetadataTableKeyGeneratorArgs() { + return Arrays.asList( + Arguments.of(COPY_ON_WRITE, 6), + Arguments.of(COPY_ON_WRITE, 8), + Arguments.of(COPY_ON_WRITE, HoodieTableVersion.current().versionCode()), + Arguments.of(MERGE_ON_READ, 6), + Arguments.of(MERGE_ON_READ, 8), + Arguments.of(MERGE_ON_READ, HoodieTableVersion.current().versionCode()) + ); + } + /** * Verify if the Metadata table is constructed with table properties including * the right key generator class name. */ @ParameterizedTest - @CsvSource({"COPY_ON_WRITE,6", "COPY_ON_WRITE,8", "MERGE_ON_READ,6", "MERGE_ON_READ,8"}) + @MethodSource("testMetadataTableKeyGeneratorArgs") public void testMetadataTableKeyGenerator(final HoodieTableType tableType, int tableVersion) throws Exception { initPath(); HoodieWriteConfig config = getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, true, false) @@ -252,7 +277,7 @@ public void testMetadataTableKeyGenerator(final HoodieTableType tableType, int t * [HUDI-2852] Table metadata returns empty for non-exist partition. */ @ParameterizedTest - @CsvSource({"COPY_ON_WRITE,6", "COPY_ON_WRITE,8", "MERGE_ON_READ,6", "MERGE_ON_READ,8"}) + @MethodSource("testMetadataTableKeyGeneratorArgs") public void testNotExistPartition(final HoodieTableType tableType, int tableVersion) throws Exception { initPath(); HoodieWriteConfig config = getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, true, true, false, true, false) @@ -279,7 +304,7 @@ public void testNotExistPartition(final HoodieTableType tableType, int tableVers * 3. Verify table services like compaction benefit from record key deduplication feature. */ @ParameterizedTest - @CsvSource({"COPY_ON_WRITE,6", "COPY_ON_WRITE,8", "MERGE_ON_READ,6", "MERGE_ON_READ,8"}) + @MethodSource("testMetadataTableKeyGeneratorArgs") public void testMetadataRecordKeyExcludeFromPayload(final HoodieTableType tableType, int tableVersion) throws Exception { initPath(); writeConfig = getWriteConfigBuilder(true, true, false) @@ -350,7 +375,7 @@ public void testMetadataRecordKeyExcludeFromPayload(final HoodieTableType tableT * plan has not been successfully executed before the new one is scheduled. */ @ParameterizedTest - @CsvSource({"COPY_ON_WRITE,6", "COPY_ON_WRITE,8", "MERGE_ON_READ,6", "MERGE_ON_READ,8"}) + @MethodSource("testMetadataTableKeyGeneratorArgs") public void testRepeatedCleanActionsWithMetadataTableEnabled(final HoodieTableType tableType, int tableVersion) throws Exception { initPath(); writeConfig = getWriteConfigBuilder(true, true, false) diff --git a/hudi-common/src/main/avro/HoodieMetadata.avsc b/hudi-common/src/main/avro/HoodieMetadata.avsc index 847b363cdfc8c..84dc97dc67206 100644 --- a/hudi-common/src/main/avro/HoodieMetadata.avsc +++ b/hudi-common/src/main/avro/HoodieMetadata.avsc @@ -362,7 +362,8 @@ "org.apache.hudi.avro.model.DecimalWrapper", "org.apache.hudi.avro.model.TimeMicrosWrapper", "org.apache.hudi.avro.model.TimestampMicrosWrapper", - "org.apache.hudi.avro.model.LocalDateWrapper" + "org.apache.hudi.avro.model.LocalDateWrapper", + "org.apache.hudi.avro.model.ArrayWrapper" ], "default": null }, @@ -417,6 +418,35 @@ "name": "isTightBound", "type": "boolean", "default": false + }, + { + "doc": "Value type information containing enum ordinal and optional additional information", + "name": "valueType", + "type": [ + "null", + { + "type": "record", + "name": "HoodieValueTypeInfo", + "namespace": "org.apache.hudi.avro.model", + "fields": [ + { + "doc": "Enum type ordinal representing the value type", + "name": "typeOrdinal", + "type": "int" + }, + { + "doc": "Optional additional information about the value type", + "name": "additionalInfo", + "type": [ + "null", + "string" + ], + "default": null + } + ] + } + ], + "default": null } ] } diff --git a/hudi-common/src/main/java/org/apache/hudi/ParquetAdapter.java b/hudi-common/src/main/java/org/apache/hudi/ParquetAdapter.java new file mode 100644 index 0000000000000..a495fd83afab6 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/ParquetAdapter.java @@ -0,0 +1,48 @@ +/* + * 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.hudi.common.util.ReflectionUtils; +import org.apache.hudi.stats.ValueType; + +import org.apache.parquet.schema.PrimitiveType; + +/** + * LogicalTypeAnnotations are added in parquet 1.11.0 + * For versions < 1.11.0, we use OriginalType, which LogicalTypeAnnotations replaced + */ +public interface ParquetAdapter { + + static ParquetAdapter getAdapter() { + try { + return ReflectionUtils.loadClass("org.apache.parquet.schema.LogicalTypeParquetAdapter"); + } catch (Throwable t) { + return ReflectionUtils.loadClass("org.apache.parquet.schema.OriginalTypeParquetAdapter"); + } + } + + boolean hasAnnotation(PrimitiveType primitiveType); + + ValueType getValueTypeFromAnnotation(PrimitiveType primitiveType); + + int getPrecision(PrimitiveType primitiveType); + + int getScale(PrimitiveType primitiveType); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java index 3e670d074e03e..f5b6eca67f333 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java @@ -224,7 +224,7 @@ private static boolean isProjectionOfInternal(Schema sourceSchema, return atomicTypeEqualityPredicate.apply(sourceSchema, targetSchema); } - public static Option findNestedFieldType(Schema schema, String fieldName) { + public static Option findNestedFieldSchema(Schema schema, String fieldName) { if (StringUtils.isNullOrEmpty(fieldName)) { return Option.empty(); } @@ -237,7 +237,11 @@ public static Option findNestedFieldType(Schema schema, String fiel } schema = foundField.schema(); } - return Option.of(resolveNullableSchema(schema).getType()); + return Option.of(resolveNullableSchema(schema)); + } + + public static Option findNestedFieldType(Schema schema, String fieldName) { + return findNestedFieldSchema(schema, fieldName).map(Schema::getType); } /** @@ -403,7 +407,8 @@ public static boolean areSchemasProjectionEquivalent(Schema schema1, Schema sche * @param requiredSchema the target schema that defines the desired structure and field requirements * @param mandatoryFields a set of top level field names that should be included from the required schema * even if they don't exist in the data schema. This allows for fields like cdc operation - * don't exist in the data schema + * don't exist in the data schema. We keep the types matching the required schema because + * timestamp partition cols can be read as a different type than the data schema * * @return a new pruned schema that matches the required schema structure while preserving * data schema metadata where possible @@ -424,17 +429,19 @@ private static Schema pruneDataSchemaInternal(Schema dataSchema, Schema required } List newFields = new ArrayList<>(); for (Schema.Field requiredSchemaField : requiredSchema.getFields()) { - Schema.Field dataSchemaField = dataSchema.getField(requiredSchemaField.name()); - if (dataSchemaField != null) { - Schema.Field newField = createNewSchemaField( - dataSchemaField.name(), - pruneDataSchema(dataSchemaField.schema(), requiredSchemaField.schema(), Collections.emptySet()), - dataSchemaField.doc(), - dataSchemaField.defaultVal() - ); - newFields.add(newField); - } else if (mandatoryFields.contains(requiredSchemaField.name())) { + if (mandatoryFields.contains(requiredSchemaField.name())) { newFields.add(createNewSchemaField(requiredSchemaField)); + } else { + Schema.Field dataSchemaField = dataSchema.getField(requiredSchemaField.name()); + if (dataSchemaField != null) { + Schema.Field newField = createNewSchemaField( + dataSchemaField.name(), + pruneDataSchema(dataSchemaField.schema(), requiredSchemaField.schema(), Collections.emptySet()), + dataSchemaField.doc(), + dataSchemaField.defaultVal() + ); + newFields.add(newField); + } } } Schema newRecord = Schema.createRecord(dataSchema.getName(), dataSchema.getDoc(), dataSchema.getNamespace(), false); diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/ConvertingGenericData.java b/hudi-common/src/main/java/org/apache/hudi/avro/ConvertingGenericData.java index 0282995d4dd3f..92eef5b58fefa 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/ConvertingGenericData.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/ConvertingGenericData.java @@ -43,12 +43,10 @@ public class ConvertingGenericData extends GenericData { private static final TimeConversions.TimeMicrosConversion TIME_MICROS_CONVERSION = new TimeConversions.TimeMicrosConversion(); private static final TimeConversions.TimestampMicrosConversion TIMESTAMP_MICROS_CONVERSION = new TimeConversions.TimestampMicrosConversion(); - // NOTE: Those are not supported in Avro 1.8.2 - // TODO re-enable upon upgrading to 1.10 - // private static final TimeConversions.TimestampMillisConversion TIMESTAMP_MILLIS_CONVERSION = new TimeConversions.TimestampMillisConversion(); - // private static final TimeConversions.TimeMillisConversion TIME_MILLIS_CONVERSION = new TimeConversions.TimeMillisConversion(); - // private static final TimeConversions.LocalTimestampMillisConversion LOCAL_TIMESTAMP_MILLIS_CONVERSION = new TimeConversions.LocalTimestampMillisConversion(); - // private static final TimeConversions.LocalTimestampMicrosConversion LOCAL_TIMESTAMP_MICROS_CONVERSION = new TimeConversions.LocalTimestampMicrosConversion(); + private static final TimeConversions.TimestampMillisConversion TIMESTAMP_MILLIS_CONVERSION = new TimeConversions.TimestampMillisConversion(); + private static final TimeConversions.TimeMillisConversion TIME_MILLIS_CONVERSION = new TimeConversions.TimeMillisConversion(); + private static final TimeConversions.LocalTimestampMillisConversion LOCAL_TIMESTAMP_MILLIS_CONVERSION = new TimeConversions.LocalTimestampMillisConversion(); + private static final TimeConversions.LocalTimestampMicrosConversion LOCAL_TIMESTAMP_MICROS_CONVERSION = new TimeConversions.LocalTimestampMicrosConversion(); public static final GenericData INSTANCE = new ConvertingGenericData(); @@ -59,11 +57,10 @@ private ConvertingGenericData() { addLogicalTypeConversion(TIME_MICROS_CONVERSION); addLogicalTypeConversion(TIMESTAMP_MICROS_CONVERSION); // NOTE: Those are not supported in Avro 1.8.2 - // TODO re-enable upon upgrading to 1.10 - // addLogicalTypeConversion(TIME_MILLIS_CONVERSION); - // addLogicalTypeConversion(TIMESTAMP_MILLIS_CONVERSION); - // addLogicalTypeConversion(LOCAL_TIMESTAMP_MILLIS_CONVERSION); - // addLogicalTypeConversion(LOCAL_TIMESTAMP_MICROS_CONVERSION); + addLogicalTypeConversion(TIME_MILLIS_CONVERSION); + addLogicalTypeConversion(TIMESTAMP_MILLIS_CONVERSION); + addLogicalTypeConversion(LOCAL_TIMESTAMP_MILLIS_CONVERSION); + addLogicalTypeConversion(LOCAL_TIMESTAMP_MICROS_CONVERSION); } @Override @@ -128,7 +125,10 @@ public boolean validate(Schema schema, Object datum) { case LONG: return isLong(datum) || TIME_MICROS_CONVERSION.getConvertedType().isInstance(datum) - || TIMESTAMP_MICROS_CONVERSION.getConvertedType().isInstance(datum); + || TIMESTAMP_MICROS_CONVERSION.getConvertedType().isInstance(datum) + || TIMESTAMP_MILLIS_CONVERSION.getConvertedType().isInstance(datum) + || LOCAL_TIMESTAMP_MICROS_CONVERSION.getConvertedType().isInstance(datum) + || LOCAL_TIMESTAMP_MILLIS_CONVERSION.getConvertedType().isInstance(datum); case FLOAT: return isFloat(datum); case DOUBLE: diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java index 78e1a846a99f4..5c73673736a10 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.config.SerializableSchema; import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.DateTimeUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.SpillableMapUtils; import org.apache.hudi.common.util.StringUtils; @@ -1196,12 +1197,33 @@ public static Object rewritePrimaryType(Object oldValue, Schema oldSchema, Schem case NULL: case BOOLEAN: case INT: - case LONG: case FLOAT: case DOUBLE: case BYTES: case STRING: return oldValue; + case LONG: + if (oldSchema.getLogicalType() != newSchema.getLogicalType()) { + if (oldSchema.getLogicalType() instanceof LogicalTypes.TimestampMillis) { + if (newSchema.getLogicalType() instanceof LogicalTypes.TimestampMicros) { + return DateTimeUtils.millisToMicros((Long) oldValue); + } + } else if (oldSchema.getLogicalType() instanceof LogicalTypes.TimestampMicros) { + if (newSchema.getLogicalType() instanceof LogicalTypes.TimestampMillis) { + return DateTimeUtils.microsToMillis((Long) oldValue); + } + } else if (oldSchema.getLogicalType() instanceof LogicalTypes.LocalTimestampMillis) { + if (newSchema.getLogicalType() instanceof LogicalTypes.LocalTimestampMicros) { + return DateTimeUtils.millisToMicros((Long) oldValue); + } + } else if (oldSchema.getLogicalType() instanceof LogicalTypes.LocalTimestampMicros) { + if (newSchema.getLogicalType() instanceof LogicalTypes.LocalTimestampMillis) { + return DateTimeUtils.microsToMillis((Long) oldValue); + } + } + throw new HoodieAvroSchemaException("Long type logical change from " + oldSchema.getLogicalType() + " to " + newSchema.getLogicalType() + " is not supported"); + } + return oldValue; case FIXED: if (oldSchema.getFixedSize() != newSchema.getFixedSize()) { // Check whether this is a [[Decimal]]'s precision change @@ -1267,6 +1289,9 @@ private static Object rewritePrimaryTypeWithDiffSchemaType(Object oldValue, Sche } break; case BYTES: + if (oldSchema.getType() == Schema.Type.FIXED) { + return ByteBuffer.wrap(((GenericFixed) oldValue).bytes()); + } if (oldSchema.getType() == Schema.Type.STRING) { return ByteBuffer.wrap(getUTF8Bytes(oldValue.toString())); } @@ -1335,8 +1360,12 @@ public static Object convertBytesToFixed(byte[] bytes, Schema schema) { * This is also what Conversions.DecimalConversion.toBytes() outputs inside a byte buffer */ public static BigDecimal convertBytesToBigDecimal(byte[] value, LogicalTypes.Decimal decimal) { + return convertBytesToBigDecimal(value, decimal.getPrecision(), decimal.getScale()); + } + + public static BigDecimal convertBytesToBigDecimal(byte[] value, int precision, int scale) { return new BigDecimal(new BigInteger(value), - decimal.getScale(), new MathContext(decimal.getPrecision(), RoundingMode.HALF_UP)); + scale, new MathContext(precision, RoundingMode.HALF_UP)); } public static boolean hasDecimalField(Schema schema) { diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroWrapperUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroWrapperUtils.java index 3ba3ecb63658c..c3c0d33c54c61 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroWrapperUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroWrapperUtils.java @@ -50,6 +50,7 @@ import java.time.Instant; import java.time.LocalDate; import java.util.List; +import java.util.function.Function; import java.util.stream.Collectors; import static org.apache.hudi.common.util.DateTimeUtils.instantToMicros; @@ -127,22 +128,21 @@ public static Object wrapValueIntoAvro(Comparable value) { .setValue(instantToMicros(instant)) .build(); } else if (value instanceof Boolean) { - return BooleanWrapper.newBuilder(BOOLEAN_WRAPPER_BUILDER_STUB.get()).setValue((Boolean) value).build(); + return wrapBoolean(value); } else if (value instanceof Integer) { - return IntWrapper.newBuilder(INT_WRAPPER_BUILDER_STUB.get()).setValue((Integer) value).build(); + return wrapInt(value); } else if (value instanceof Long) { - return LongWrapper.newBuilder(LONG_WRAPPER_BUILDER_STUB.get()).setValue((Long) value).build(); + return wrapLong(value); } else if (value instanceof Float) { - return FloatWrapper.newBuilder(FLOAT_WRAPPER_BUILDER_STUB.get()).setValue((Float) value).build(); + return wrapFloat(value); } else if (value instanceof Double) { - return DoubleWrapper.newBuilder(DOUBLE_WRAPPER_BUILDER_STUB.get()).setValue((Double) value).build(); + return wrapDouble(value); } else if (value instanceof ByteBuffer) { - return BytesWrapper.newBuilder(BYTES_WRAPPER_BUILDER_STUB.get()).setValue((ByteBuffer) value).build(); + return wrapBytes(value); } else if (value instanceof String || value instanceof Utf8) { - return StringWrapper.newBuilder(STRING_WRAPPER_BUILDER_STUB.get()).setValue(value.toString()).build(); + return wrapString(value.toString()); } else if (value instanceof ArrayComparable) { - List avroValues = OrderingValues.getValues((ArrayComparable) value).stream().map(HoodieAvroWrapperUtils::wrapValueIntoAvro).collect(Collectors.toList()); - return ArrayWrapper.newBuilder(ARRAY_WRAPPER_BUILDER_STUB.get()).setWrappedValues(avroValues).build(); + return wrapArray(value, HoodieAvroWrapperUtils::wrapValueIntoAvro); } else { throw new UnsupportedOperationException(String.format("Unsupported type of the value (%s)", value.getClass())); } @@ -174,30 +174,26 @@ public static Comparable unwrapAvroValueWrapper(Object avroValueWrapper) { } else if (avroValueWrapper instanceof TimestampMicrosWrapper) { return microsToInstant(((TimestampMicrosWrapper) avroValueWrapper).getValue()); } else if (avroValueWrapper instanceof BooleanWrapper) { - return ((BooleanWrapper) avroValueWrapper).getValue(); + return unwrapBoolean(avroValueWrapper); } else if (avroValueWrapper instanceof IntWrapper) { - return ((IntWrapper) avroValueWrapper).getValue(); + return unwrapInt(avroValueWrapper); } else if (avroValueWrapper instanceof LongWrapper) { - return ((LongWrapper) avroValueWrapper).getValue(); + return unwrapLong(avroValueWrapper); } else if (avroValueWrapper instanceof FloatWrapper) { - return ((FloatWrapper) avroValueWrapper).getValue(); + return unwrapFloat(avroValueWrapper); } else if (avroValueWrapper instanceof DoubleWrapper) { - return ((DoubleWrapper) avroValueWrapper).getValue(); + return unwrapDouble(avroValueWrapper); } else if (avroValueWrapper instanceof BytesWrapper) { - return ((BytesWrapper) avroValueWrapper).getValue(); + return unwrapBytes(avroValueWrapper); } else if (avroValueWrapper instanceof StringWrapper) { - return ((StringWrapper) avroValueWrapper).getValue(); + return unwrapString(avroValueWrapper); } else if (avroValueWrapper instanceof ArrayWrapper) { ArrayWrapper arrayWrapper = (ArrayWrapper) avroValueWrapper; return OrderingValues.create(arrayWrapper.getWrappedValues().stream() .map(HoodieAvroWrapperUtils::unwrapAvroValueWrapper) .toArray(Comparable[]::new)); } else if (avroValueWrapper instanceof GenericRecord) { - // NOTE: This branch could be hit b/c Avro records could be reconstructed - // as {@code GenericRecord) - // TODO add logical type decoding - GenericRecord genRec = (GenericRecord) avroValueWrapper; - return (Comparable) genRec.get("value"); + return unwrapGenericRecord(avroValueWrapper); } else { throw new UnsupportedOperationException(String.format("Unsupported type of the value (%s)", avroValueWrapper.getClass())); } @@ -238,4 +234,120 @@ private static Pair getIsValueWrapperObfuscated(Object statsVal } return Pair.of(false, null); } + + public enum PrimitiveWrapperType { + V1(Object.class, HoodieAvroWrapperUtils::wrapValueIntoAvro, HoodieAvroWrapperUtils::unwrapAvroValueWrapper, GenericRecord.class), + NULL(Void.class, HoodieAvroWrapperUtils::wrapNull, HoodieAvroWrapperUtils::unwrapNull, Void.class), + BOOLEAN(Boolean.class, HoodieAvroWrapperUtils::wrapBoolean, HoodieAvroWrapperUtils::unwrapBoolean, BooleanWrapper.class), + INT(Integer.class, HoodieAvroWrapperUtils::wrapInt, HoodieAvroWrapperUtils::unwrapInt, IntWrapper.class), + LONG(Long.class, HoodieAvroWrapperUtils::wrapLong, HoodieAvroWrapperUtils::unwrapLong, LongWrapper.class), + FLOAT(Float.class, HoodieAvroWrapperUtils::wrapFloat, HoodieAvroWrapperUtils::unwrapFloat, FloatWrapper.class), + DOUBLE(Double.class, HoodieAvroWrapperUtils::wrapDouble, HoodieAvroWrapperUtils::unwrapDouble, DoubleWrapper.class), + STRING(String.class, HoodieAvroWrapperUtils::wrapString, HoodieAvroWrapperUtils::unwrapString, StringWrapper.class), + BYTES(ByteBuffer.class, HoodieAvroWrapperUtils::wrapBytes, HoodieAvroWrapperUtils::unwrapBytes, BytesWrapper.class); + + private final Class clazz; + private final Function, Object> wrapper; + private final Function> unwrapper; + private final Class wrapperClass; + + PrimitiveWrapperType(Class clazz, Function, Object> wrapper, Function> unwrapper, Class wrapperClass) { + this.clazz = clazz; + this.wrapper = wrapper; + this.unwrapper = unwrapper; + this.wrapperClass = wrapperClass; + } + + public Class getClazz() { + return clazz; + } + + public Object wrap(Comparable value) { + return wrapper.apply(value); + } + + public Comparable unwrap(Object value) { + return unwrapper.apply(value); + } + + public Class getWrapperClass() { + return wrapperClass; + } + } + + private static Object wrapNull(Comparable value) { + return value; + } + + private static Object wrapBoolean(Comparable value) { + return BooleanWrapper.newBuilder(BOOLEAN_WRAPPER_BUILDER_STUB.get()).setValue((Boolean) value).build(); + } + + private static Object wrapInt(Comparable value) { + return IntWrapper.newBuilder(INT_WRAPPER_BUILDER_STUB.get()).setValue((Integer) value).build(); + } + + private static Object wrapLong(Comparable value) { + return LongWrapper.newBuilder(LONG_WRAPPER_BUILDER_STUB.get()).setValue((Long) value).build(); + } + + private static Object wrapFloat(Comparable value) { + return FloatWrapper.newBuilder(FLOAT_WRAPPER_BUILDER_STUB.get()).setValue((Float) value).build(); + } + + private static Object wrapDouble(Comparable value) { + return DoubleWrapper.newBuilder(DOUBLE_WRAPPER_BUILDER_STUB.get()).setValue((Double) value).build(); + } + + private static Object wrapString(Comparable value) { + return StringWrapper.newBuilder(STRING_WRAPPER_BUILDER_STUB.get()).setValue((String) value).build(); + } + + private static Object wrapBytes(Comparable value) { + return BytesWrapper.newBuilder(BYTES_WRAPPER_BUILDER_STUB.get()).setValue((ByteBuffer) value).build(); + } + + public static Object wrapArray(Comparable value, Function, Object> wrapper) { + List avroValues = OrderingValues.getValues((ArrayComparable) value).stream().map(wrapper::apply).collect(Collectors.toList()); + return ArrayWrapper.newBuilder(ARRAY_WRAPPER_BUILDER_STUB.get()).setWrappedValues(avroValues).build(); + } + + private static Comparable unwrapNull(Object val) { + return (Comparable) val; + } + + private static Comparable unwrapBoolean(Object val) { + return ((BooleanWrapper) val).getValue(); + } + + private static Comparable unwrapInt(Object val) { + return ((IntWrapper) val).getValue(); + } + + private static Comparable unwrapLong(Object val) { + return ((LongWrapper) val).getValue(); + } + + private static Comparable unwrapFloat(Object val) { + return ((FloatWrapper) val).getValue(); + } + + private static Comparable unwrapDouble(Object val) { + return ((DoubleWrapper) val).getValue(); + } + + private static Comparable unwrapString(Object val) { + return ((StringWrapper) val).getValue(); + } + + private static Comparable unwrapBytes(Object val) { + return ((BytesWrapper) val).getValue(); + } + + // NOTE: This branch could be hit b/c Avro records could be reconstructed + // as {@code GenericRecord) + public static Comparable unwrapGenericRecord(Object val) { + GenericRecord genRec = (GenericRecord) val; + return (Comparable) genRec.get("value"); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/processors/Parser.java b/hudi-common/src/main/java/org/apache/hudi/avro/processors/Parser.java index 286516a28cb9b..ee011ae1c4642 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/processors/Parser.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/processors/Parser.java @@ -20,6 +20,8 @@ import org.apache.hudi.common.util.collection.Pair; +import java.time.LocalDate; + public abstract class Parser { abstract Pair handleNumberValue(Number value); @@ -46,21 +48,19 @@ public Pair handleStringValue(String value) { public static class DateParser extends Parser { - private static final long MILLI_SECONDS_PER_DAY = 86400000; - @Override public Pair handleNumberValue(Number value) { - return Pair.of(true, new java.sql.Date(value.intValue() * MILLI_SECONDS_PER_DAY)); + return Pair.of(true, LocalDate.ofEpochDay(value.intValue())); } @Override public Pair handleStringNumber(String value) { - return Pair.of(true, new java.sql.Date(Integer.parseInt(value) * MILLI_SECONDS_PER_DAY)); + return Pair.of(true, LocalDate.ofEpochDay(Integer.parseInt(value))); } @Override public Pair handleStringValue(String value) { - return Pair.of(true, java.sql.Date.valueOf(value)); + return Pair.of(true, java.sql.Date.valueOf(value).toLocalDate()); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieWriteStat.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieWriteStat.java index 820a255853bb3..5f280d3e7de32 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieWriteStat.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieWriteStat.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.table.read.HoodieReadStats; import org.apache.hudi.common.util.JsonUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; import org.apache.hudi.storage.StoragePath; import com.fasterxml.jackson.annotation.JsonIgnore; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java index 9dde7727806c2..3fab8c7220453 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java @@ -52,6 +52,12 @@ public static Instant microsToInstant(long microsFromEpoch) { return Instant.ofEpochSecond(epochSeconds, nanoAdjustment); } + public static Instant nanosToInstant(long nanosFromEpoch) { + long epochSeconds = nanosFromEpoch / (1_000_000_000L); + long nanoAdjustment = nanosFromEpoch % (1_000_000_000L); + return Instant.ofEpochSecond(epochSeconds, nanoAdjustment); + } + /** * Converts provided {@link Instant} to microseconds (from epoch) */ @@ -71,6 +77,44 @@ public static long instantToMicros(Instant instant) { } } + /** + * This is based off instantToMicros above. + * */ + public static long instantToNanos(Instant instant) { + long seconds = instant.getEpochSecond(); + int nanos = instant.getNano(); + + if (seconds < 0 && nanos > 0) { + // Shift seconds by +1, then subtract a full second in nanos + long totalNanos = Math.multiplyExact(seconds + 1, 1_000_000_000L); + long adjustment = nanos - 1_000_000_000L; + return Math.addExact(totalNanos, adjustment); + } else { + long totalNanos = Math.multiplyExact(seconds, 1_000_000_000L); + return Math.addExact(totalNanos, nanos); + } + } + + public static final long MICROS_PER_MILLIS = 1000L; + + /** + * Converts the timestamp to milliseconds since epoch. In Spark timestamp values have microseconds + * precision, so this conversion is lossy. + */ + public static Long microsToMillis(Long micros) { + // When the timestamp is negative i.e before 1970, we need to adjust the milliseconds portion. + // Example - 1965-01-01 10:11:12.123456 is represented as (-157700927876544) in micro precision. + // In millis precision the above needs to be represented as (-157700927877). + return Math.floorDiv(micros, MICROS_PER_MILLIS); + } + + /** + * Converts milliseconds since the epoch to microseconds. + */ + public static Long millisToMicros(Long millis) { + return Math.multiplyExact(millis, MICROS_PER_MILLIS); + } + /** * Parse input String to a {@link java.time.Instant}. * diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/FileFormatUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/FileFormatUtils.java index ce7ae934c27b6..3c59563e3610d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/FileFormatUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/FileFormatUtils.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.bloom.BloomFilterTypeCode; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -31,7 +30,10 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.metadata.HoodieIndexVersion; import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; +import org.apache.hudi.stats.ValueMetadata; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; @@ -64,10 +66,24 @@ public abstract class FileFormatUtils { * @param fileColumnRanges List of column range statistics for each file in a partition */ public static > HoodieColumnRangeMetadata getColumnRangeInPartition(String relativePartitionPath, + String columnName, @Nonnull List> fileColumnRanges, - Map colsToIndexSchemaMap) { + Map colsToIndexSchemaMap, + HoodieIndexVersion indexVersion) { ValidationUtils.checkArgument(!fileColumnRanges.isEmpty(), "fileColumnRanges should not be empty."); + if (indexVersion.greaterThanOrEquals(HoodieIndexVersion.V2)) { + ValueMetadata valueMetadata = ValueMetadata.getValueMetadata(colsToIndexSchemaMap.get(columnName), indexVersion); + return fileColumnRanges.stream() + .map(e -> { + T minValue = (T) valueMetadata.standardizeJavaTypeAndPromote(e.getMinValue()); + T maxValue = (T) valueMetadata.standardizeJavaTypeAndPromote(e.getMaxValue()); + return HoodieColumnRangeMetadata.create( + relativePartitionPath, e.getColumnName(), minValue, maxValue, e.getNullCount(), e.getValueCount(), e.getTotalSize(), + e.getTotalUncompressedSize(), valueMetadata); + }).reduce(HoodieColumnRangeMetadata::merge).orElseThrow(() -> new HoodieException("MergingColumnRanges failed.")); + } + // Let's do one pass and deduce all columns that needs to go through schema evolution. Map>> schemaSeenForColsToIndex = new HashMap<>(); Set colsWithSchemaEvolved = new HashSet<>(); @@ -91,7 +107,8 @@ public static > HoodieColumnRangeMetadata getColumnRa return fileColumnRanges.stream() .map(e -> HoodieColumnRangeMetadata.create( relativePartitionPath, e.getColumnName(), e.getMinValue(), e.getMaxValue(), - e.getNullCount(), e.getValueCount(), e.getTotalSize(), e.getTotalUncompressedSize())) + e.getNullCount(), e.getValueCount(), e.getTotalSize(), e.getTotalUncompressedSize(), + e.getValueMetadata())) .reduce((a,b) -> { if (colsWithSchemaEvolved.isEmpty() || colsToIndexSchemaMap.isEmpty() || a.getMinValue() == null || a.getMaxValue() == null || b.getMinValue() == null || b.getMaxValue() == null @@ -103,11 +120,11 @@ public static > HoodieColumnRangeMetadata getColumnRa HoodieColumnRangeMetadata left = HoodieColumnRangeMetadata.create(a.getFilePath(), a.getColumnName(), (T) HoodieTableMetadataUtil.coerceToComparable(schema, a.getMinValue()), (T) HoodieTableMetadataUtil.coerceToComparable(schema, a.getMaxValue()), a.getNullCount(), - a.getValueCount(), a.getTotalSize(), a.getTotalUncompressedSize()); + a.getValueCount(), a.getTotalSize(), a.getTotalUncompressedSize(), a.getValueMetadata()); HoodieColumnRangeMetadata right = HoodieColumnRangeMetadata.create(b.getFilePath(), b.getColumnName(), (T) HoodieTableMetadataUtil.coerceToComparable(schema, b.getMinValue()), (T) HoodieTableMetadataUtil.coerceToComparable(schema, b.getMaxValue()), b.getNullCount(), - b.getValueCount(), b.getTotalSize(), b.getTotalUncompressedSize()); + b.getValueCount(), b.getTotalSize(), b.getTotalUncompressedSize(), b.getValueMetadata()); return HoodieColumnRangeMetadata.merge(left, right); } }).orElseThrow(() -> new HoodieException("MergingColumnRanges failed.")); @@ -302,7 +319,8 @@ public abstract ClosableIterator> fetchRecordKeysWithPosit @SuppressWarnings("rawtype") public abstract List> readColumnStatsFromMetadata(HoodieStorage storage, StoragePath filePath, - List columnList); + List columnList, + HoodieIndexVersion indexVersion); /** * @return The subclass's {@link HoodieFileFormat}. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/HFileUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/HFileUtils.java index 275059f57e310..465667fe3c25b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/HFileUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/HFileUtils.java @@ -22,7 +22,6 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.config.HoodieReaderConfig; import org.apache.hudi.common.model.HoodieAvroIndexedRecord; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -38,6 +37,8 @@ import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieIOFactory; import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.metadata.HoodieIndexVersion; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; @@ -166,7 +167,7 @@ public Schema readAvroSchema(HoodieStorage storage, StoragePath filePath) { } @Override - public List> readColumnStatsFromMetadata(HoodieStorage storage, StoragePath filePath, List columnList) { + public List> readColumnStatsFromMetadata(HoodieStorage storage, StoragePath filePath, List columnList, HoodieIndexVersion indexVersion) { throw new UnsupportedOperationException( "Reading column statistics from metadata is not supported for HFile format yet"); } diff --git a/hudi-common/src/main/java/org/apache/hudi/expression/Comparators.java b/hudi-common/src/main/java/org/apache/hudi/expression/Comparators.java index 14e208197f14f..29bdc142c8817 100644 --- a/hudi-common/src/main/java/org/apache/hudi/expression/Comparators.java +++ b/hudi-common/src/main/java/org/apache/hudi/expression/Comparators.java @@ -40,6 +40,9 @@ public class Comparators { put(Types.DateType.get(), Comparator.naturalOrder()); put(Types.TimeType.get(), Comparator.naturalOrder()); put(Types.TimestampType.get(), Comparator.naturalOrder()); + put(Types.TimestampMillisType.get(), Comparator.naturalOrder()); + put(Types.LocalTimestampMillisType.get(), Comparator.naturalOrder()); + put(Types.LocalTimestampMicrosType.get(), Comparator.naturalOrder()); put(Types.StringType.get(), Comparator.naturalOrder()); put(Types.UUIDType.get(), Comparator.naturalOrder()); } diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java index bc8b89004d695..31231bf96f591 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Type.java @@ -64,7 +64,14 @@ enum TypeID { TIME(Long.class), TIMESTAMP(Long.class), DECIMAL(BigDecimal.class), - UUID(UUID.class); + UUID(UUID.class), + DECIMAL_BYTES(BigDecimal.class), + DECIMAL_FIXED(BigDecimal.class), + TIME_MILLIS(Integer.class), + TIMESTAMP_MILLIS(Long.class), + LOCAL_TIMESTAMP_MILLIS(Long.class), + LOCAL_TIMESTAMP_MICROS(Long.class); + private final String name; private final Class classTag; @@ -107,6 +114,8 @@ static Object fromPartitionString(String partitionValue, Type type) { case FLOAT: return Float.parseFloat(partitionValue); case DECIMAL: + case DECIMAL_BYTES: + case DECIMAL_FIXED: return new BigDecimal(partitionValue); case DOUBLE: return Double.parseDouble(partitionValue); diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java index ac0768897584c..9ae3bf6bdbfa3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java @@ -299,18 +299,12 @@ public int hashCode() { } } - /** - * Decimal primitive type. - */ - public static class DecimalType extends PrimitiveType { - public static DecimalType get(int precision, int scale) { - return new DecimalType(precision, scale); - } + public abstract static class DecimalBase extends PrimitiveType { - private final int scale; - private final int precision; + protected final int scale; + protected final int precision; - private DecimalType(int precision, int scale) { + protected DecimalBase(int precision, int scale) { this.scale = scale; this.precision = precision; } @@ -320,12 +314,12 @@ private DecimalType(int precision, int scale) { * can be casted into `this` safely without losing any precision or range. */ public boolean isWiderThan(PrimitiveType other) { - if (other instanceof DecimalType) { - DecimalType dt = (DecimalType) other; + if (other instanceof DecimalBase) { + DecimalBase dt = (DecimalBase) other; return (precision - scale) >= (dt.precision - dt.scale) && scale > dt.scale; } if (other instanceof IntType) { - return isWiderThan(get(10, 0)); + return (precision - scale) >= 10 && scale > 0; } return false; } @@ -335,12 +329,12 @@ public boolean isWiderThan(PrimitiveType other) { * can be casted into `other` safely without losing any precision or range. */ public boolean isTighterThan(PrimitiveType other) { - if (other instanceof DecimalType) { - DecimalType dt = (DecimalType) other; + if (other instanceof DecimalBase) { + DecimalBase dt = (DecimalBase) other; return (precision - scale) <= (dt.precision - dt.scale) && scale <= dt.scale; } if (other instanceof IntType) { - return isTighterThan(get(10, 0)); + return (precision - scale) <= 10 && scale <= 0; } return false; } @@ -353,6 +347,51 @@ public int precision() { return precision; } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof DecimalBase)) { + return false; + } + + DecimalBase that = (DecimalBase) o; + if (scale != that.scale) { + return false; + } + return precision == that.precision; + } + + @Override + public int hashCode() { + return Objects.hash(this.getClass(), scale, precision); + } + } + + /** + * Decimal primitive type. + */ + public static class DecimalType extends DecimalTypeFixed { + public static DecimalType get(int precision, int scale) { + return new DecimalType(precision, scale); + } + + /** + * Return the minimum number of bytes needed to store a decimal with a give 'precision'. + * reference from Spark release 3.1 . + */ + private static int computeMinBytesForDecimalPrecision(int precision) { + int numBytes = 1; + while (Math.pow(2.0, 8 * numBytes - 1) < Math.pow(10.0, precision)) { + numBytes += 1; + } + return numBytes; + } + + private DecimalType(int precision, int scale) { + super(precision, scale, computeMinBytesForDecimalPrecision(precision)); + } + @Override public TypeID typeId() { return TypeID.DECIMAL; @@ -362,25 +401,73 @@ public TypeID typeId() { public String toString() { return String.format("decimal(%d, %d)", precision, scale); } + } + + public static class DecimalTypeBytes extends DecimalBase { + public static DecimalTypeBytes get(int precision, int scale) { + return new DecimalTypeBytes(precision, scale); + } + + private DecimalTypeBytes(int precision, int scale) { + super(precision, scale); + } + + @Override + public TypeID typeId() { + return TypeID.DECIMAL_BYTES; + } + + @Override + public String toString() { + return String.format("decimal_bytes(%d, %d)", precision, scale); + } @Override public boolean equals(Object o) { - if (this == o) { - return true; - } else if (!(o instanceof DecimalType)) { - return false; + if (super.equals(o)) { + return o instanceof DecimalTypeBytes; } + return false; + } + } - DecimalType that = (DecimalType) o; - if (scale != that.scale) { - return false; - } - return precision == that.precision; + public static class DecimalTypeFixed extends DecimalBase { + public static DecimalTypeFixed get(int precision, int scale, int size) { + return new DecimalTypeFixed(precision, scale, size); + } + + private final int size; + + public int getFixedSize() { + return size; + } + + private DecimalTypeFixed(int precision, int scale, int size) { + super(precision, scale); + this.size = size; + } + + @Override + public TypeID typeId() { + return TypeID.DECIMAL_FIXED; + } + + @Override + public String toString() { + return String.format("decimal_fixed(%d, %d)[%d]", precision, scale, size); } @Override public int hashCode() { - return Objects.hash(DecimalType.class, scale, precision); + return Objects.hash(DecimalTypeFixed.class, scale, precision, size); + } + + @Override + public boolean equals(Object o) { + if (super.equals(o)) { + return o instanceof DecimalTypeFixed && ((DecimalTypeFixed) o).size == size; + } + return false; } } @@ -405,6 +492,78 @@ public String toString() { } } + public static class TimeMillisType extends PrimitiveType { + private static final TimeMillisType INSTANCE = new TimeMillisType(); + + public static TimeMillisType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.TIME_MILLIS; + } + + @Override + public String toString() { + return "time-millis"; + } + } + + public static class TimestampMillisType extends PrimitiveType { + private static final TimestampMillisType INSTANCE = new TimestampMillisType(); + + public static TimestampMillisType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.TIMESTAMP_MILLIS; + } + + @Override + public String toString() { + return "timestamp-millis"; + } + } + + public static class LocalTimestampMillisType extends PrimitiveType { + private static final LocalTimestampMillisType INSTANCE = new LocalTimestampMillisType(); + + public static LocalTimestampMillisType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.LOCAL_TIMESTAMP_MILLIS; + } + + @Override + public String toString() { + return "local-timestamp-millis"; + } + } + + public static class LocalTimestampMicrosType extends PrimitiveType { + private static final LocalTimestampMicrosType INSTANCE = new LocalTimestampMicrosType(); + + public static LocalTimestampMicrosType get() { + return INSTANCE; + } + + @Override + public TypeID typeId() { + return TypeID.LOCAL_TIMESTAMP_MICROS; + } + + @Override + public String toString() { + return "local-timestamp-micros"; + } + } + /** A field within a record. */ public static class Field implements Serializable { // Experimental method to support defaultValue diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java index 924917d5f5157..acd229f3ced89 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java @@ -338,22 +338,30 @@ private static Type visitAvroPrimitiveToBuildInternalType(Schema primitive) { if (logical != null) { String name = logical.getName(); if (logical instanceof LogicalTypes.Decimal) { - return Types.DecimalType.get( - ((LogicalTypes.Decimal) logical).getPrecision(), - ((LogicalTypes.Decimal) logical).getScale()); - + if (primitive.getType() == Schema.Type.FIXED) { + return Types.DecimalTypeFixed.get(((LogicalTypes.Decimal) logical).getPrecision(), + ((LogicalTypes.Decimal) logical).getScale(), primitive.getFixedSize()); + } else if (primitive.getType() == Schema.Type.BYTES) { + return Types.DecimalTypeBytes.get( + ((LogicalTypes.Decimal) logical).getPrecision(), + ((LogicalTypes.Decimal) logical).getScale()); + } else { + throw new IllegalArgumentException("Unsupported primitive type for Decimal: " + primitive.getType().getName()); + } } else if (logical instanceof LogicalTypes.Date) { return Types.DateType.get(); - - } else if ( - logical instanceof LogicalTypes.TimeMillis - || logical instanceof LogicalTypes.TimeMicros) { + } else if (logical instanceof LogicalTypes.TimeMillis) { + return Types.TimeMillisType.get(); + } else if (logical instanceof LogicalTypes.TimeMicros) { return Types.TimeType.get(); - - } else if ( - logical instanceof LogicalTypes.TimestampMillis - || logical instanceof LogicalTypes.TimestampMicros) { + } else if (logical instanceof LogicalTypes.TimestampMillis) { + return Types.TimestampMillisType.get(); + } else if (logical instanceof LogicalTypes.TimestampMicros) { return Types.TimestampType.get(); + } else if (logical instanceof LogicalTypes.LocalTimestampMillis) { + return Types.LocalTimestampMillisType.get(); + } else if (logical instanceof LogicalTypes.LocalTimestampMicros) { + return Types.LocalTimestampMicrosType.get(); } else if (LogicalTypes.uuid().getName().equals(name)) { return Types.UUIDType.get(); } @@ -542,9 +550,21 @@ private static Schema visitInternalPrimitiveToBuildAvroPrimitiveType(Type.Primit case TIME: return LogicalTypes.timeMicros().addToSchema(Schema.create(Schema.Type.LONG)); + case TIME_MILLIS: + return LogicalTypes.timeMillis().addToSchema(Schema.create(Schema.Type.INT)); + case TIMESTAMP: return LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + case TIMESTAMP_MILLIS: + return LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)); + + case LOCAL_TIMESTAMP_MICROS: + return LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + + case LOCAL_TIMESTAMP_MILLIS: + return LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)); + case STRING: return Schema.create(Schema.Type.STRING); @@ -567,32 +587,27 @@ private static Schema visitInternalPrimitiveToBuildAvroPrimitiveType(Type.Primit return Schema.createFixed(name, null, null, fixed.getFixedSize()); } - case DECIMAL: { - Types.DecimalType decimal = (Types.DecimalType) primitive; + case DECIMAL: + case DECIMAL_FIXED: { + Types.DecimalTypeFixed decimal = (Types.DecimalTypeFixed) primitive; // NOTE: All schemas corresponding to Avro's type [[FIXED]] are generated // with the "fixed" name to stay compatible w/ [[SchemaConverters]] String name = recordName + AVRO_NAME_DELIMITER + "fixed"; Schema fixedSchema = Schema.createFixed(name, - null, null, computeMinBytesForPrecision(decimal.precision())); + null, null, decimal.getFixedSize()); return LogicalTypes.decimal(decimal.precision(), decimal.scale()) .addToSchema(fixedSchema); } + case DECIMAL_BYTES: { + Types.DecimalTypeBytes decimal = (Types.DecimalTypeBytes) primitive; + return LogicalTypes.decimal(decimal.precision(), decimal.scale()) + .addToSchema(Schema.create(Schema.Type.BYTES)); + } + default: throw new UnsupportedOperationException( "Unsupported type ID: " + primitive.typeId()); } } - - /** - * Return the minimum number of bytes needed to store a decimal with a give 'precision'. - * reference from Spark release 3.1 . - */ - private static int computeMinBytesForPrecision(int precision) { - int numBytes = 1; - while (Math.pow(2.0, 8 * numBytes - 1) < Math.pow(10.0, precision)) { - numBytes += 1; - } - return numBytes; - } } diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/Conversions.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/Conversions.java index 67adff88ac76d..2f672a974e52e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/Conversions.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/Conversions.java @@ -32,6 +32,8 @@ public class Conversions { Type.TypeID.BOOLEAN, Type.TypeID.FLOAT, Type.TypeID.DECIMAL, + Type.TypeID.DECIMAL_FIXED, + Type.TypeID.DECIMAL_BYTES, Type.TypeID.DOUBLE, Type.TypeID.UUID, Type.TypeID.DATE, diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SchemaChangeUtils.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SchemaChangeUtils.java index b2751cc43e87a..65f452d975f7b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SchemaChangeUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SchemaChangeUtils.java @@ -48,62 +48,83 @@ private SchemaChangeUtils() { * TODO: support more type update. * * @param src origin column type. - * @param dsr new column type. + * @param dst new column type. * @return whether to allow the column type to be updated. */ - public static boolean isTypeUpdateAllow(Type src, Type dsr) { - if (src.isNestedType() || dsr.isNestedType()) { + public static boolean isTypeUpdateAllow(Type src, Type dst) { + if (src.isNestedType() || dst.isNestedType()) { throw new IllegalArgumentException("only support update primitive type"); } - if (src.equals(dsr)) { + if (src.equals(dst)) { return true; } - return isTypeUpdateAllowInternal(src, dsr); + return isTypeUpdateAllowInternal(src, dst); } - public static boolean shouldPromoteType(Type src, Type dsr) { - if (src.equals(dsr) || src.isNestedType() || dsr.isNestedType()) { + public static boolean shouldPromoteType(Type src, Type dst) { + if (src.equals(dst) || src.isNestedType() || dst.isNestedType()) { return false; } - return isTypeUpdateAllowInternal(src, dsr); + return isTypeUpdateAllowInternal(src, dst); } - private static boolean isTypeUpdateAllowInternal(Type src, Type dsr) { + private static boolean isTypeUpdateAllowInternal(Type src, Type dst) { switch (src.typeId()) { case INT: - return dsr == Types.LongType.get() || dsr == Types.FloatType.get() - || dsr == Types.DoubleType.get() || dsr == Types.StringType.get() || dsr.typeId() == Type.TypeID.DECIMAL; + return dst == Types.LongType.get() || dst == Types.FloatType.get() + || dst == Types.DoubleType.get() || dst == Types.StringType.get() || dst.typeId() == Type.TypeID.DECIMAL || dst.typeId() == Type.TypeID.DECIMAL_FIXED; case LONG: - return dsr == Types.FloatType.get() || dsr == Types.DoubleType.get() || dsr == Types.StringType.get() || dsr.typeId() == Type.TypeID.DECIMAL; + return dst == Types.FloatType.get() || dst == Types.DoubleType.get() || dst == Types.StringType.get() || dst.typeId() == Type.TypeID.DECIMAL || dst.typeId() == Type.TypeID.DECIMAL_FIXED; case FLOAT: - return dsr == Types.DoubleType.get() || dsr == Types.StringType.get() || dsr.typeId() == Type.TypeID.DECIMAL; + return dst == Types.DoubleType.get() || dst == Types.StringType.get() || dst.typeId() == Type.TypeID.DECIMAL || dst.typeId() == Type.TypeID.DECIMAL_FIXED; case DOUBLE: - return dsr == Types.StringType.get() || dsr.typeId() == Type.TypeID.DECIMAL; + return dst == Types.StringType.get() || dst.typeId() == Type.TypeID.DECIMAL || dst.typeId() == Type.TypeID.DECIMAL_FIXED; case DATE: case BINARY: - return dsr == Types.StringType.get(); + return dst == Types.StringType.get(); + case DECIMAL_BYTES: + return isDecimalBytesUpdateAllowInternal(src, dst); case DECIMAL: - if (dsr.typeId() == Type.TypeID.DECIMAL) { - Types.DecimalType decimalSrc = (Types.DecimalType)src; - Types.DecimalType decimalDsr = (Types.DecimalType)dsr; - if (decimalDsr.isWiderThan(decimalSrc)) { - return true; - } - if (decimalDsr.precision() >= decimalSrc.precision() && decimalDsr.scale() == decimalSrc.scale()) { - return true; - } - } else if (dsr.typeId() == Type.TypeID.STRING) { - return true; - } - break; + case DECIMAL_FIXED: + return isDecimalFixedUpdateAllowInternal(src, dst); case STRING: - return dsr == Types.DateType.get() || dsr.typeId() == Type.TypeID.DECIMAL || dsr == Types.BinaryType.get(); + return dst == Types.DateType.get() || dst.typeId() == Type.TypeID.DECIMAL || dst.typeId() == Type.TypeID.DECIMAL_FIXED || dst == Types.BinaryType.get(); default: return false; } + } + + private static boolean isDecimalBytesUpdateAllowInternal(Type src, Type dst) { + if (dst.typeId() == Type.TypeID.DECIMAL_BYTES || dst.typeId() == Type.TypeID.DECIMAL_FIXED || dst.typeId() == Type.TypeID.DECIMAL) { + return isDecimalUpdateAllowInternalBase((Types.DecimalBase)src, (Types.DecimalBase)dst); + } + return dst.typeId() == Type.TypeID.STRING; + } + + private static boolean isDecimalUpdateAllowInternalBase(Types.DecimalBase src, Types.DecimalBase dst) { + if (dst.isWiderThan(src)) { + return true; + } + if (dst.precision() >= src.precision() && dst.scale() == src.scale()) { + return true; + } return false; } + private static boolean isDecimalFixedUpdateAllowInternal(Type src, Type dst) { + if (dst instanceof Types.DecimalBase) { + if (dst.typeId() == Type.TypeID.DECIMAL_FIXED || dst.typeId() == Type.TypeID.DECIMAL) { + Types.DecimalTypeFixed decimalSrc = (Types.DecimalTypeFixed)src; + Types.DecimalTypeFixed decimaldst = (Types.DecimalTypeFixed)dst; + if (decimalSrc.getFixedSize() > decimaldst.getFixedSize()) { + return false; + } + } + return isDecimalUpdateAllowInternalBase((Types.DecimalBase)src, (Types.DecimalBase)dst); + } + return dst.typeId() == Type.TypeID.STRING; + } + /** * Apply all the DDL add operations to internalSchema to produce a new internalSchema. * diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java index 35545f16cdf37..9dc86178bb32a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SerDeHelper.java @@ -73,6 +73,8 @@ private SerDeHelper() { private static final Pattern FIXED = Pattern.compile("fixed\\[(\\d+)\\]"); private static final Pattern DECIMAL = Pattern.compile("decimal\\((\\d+),\\s+(\\d+)\\)"); + private static final Pattern DECIMAL_BYTES = Pattern.compile("decimal_bytes\\((\\d+),\\s*(\\d+)\\)"); + private static final Pattern DECIMAL_FIXED = Pattern.compile("decimal_fixed\\((\\d+),\\s*(\\d+)\\)\\[(\\d+)\\]"); /** * Convert history internalSchemas to json. @@ -191,10 +193,25 @@ private static Type parseTypeFromJson(JsonNode jsonNode) { if (jsonNode.isTextual()) { String type = jsonNode.asText().toLowerCase(Locale.ROOT); // deal with fixed and decimal + Matcher decimalFixed = DECIMAL_FIXED.matcher(type); + if (decimalFixed.matches()) { + return Types.DecimalTypeFixed.get( + Integer.parseInt(decimalFixed.group(1)), + Integer.parseInt(decimalFixed.group(2)), + Integer.parseInt(decimalFixed.group(3))); + } Matcher fixed = FIXED.matcher(type); if (fixed.matches()) { return Types.FixedType.getFixed(Integer.parseInt(fixed.group(1))); } + + Matcher decimalBytes = DECIMAL_BYTES.matcher(type); + if (decimalBytes.matches()) { + return Types.DecimalTypeBytes.get( + Integer.parseInt(decimalBytes.group(1)), + Integer.parseInt(decimalBytes.group(2))); + } + Matcher decimal = DECIMAL.matcher(type); if (decimal.matches()) { return Types.DecimalType.get( @@ -217,8 +234,16 @@ private static Type parseTypeFromJson(JsonNode jsonNode) { return Types.DateType.get(); case TIME: return Types.TimeType.get(); + case TIME_MILLIS: + return Types.TimeMillisType.get(); case TIMESTAMP: return Types.TimestampType.get(); + case TIMESTAMP_MILLIS: + return Types.TimestampMillisType.get(); + case LOCAL_TIMESTAMP_MICROS: + return Types.LocalTimestampMicrosType.get(); + case LOCAL_TIMESTAMP_MILLIS: + return Types.LocalTimestampMillisType.get(); case STRING: return Types.StringType.get(); case UUID: diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/ColumnRangeMetadataProvider.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/ColumnRangeMetadataProvider.java index 25e0be1270afa..da3571d0d1e37 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/ColumnRangeMetadataProvider.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/ColumnRangeMetadataProvider.java @@ -18,7 +18,8 @@ package org.apache.hudi.io.storage; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; +import org.apache.hudi.metadata.HoodieIndexVersion; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; import java.util.Map; @@ -30,5 +31,5 @@ public interface ColumnRangeMetadataProvider { /** * Get the column statistics, key is column name, value is the statistic for the column. */ - Map> getColumnRangeMeta(String filePath); + Map> getColumnRangeMeta(String filePath, HoodieIndexVersion indexVersion); } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieIndexVersion.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieIndexVersion.java index 3dc8397576249..91b534d8d9f2e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieIndexVersion.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieIndexVersion.java @@ -85,27 +85,36 @@ public static HoodieIndexVersion getCurrentVersion(HoodieTableVersion tableVersi * @return the appropriate HoodieIndexVersion for the given parameters */ public static HoodieIndexVersion getCurrentVersion(HoodieTableVersion tableVersion, MetadataPartitionType partitionType) { - if (partitionType == MetadataPartitionType.RECORD_INDEX) { - return V1; - } else if (partitionType == MetadataPartitionType.COLUMN_STATS) { - return V1; - } else if (partitionType == MetadataPartitionType.BLOOM_FILTERS) { - return V1; - } else if (partitionType == MetadataPartitionType.EXPRESSION_INDEX) { - return V1; - } else if (partitionType == MetadataPartitionType.SECONDARY_INDEX) { - if (tableVersion.greaterThanOrEquals(HoodieTableVersion.NINE)) { + switch (partitionType) { + case RECORD_INDEX: + return V1; + + case COLUMN_STATS: + case PARTITION_STATS: + case EXPRESSION_INDEX: + // column stats, partition stats, expression index must be updated together + if (tableVersion.lesserThan(HoodieTableVersion.NINE)) { + return V1; + } return V2; - } - return V1; - } else if (partitionType == MetadataPartitionType.FILES) { - return V1; - } else if (partitionType == MetadataPartitionType.PARTITION_STATS) { - return V1; - } else if (partitionType == MetadataPartitionType.ALL_PARTITIONS) { - return V1; - } else { - throw new HoodieException("Unknown metadata partition type: " + partitionType); + + case BLOOM_FILTERS: + return V1; + + case SECONDARY_INDEX: + if (tableVersion.greaterThanOrEquals(HoodieTableVersion.NINE)) { + return V2; + } + return V1; + + case FILES: + return V1; + + case ALL_PARTITIONS: + return V1; + + default: + throw new HoodieException("Unknown metadata partition type: " + partitionType); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java index b45aaca877865..138656ca405e3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java @@ -28,7 +28,6 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieAvroRecord; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordGlobalLocation; @@ -42,6 +41,7 @@ import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.index.expression.HoodieExpressionIndex; import org.apache.hudi.io.storage.HoodieAvroHFileReaderImplBase; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.StoragePathInfo; @@ -67,7 +67,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.hudi.avro.HoodieAvroWrapperUtils.wrapValueIntoAvro; import static org.apache.hudi.common.util.StringUtils.EMPTY_STRING; import static org.apache.hudi.common.util.ValidationUtils.checkArgument; import static org.apache.hudi.common.util.ValidationUtils.checkState; @@ -148,6 +147,9 @@ public class HoodieMetadataPayload implements HoodieRecordPayload createColumnStatsRecord(S HoodieMetadataColumnStats.newBuilder() .setFileName(new StoragePath(columnRangeMetadata.getFilePath()).getName()) .setColumnName(columnRangeMetadata.getColumnName()) - .setMinValue(wrapValueIntoAvro(columnRangeMetadata.getMinValue())) - .setMaxValue(wrapValueIntoAvro(columnRangeMetadata.getMaxValue())) + .setMinValue(columnRangeMetadata.getMinValueWrapped()) + .setMaxValue(columnRangeMetadata.getMaxValueWrapped()) .setNullCount(columnRangeMetadata.getNullCount()) .setValueCount(columnRangeMetadata.getValueCount()) .setTotalSize(columnRangeMetadata.getTotalSize()) .setTotalUncompressedSize(columnRangeMetadata.getTotalUncompressedSize()) .setIsDeleted(isDeleted) + .setValueType(columnRangeMetadata.getValueMetadata().getValueTypeInfo()) .build(), recordType); @@ -603,14 +606,15 @@ public static Stream createPartitionStatsRecords(String partitionP HoodieMetadataColumnStats.newBuilder() .setFileName(columnRangeMetadata.getFilePath()) .setColumnName(columnRangeMetadata.getColumnName()) - .setMinValue(wrapValueIntoAvro(columnRangeMetadata.getMinValue())) - .setMaxValue(wrapValueIntoAvro(columnRangeMetadata.getMaxValue())) + .setMinValue(columnRangeMetadata.getMinValueWrapped()) + .setMaxValue(columnRangeMetadata.getMaxValueWrapped()) .setNullCount(columnRangeMetadata.getNullCount()) .setValueCount(columnRangeMetadata.getValueCount()) .setTotalSize(columnRangeMetadata.getTotalSize()) .setTotalUncompressedSize(columnRangeMetadata.getTotalUncompressedSize()) .setIsDeleted(isDeleted) .setIsTightBound(isTightBound) + .setValueType(columnRangeMetadata.getValueMetadata().getValueTypeInfo()) .build(), MetadataPartitionType.PARTITION_STATS.getRecordType()); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index 7e14f5b73fbdc..9f0adf8392021 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -58,7 +58,6 @@ import org.apache.hudi.common.function.SerializableBiFunction; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieDeltaWriteStat; import org.apache.hudi.common.model.HoodieFileFormat; @@ -111,6 +110,8 @@ import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieIOFactory; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; +import org.apache.hudi.stats.ValueMetadata; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.HoodieStorageUtils; import org.apache.hudi.storage.StorageConfiguration; @@ -162,8 +163,6 @@ import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema; import static org.apache.hudi.avro.HoodieAvroUtils.addMetadataFields; import static org.apache.hudi.avro.HoodieAvroUtils.projectSchema; -import static org.apache.hudi.avro.HoodieAvroWrapperUtils.unwrapAvroValueWrapper; -import static org.apache.hudi.avro.HoodieAvroWrapperUtils.wrapValueIntoAvro; import static org.apache.hudi.common.config.HoodieCommonConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES; import static org.apache.hudi.common.config.HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED; import static org.apache.hudi.common.config.HoodieCommonConfig.MAX_MEMORY_FOR_COMPACTION; @@ -193,6 +192,7 @@ import static org.apache.hudi.metadata.MetadataPartitionType.fromPartitionPath; import static org.apache.hudi.metadata.MetadataPartitionType.isNewExpressionIndexDefinitionRequired; import static org.apache.hudi.metadata.MetadataPartitionType.isNewSecondaryIndexDefinitionRequired; +import static org.apache.hudi.stats.ValueMetadata.getValueMetadata; /** * A utility to convert timeline information to metadata table records. @@ -245,6 +245,19 @@ public static boolean isFilesPartitionAvailable(HoodieTableMetaClient metaClient .contains(HoodieTableMetadataUtil.PARTITION_NAME_FILES); } + // Helper class to calculate column stats + private static class ColumnStats { + Object minValue; + Object maxValue; + long nullCount; + long valueCount; + final ValueMetadata valueMetadata; + + ColumnStats(ValueMetadata valueMetadata) { + this.valueMetadata = valueMetadata; + } + } + /** * Collects {@link HoodieColumnRangeMetadata} for the provided collection of records, pretending * as if provided records have been persisted w/in given {@code filePath} @@ -261,17 +274,9 @@ public static Map> collectColumnRa List> targetFields, String filePath, Schema recordSchema, - StorageConfiguration storageConfig) { - // Helper class to calculate column stats - class ColumnStats { - Object minValue; - Object maxValue; - long nullCount; - long valueCount; - } - + StorageConfiguration storageConfig, + HoodieIndexVersion indexVersion) { HashMap allColumnStats = new HashMap<>(); - final Properties properties = new Properties(); properties.setProperty(HoodieStorageConfig.WRITE_UTC_TIMEZONE.key(), storageConfig.getString(HoodieStorageConfig.WRITE_UTC_TIMEZONE.key(), HoodieStorageConfig.WRITE_UTC_TIMEZONE.defaultValue().toString())); @@ -283,27 +288,11 @@ class ColumnStats { targetFields.forEach(fieldNameFieldPair -> { String fieldName = fieldNameFieldPair.getKey(); Schema fieldSchema = resolveNullableSchema(fieldNameFieldPair.getValue().schema()); - ColumnStats colStats = allColumnStats.computeIfAbsent(fieldName, ignored -> new ColumnStats()); - Object fieldValue; - if (record.getRecordType() == HoodieRecordType.AVRO) { - fieldValue = HoodieAvroUtils.getRecordColumnValues(record, new String[]{fieldName}, recordSchema, false)[0]; - if (fieldValue != null && fieldSchema.getType() == Schema.Type.INT && fieldSchema.getLogicalType() != null && fieldSchema.getLogicalType() == LogicalTypes.date()) { - fieldValue = java.sql.Date.valueOf(fieldValue.toString()); - } - - } else if (record.getRecordType() == HoodieRecordType.SPARK) { - fieldValue = record.getColumnValues(recordSchema, new String[]{fieldName}, false)[0]; - if (fieldValue != null && fieldSchema.getType() == Schema.Type.INT && fieldSchema.getLogicalType() != null && fieldSchema.getLogicalType() == LogicalTypes.date()) { - fieldValue = java.sql.Date.valueOf(LocalDate.ofEpochDay((Integer) fieldValue).toString()); - } - } else if (record.getRecordType() == HoodieRecordType.FLINK) { - fieldValue = record.getColumnValueAsJava(recordSchema, fieldName, properties); - } else { - throw new HoodieException(String.format("Unknown record type: %s", record.getRecordType())); - } + ColumnStats colStats = allColumnStats.computeIfAbsent(fieldName, ignored -> new ColumnStats(getValueMetadata(fieldSchema, indexVersion))); + Object fieldValue = collectColumnRangeFieldValue(record, colStats.valueMetadata, fieldName, fieldSchema, recordSchema, properties); colStats.valueCount++; - if (fieldValue != null && isColumnTypeSupported(fieldSchema, Option.of(record.getRecordType()))) { + if (fieldValue != null && isColumnTypeSupported(fieldSchema, Option.of(record.getRecordType()), indexVersion)) { // Set the min value of the field if (colStats.minValue == null || ConvertingGenericData.INSTANCE.compare(fieldValue, colStats.minValue, fieldSchema) < 0) { @@ -319,28 +308,90 @@ class ColumnStats { }); }); - Stream> hoodieColumnRangeMetadataStream = - targetFields.stream().map(fieldNameFieldPair -> { - String fieldName = fieldNameFieldPair.getKey(); - Schema fieldSchema = fieldNameFieldPair.getValue().schema(); - ColumnStats colStats = allColumnStats.get(fieldName); - HoodieColumnRangeMetadata hcrm = HoodieColumnRangeMetadata.create( - filePath, - fieldName, - colStats == null ? null : coerceToComparable(fieldSchema, colStats.minValue), - colStats == null ? null : coerceToComparable(fieldSchema, colStats.maxValue), - colStats == null ? 0L : colStats.nullCount, - colStats == null ? 0L : colStats.valueCount, - // NOTE: Size and compressed size statistics are set to 0 to make sure we're not - // mixing up those provided by Parquet with the ones from other encodings, - // since those are not directly comparable - 0L, - 0L - ); - return hcrm; - }); - return hoodieColumnRangeMetadataStream.collect( - Collectors.toMap(HoodieColumnRangeMetadata::getColumnName, Function.identity())); + return targetFields.stream().map(fieldNameFieldPair -> { + String fieldName = fieldNameFieldPair.getKey(); + Schema fieldSchema = fieldNameFieldPair.getValue().schema(); + ColumnStats colStats = allColumnStats.get(fieldName); + return colStatsToColRangeMetadata(fieldName, fieldSchema, colStats, filePath, indexVersion); + }).collect(Collectors.toMap(HoodieColumnRangeMetadata::getColumnName, Function.identity())); + } + + private static Object collectColumnRangeFieldValue(HoodieRecord record, ValueMetadata valueMetadata, String fieldName, Schema fieldSchema, Schema recordSchema, Properties properties) { + if (valueMetadata.isV1()) { + return collectColumnRangeFieldValueV1(record, fieldName, fieldSchema, recordSchema, properties); + } else { + return collectColumnRangeFieldValueV2(record, valueMetadata, fieldName, recordSchema, properties); + } + } + + private static Object collectColumnRangeFieldValueV1(HoodieRecord record, String fieldName, Schema fieldSchema, Schema recordSchema, Properties properties) { + Object fieldValue; + if (record.getRecordType() == HoodieRecordType.AVRO) { + fieldValue = HoodieAvroUtils.getRecordColumnValues(record, new String[]{fieldName}, recordSchema, false)[0]; + if (fieldValue != null && fieldSchema.getType() == Schema.Type.INT && fieldSchema.getLogicalType() != null && fieldSchema.getLogicalType() == LogicalTypes.date()) { + fieldValue = java.sql.Date.valueOf(fieldValue.toString()); + } + + } else if (record.getRecordType() == HoodieRecordType.SPARK) { + fieldValue = record.getColumnValues(recordSchema, new String[]{fieldName}, false)[0]; + if (fieldValue != null && fieldSchema.getType() == Schema.Type.INT && fieldSchema.getLogicalType() != null && fieldSchema.getLogicalType() == LogicalTypes.date()) { + fieldValue = java.sql.Date.valueOf(LocalDate.ofEpochDay((Integer) fieldValue).toString()); + } + } else if (record.getRecordType() == HoodieRecordType.FLINK) { + fieldValue = record.getColumnValueAsJava(recordSchema, fieldName, properties); + } else { + throw new HoodieException(String.format("Unknown record type: %s", record.getRecordType())); + } + return fieldValue; + } + + private static Comparable collectColumnRangeFieldValueV2(HoodieRecord record, ValueMetadata valueMetadata, String fieldName, Schema recordSchema, Properties properties) { + return valueMetadata.standardizeJavaTypeAndPromote(record.getColumnValueAsJava(recordSchema, fieldName, properties)); + } + + private static HoodieColumnRangeMetadata colStatsToColRangeMetadata(String fieldName, Schema fieldSchema, ColumnStats colStats, + String filePath, HoodieIndexVersion indexVersion) { + if (colStats == null) { + return HoodieColumnRangeMetadata.createEmpty(filePath, fieldName, indexVersion); + } else if (colStats.valueMetadata.isV1()) { + return colStatsToColRangeMetadataV1(fieldName, fieldSchema, colStats, filePath); + } else { + return colStatsToColRangeMetadataV2(fieldName, colStats, filePath); + } + } + + private static HoodieColumnRangeMetadata colStatsToColRangeMetadataV1(String fieldName, Schema fieldSchema, ColumnStats colStats, String filePath) { + return HoodieColumnRangeMetadata.create( + filePath, + fieldName, + coerceToComparable(fieldSchema, colStats.minValue), + coerceToComparable(fieldSchema, colStats.maxValue), + colStats.nullCount, + colStats.valueCount, + // NOTE: Size and compressed size statistics are set to 0 to make sure we're not + // mixing up those provided by Parquet with the ones from other encodings, + // since those are not directly comparable + 0L, + 0L, + ValueMetadata.V1EmptyMetadata.get() + ); + } + + private static HoodieColumnRangeMetadata colStatsToColRangeMetadataV2(String fieldName, ColumnStats colStats, String filePath) { + return HoodieColumnRangeMetadata.create( + filePath, + fieldName, + (Comparable) colStats.minValue, + (Comparable) colStats.maxValue, + colStats.nullCount, + colStats.valueCount, + // NOTE: Size and compressed size statistics are set to 0 to make sure we're not + // mixing up those provided by Parquet with the ones from other encodings, + // since those are not directly comparable + 0L, + 0L, + colStats.valueMetadata + ); } public static Option getColumnStatsValueAsString(Object statsValue) { @@ -809,8 +860,10 @@ public static HoodieData convertMetadataToColumnStatsRecords(Hoodi return engineContext.emptyHoodieData(); } + HoodieIndexVersion indexVersion = existingIndexVersionOrDefault(PARTITION_NAME_COLUMN_STATS, dataMetaClient); + List columnsToIndex = new ArrayList<>(getColumnsToIndex(dataMetaClient.getTableConfig(), metadataConfig, - Lazy.lazily(() -> tryResolveSchemaForTable(dataMetaClient)), false, recordTypeOpt).keySet()); + Lazy.lazily(() -> tryResolveSchemaForTable(dataMetaClient)), false, recordTypeOpt, indexVersion).keySet()); if (columnsToIndex.isEmpty()) { // In case there are no columns to index, bail @@ -1557,8 +1610,9 @@ public static Map getColumnsToIndex(HoodieCommitMetadata commitM ? tableConfig.getTableCreateSchema() // the write schema does not set up correctly : writerSchema.map(schema -> tableConfig.populateMetaFields() ? addMetadataFields(schema) : schema); + HoodieIndexVersion indexVersion = existingIndexVersionOrDefault(PARTITION_NAME_COLUMN_STATS, dataMetaClient); return getColumnsToIndex(tableConfig, metadataConfig, - Lazy.eagerly(tableSchema), false, recordTypeOpt); + Lazy.eagerly(tableSchema), false, recordTypeOpt, indexVersion); } @VisibleForTesting @@ -1574,27 +1628,30 @@ public static Map getColumnsToIndex(HoodieCommitMetadata commitM @VisibleForTesting public static Map getColumnsToIndex(HoodieTableConfig tableConfig, - HoodieMetadataConfig metadataConfig, - Lazy> tableSchemaLazyOpt, - Option recordType) { - return getColumnsToIndex(tableConfig, metadataConfig, tableSchemaLazyOpt, false, recordType); + HoodieMetadataConfig metadataConfig, + Lazy> tableSchemaLazyOpt, + Option recordType, + HoodieIndexVersion indexVersion) { + return getColumnsToIndex(tableConfig, metadataConfig, tableSchemaLazyOpt, false, recordType, indexVersion); } @VisibleForTesting public static Map getColumnsToIndex(HoodieTableConfig tableConfig, - HoodieMetadataConfig metadataConfig, - Lazy> tableSchemaLazyOpt, - boolean isTableInitializing) { - return getColumnsToIndex(tableConfig, metadataConfig, tableSchemaLazyOpt, isTableInitializing, Option.empty()); + HoodieMetadataConfig metadataConfig, + Lazy> tableSchemaLazyOpt, + boolean isTableInitializing, + HoodieIndexVersion indexVersion) { + return getColumnsToIndex(tableConfig, metadataConfig, tableSchemaLazyOpt, isTableInitializing, Option.empty(), indexVersion); } @VisibleForTesting public static Map getColumnsToIndex(HoodieTableConfig tableConfig, - HoodieMetadataConfig metadataConfig, - Lazy> tableSchemaLazyOpt, - boolean isTableInitializing, - Option recordType) { - Map columnsToIndexWithoutRequiredMetas = getColumnsToIndexWithoutRequiredMetaFields(metadataConfig, tableSchemaLazyOpt, isTableInitializing, recordType); + HoodieMetadataConfig metadataConfig, + Lazy> tableSchemaLazyOpt, + boolean isTableInitializing, + Option recordType, + HoodieIndexVersion indexVersion) { + Map columnsToIndexWithoutRequiredMetas = getColumnsToIndexWithoutRequiredMetaFields(metadataConfig, tableSchemaLazyOpt, isTableInitializing, recordType, indexVersion); if (!tableConfig.populateMetaFields()) { return columnsToIndexWithoutRequiredMetas; } @@ -1618,9 +1675,10 @@ public static Map getColumnsToIndex(HoodieTableConfig tableConfi * @return list of columns that should be indexed */ private static Map getColumnsToIndexWithoutRequiredMetaFields(HoodieMetadataConfig metadataConfig, - Lazy> tableSchemaLazyOpt, - boolean isTableInitializing, - Option recordType) { + Lazy> tableSchemaLazyOpt, + boolean isTableInitializing, + Option recordType, + HoodieIndexVersion indexVersion) { List columnsToIndex = metadataConfig.getColumnsEnabledForColumnStatsIndex(); if (!columnsToIndex.isEmpty()) { // if explicitly overridden @@ -1635,14 +1693,14 @@ private static Map getColumnsToIndexWithoutRequiredMetaFields(Ho Map colsToIndexSchemaMap = new LinkedHashMap<>(); columnsToIndex.stream().filter(fieldName -> !META_COL_SET_TO_INDEX.contains(fieldName)) .map(colName -> Pair.of(colName, HoodieAvroUtils.getSchemaForField(tableSchema.get(), colName).getRight().schema())) - .filter(fieldNameSchemaPair -> isColumnTypeSupported(fieldNameSchemaPair.getValue(), recordType)) + .filter(fieldNameSchemaPair -> isColumnTypeSupported(fieldNameSchemaPair.getValue(), recordType, indexVersion)) .forEach(entry -> colsToIndexSchemaMap.put(entry.getKey(), entry.getValue())); return colsToIndexSchemaMap; } // if not overridden if (tableSchemaLazyOpt.get().isPresent()) { Map colsToIndexSchemaMap = new LinkedHashMap<>(); - tableSchemaLazyOpt.get().map(schema -> getFirstNSupportedFields(schema, metadataConfig.maxColumnsToIndexForColStats(), recordType)).orElse(Stream.empty()) + tableSchemaLazyOpt.get().map(schema -> getFirstNSupportedFields(schema, metadataConfig.maxColumnsToIndexForColStats(), recordType, indexVersion)).orElse(Stream.empty()) .forEach(entry -> colsToIndexSchemaMap.put(entry.getKey(), entry.getValue())); return colsToIndexSchemaMap; } else { @@ -1651,9 +1709,9 @@ private static Map getColumnsToIndexWithoutRequiredMetaFields(Ho } } - private static Stream> getFirstNSupportedFields(Schema tableSchema, int n, Option recordType) { + private static Stream> getFirstNSupportedFields(Schema tableSchema, int n, Option recordType, HoodieIndexVersion indexVersion) { return getFirstNFields(tableSchema.getFields().stream() - .filter(field -> isColumnTypeSupported(field.schema(), recordType)).map(field -> Pair.of(field.name(), field.schema())), n); + .filter(field -> isColumnTypeSupported(field.schema(), recordType, indexVersion)).map(field -> Pair.of(field.name(), field.schema())), n); } private static Stream> getFirstNFields(Stream> fieldSchemaPairStream, int n) { @@ -1688,15 +1746,16 @@ private static Stream getColumnStatsRecords(String partitionPath, boolean isDeleted, int maxBufferSize) { + HoodieIndexVersion indexVersion = existingIndexVersionOrDefault(PARTITION_NAME_COLUMN_STATS, datasetMetaClient); if (isDeleted) { List> columnRangeMetadataList = columnsToIndex.stream() - .map(entry -> HoodieColumnRangeMetadata.stub(fileName, entry)) + .map(entry -> HoodieColumnRangeMetadata.stub(fileName, entry, indexVersion)) .collect(Collectors.toList()); return HoodieMetadataPayload.createColumnStatsRecords(partitionPath, columnRangeMetadataList, true); } List> columnRangeMetadata = - readColumnRangeMetadataFrom(partitionPath, fileName, datasetMetaClient, columnsToIndex, maxBufferSize); + readColumnRangeMetadataFrom(partitionPath, fileName, datasetMetaClient, columnsToIndex, maxBufferSize, indexVersion); return HoodieMetadataPayload.createColumnStatsRecords(partitionPath, columnRangeMetadata, false); } @@ -1705,7 +1764,8 @@ private static List> readColumnRangeMetada String fileName, HoodieTableMetaClient datasetMetaClient, List columnsToIndex, - int maxBufferSize) { + int maxBufferSize, + HoodieIndexVersion indexVersion) { String partitionPathFileName = (partitionPath.equals(EMPTY_PARTITION_NAME) || partitionPath.equals(NON_PARTITIONED_NAME)) ? fileName : partitionPath + "/" + fileName; try { @@ -1713,7 +1773,7 @@ private static List> readColumnRangeMetada if (partitionPathFileName.endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { return HoodieIOFactory.getIOFactory(datasetMetaClient.getStorage()) .getFileFormatUtils(HoodieFileFormat.PARQUET) - .readColumnStatsFromMetadata(datasetMetaClient.getStorage(), fullFilePath, columnsToIndex); + .readColumnStatsFromMetadata(datasetMetaClient.getStorage(), fullFilePath, columnsToIndex, indexVersion); } else if (FSUtils.isLogFile(fileName)) { Option writerSchemaOpt = tryResolveSchemaForTable(datasetMetaClient); LOG.warn("Reading log file: {}, to build column range metadata.", partitionPathFileName); @@ -1764,8 +1824,10 @@ public static List> getLogFileColumnRangeM if (!recordIterator.hasNext()) { return Collections.emptyList(); } + // TODO: maybe get the actual mdt partition path + HoodieIndexVersion indexVersion = existingIndexVersionOrDefault(PARTITION_NAME_COLUMN_STATS, datasetMetaClient); Map> columnRangeMetadataMap = - collectColumnRangeMetadata(recordIterator, fieldsToIndex, getFileNameFromPath(filePath), writerSchemaOpt.get(), datasetMetaClient.getStorage().getConf()); + collectColumnRangeMetadata(recordIterator, fieldsToIndex, getFileNameFromPath(filePath), writerSchemaOpt.get(), datasetMetaClient.getStorage().getConf(), indexVersion); return new ArrayList<>(columnRangeMetadataMap.values()); } } @@ -1965,10 +2027,16 @@ private static Double castToDouble(Object val) { } } - public static boolean isColumnTypeSupported(Schema schema, Option recordType) { + public static boolean isColumnTypeSupported(Schema schema, Option recordType, HoodieIndexVersion indexVersion) { Schema schemaToCheck = resolveNullableSchema(schema); - // Check for precision and scale if the schema has a logical decimal type. - LogicalType logicalType = schemaToCheck.getLogicalType(); + if (indexVersion.lowerThan(HoodieIndexVersion.V2)) { + return isColumnTypeSupportedV1(schemaToCheck, recordType); + } + return isColumnTypeSupportedV2(schemaToCheck); + } + + private static boolean isColumnTypeSupportedV1(Schema schema, Option recordType) { + LogicalType logicalType = schema.getLogicalType(); if (logicalType != null && logicalType instanceof LogicalTypes.Decimal) { LogicalTypes.Decimal decimalType = (LogicalTypes.Decimal) logicalType; if (decimalType.getPrecision() + (DECIMAL_MAX_SCALE - decimalType.getScale()) > DECIMAL_MAX_PRECISION || decimalType.getScale() > DECIMAL_MAX_SCALE) { @@ -1978,13 +2046,19 @@ public static boolean isColumnTypeSupported(Schema schema, Option getInflightMetadataPartitions(HoodieTableConfig tableConfig) { @@ -2543,25 +2617,28 @@ public HoodieRecord next() { private static Stream collectAndProcessColumnMetadata( List>> fileColumnMetadata, String partitionPath, boolean isTightBound, - Map colsToIndexSchemaMap + Map colsToIndexSchemaMap, + HoodieIndexVersion partitionStatsIndexVersion ) { - return collectAndProcessColumnMetadata(partitionPath, isTightBound, Option.empty(), fileColumnMetadata.stream().flatMap(List::stream), colsToIndexSchemaMap); + return collectAndProcessColumnMetadata(partitionPath, isTightBound, Option.empty(), fileColumnMetadata.stream().flatMap(List::stream), colsToIndexSchemaMap, partitionStatsIndexVersion); } private static Stream collectAndProcessColumnMetadata(Iterable> fileColumnMetadataIterable, String partitionPath, boolean isTightBound, Option indexPartitionOpt, - Map colsToIndexSchemaMap + Map colsToIndexSchemaMap, + HoodieIndexVersion partitionStatsIndexVersion ) { List> fileColumnMetadata = new ArrayList<>(); fileColumnMetadataIterable.forEach(fileColumnMetadata::add); // Group by Column Name - return collectAndProcessColumnMetadata(partitionPath, isTightBound, indexPartitionOpt, fileColumnMetadata.stream(), colsToIndexSchemaMap); + return collectAndProcessColumnMetadata(partitionPath, isTightBound, indexPartitionOpt, fileColumnMetadata.stream(), colsToIndexSchemaMap, partitionStatsIndexVersion); } private static Stream collectAndProcessColumnMetadata(String partitionPath, boolean isTightBound, Option indexPartitionOpt, Stream> fileColumnMetadata, - Map colsToIndexSchemaMap + Map colsToIndexSchemaMap, + HoodieIndexVersion partitionStatsIndexVersion ) { // Group by Column Name Map>> columnMetadataMap = @@ -2569,7 +2646,7 @@ private static Stream collectAndProcessColumnMetadata(String parti // Aggregate Column Ranges Stream> partitionStatsRangeMetadata = columnMetadataMap.entrySet().stream() - .map(entry -> FileFormatUtils.getColumnRangeInPartition(partitionPath, entry.getValue(), colsToIndexSchemaMap)); + .map(entry -> FileFormatUtils.getColumnRangeInPartition(partitionPath, entry.getKey(), entry.getValue(), colsToIndexSchemaMap, partitionStatsIndexVersion)); // Create Partition Stats Records return HoodieMetadataPayload.createPartitionStatsRecords(partitionPath, partitionStatsRangeMetadata.collect(Collectors.toList()), false, isTightBound, indexPartitionOpt); @@ -2587,7 +2664,7 @@ public static HoodieData collectAndProcessExprIndexPartitionStatRe iterable.forEach(e -> { HoodieColumnRangeMetadata rangeMetadata = HoodieColumnRangeMetadata.create( partitionName, e.getColumnName(), e.getMinValue(), e.getMaxValue(), - e.getNullCount(), e.getValueCount(), e.getTotalSize(), e.getTotalUncompressedSize()); + e.getNullCount(), e.getValueCount(), e.getTotalSize(), e.getTotalUncompressedSize(), e.getValueMetadata()); finalMetadata[0] = HoodieColumnRangeMetadata.merge(finalMetadata[0], rangeMetadata); }); return HoodieMetadataPayload.createPartitionStatsRecords(partitionName, Collections.singletonList(finalMetadata[0]), false, isTightBound, indexPartitionOpt) @@ -2604,8 +2681,9 @@ public static HoodieData convertFilesToPartitionStatsRecords(Hoodi if (partitionInfoList.isEmpty()) { return engineContext.emptyHoodieData(); } + HoodieIndexVersion partitionStatsIndexVersion = existingIndexVersionOrDefault(PARTITION_NAME_PARTITION_STATS, dataTableMetaClient); final Map columnsToIndexSchemaMap = getColumnsToIndex(dataTableMetaClient.getTableConfig(), metadataConfig, lazyWriterSchemaOpt, - dataTableMetaClient.getActiveTimeline().getWriteTimeline().filterCompletedInstants().empty(), recordTypeOpt); + dataTableMetaClient.getActiveTimeline().getWriteTimeline().filterCompletedInstants().empty(), recordTypeOpt, partitionStatsIndexVersion); if (columnsToIndexSchemaMap.isEmpty()) { LOG.warn("No columns to index for partition stats index"); return engineContext.emptyHoodieData(); @@ -2628,10 +2706,10 @@ public static HoodieData convertFilesToPartitionStatsRecords(Hoodi // Step 1: Collect Column Metadata for Each File List>> fileColumnMetadata = partitionInfo.getValue().stream() .map(fileName -> getFileStatsRangeMetadata(partitionPath, fileName, dataTableMetaClient, new ArrayList<>(columnsToIndexSchemaMap.keySet()), false, - metadataConfig.getMaxReaderBufferSize())) + metadataConfig.getMaxReaderBufferSize(), partitionStatsIndexVersion)) .collect(Collectors.toList()); - return collectAndProcessColumnMetadata(fileColumnMetadata, partitionPath, true, columnsToIndexSchemaMap).iterator(); + return collectAndProcessColumnMetadata(fileColumnMetadata, partitionPath, true, columnsToIndexSchemaMap, partitionStatsIndexVersion).iterator(); }); } @@ -2648,18 +2726,20 @@ private static List> getFileStatsRangeMeta HoodieTableMetaClient datasetMetaClient, List columnsToIndex, boolean isDeleted, - int maxBufferSize) { + int maxBufferSize, + HoodieIndexVersion indexVersion) { if (isDeleted) { return columnsToIndex.stream() - .map(entry -> HoodieColumnRangeMetadata.stub(fileName, entry)) + .map(entry -> HoodieColumnRangeMetadata.stub(fileName, entry, indexVersion)) .collect(Collectors.toList()); } - return readColumnRangeMetadataFrom(partitionPath, fileName, datasetMetaClient, columnsToIndex, maxBufferSize); + return readColumnRangeMetadataFrom(partitionPath, fileName, datasetMetaClient, columnsToIndex, maxBufferSize, indexVersion); } private static HoodieData convertMetadataToPartitionStatsRecords(HoodiePairData>> columnRangeMetadataPartitionPair, HoodieTableMetaClient dataMetaClient, - Map colsToIndexSchemaMap + Map colsToIndexSchemaMap, + HoodieIndexVersion partitionStatsIndexVersion ) { try { return columnRangeMetadataPartitionPair @@ -2667,7 +2747,7 @@ private static HoodieData convertMetadataToPartitionStatsRecords(H .groupByKey() .map(pair -> { final String partitionName = pair.getLeft(); - return collectAndProcessColumnMetadata(pair.getRight(), partitionName, isShouldScanColStatsForTightBound(dataMetaClient), Option.empty(), colsToIndexSchemaMap); + return collectAndProcessColumnMetadata(pair.getRight(), partitionName, isShouldScanColStatsForTightBound(dataMetaClient), Option.empty(), colsToIndexSchemaMap, partitionStatsIndexVersion); }) .flatMap(recordStream -> recordStream.iterator()); } catch (Exception e) { @@ -2690,8 +2770,9 @@ public static HoodieData convertMetadataToPartitionStatRecords(Hoo if (tableSchema.isEmpty()) { return engineContext.emptyHoodieData(); } + HoodieIndexVersion partitionStatsIndexVersion = existingIndexVersionOrDefault(PARTITION_NAME_PARTITION_STATS, dataMetaClient); Lazy> writerSchemaOpt = Lazy.eagerly(tableSchema); - Map columnsToIndexSchemaMap = getColumnsToIndex(dataMetaClient.getTableConfig(), metadataConfig, writerSchemaOpt, false, recordTypeOpt); + Map columnsToIndexSchemaMap = getColumnsToIndex(dataMetaClient.getTableConfig(), metadataConfig, writerSchemaOpt, false, recordTypeOpt, partitionStatsIndexVersion); if (columnsToIndexSchemaMap.isEmpty()) { return engineContext.emptyHoodieData(); } @@ -2708,7 +2789,7 @@ public static HoodieData convertMetadataToPartitionStatRecords(Hoo Stream columnRangeMetadata = columnsToIndexSchemaMap.keySet().stream() .flatMap(column -> HoodieMetadataPayload.createPartitionStatsRecords( partition, - Collections.singletonList(HoodieColumnRangeMetadata.stub("", column)), + Collections.singletonList(HoodieColumnRangeMetadata.stub("", column, partitionStatsIndexVersion)), true, true, Option.empty())); return columnRangeMetadata.iterator(); }); @@ -2738,7 +2819,7 @@ public static HoodieData convertMetadataToPartitionStatRecords(Hoo final String partitionName = partitionedWriteStat.get(0).getPartitionPath(); // Step 1: Collect Column Metadata for Each File part of current commit metadata List> fileColumnMetadata = partitionedWriteStat.stream() - .flatMap(writeStat -> translateWriteStatToFileStats(writeStat, dataMetaClient, colsToIndex, tableSchema).stream()).collect(toList()); + .flatMap(writeStat -> translateWriteStatToFileStats(writeStat, dataMetaClient, colsToIndex, partitionStatsIndexVersion).stream()).collect(toList()); if (shouldScanColStatsForTightBound) { checkState(tableMetadata != null, "tableMetadata should not be null when scanning metadata table"); @@ -2771,7 +2852,7 @@ public static HoodieData convertMetadataToPartitionStatRecords(Hoo return Pair.of(partitionName, fileColumnMetadata); }); - return convertMetadataToPartitionStatsRecords(columnRangeMetadata, dataMetaClient, columnsToIndexSchemaMap); + return convertMetadataToPartitionStatsRecords(columnRangeMetadata, dataMetaClient, columnsToIndexSchemaMap, partitionStatsIndexVersion); } catch (Exception e) { throw new HoodieException("Failed to generate column stats records for metadata table", e); } @@ -2829,14 +2910,14 @@ public static List generateColumnStatsKeys(List> translateWriteStatToFileStats(HoodieWriteStat writeStat, HoodieTableMetaClient datasetMetaClient, List columnsToIndex, - Option writerSchemaOpt) { + HoodieIndexVersion indexVersion) { if (writeStat instanceof HoodieDeltaWriteStat && ((HoodieDeltaWriteStat) writeStat).getColumnStats().isPresent()) { Map> columnRangeMap = ((HoodieDeltaWriteStat) writeStat).getColumnStats().get(); return new ArrayList<>(columnRangeMap.values()); } String filePath = writeStat.getPath(); - return getFileStatsRangeMetadata(writeStat.getPartitionPath(), getFileNameFromPath(filePath), datasetMetaClient, columnsToIndex, false, -1); + return getFileStatsRangeMetadata(writeStat.getPartitionPath(), getFileNameFromPath(filePath), datasetMetaClient, columnsToIndex, false, -1, indexVersion); } public static String getPartitionStatsIndexKey(String partitionPath, String columnName) { @@ -2870,31 +2951,33 @@ public static HoodieMetadataColumnStats mergeColumnStatsRecords(HoodieMetadataCo return newColumnStats; } - Comparable minValue = - (Comparable) Stream.of( - (Comparable) unwrapAvroValueWrapper(prevColumnStats.getMinValue()), - (Comparable) unwrapAvroValueWrapper(newColumnStats.getMinValue())) + ValueMetadata prevValueMetadata = getValueMetadata(prevColumnStats.getValueType()); + ValueMetadata newValueMetadata = getValueMetadata(newColumnStats.getValueType()); + + Comparable minValue = newValueMetadata.standardizeJavaTypeAndPromote( + (Comparable) Stream.of((Comparable) prevValueMetadata.unwrapValue(prevColumnStats.getMinValue()), + (Comparable) newValueMetadata.unwrapValue(newColumnStats.getMinValue())) .filter(Objects::nonNull) .min(Comparator.naturalOrder()) - .orElse(null); + .orElse(null)); - Comparable maxValue = - (Comparable) Stream.of( - (Comparable) unwrapAvroValueWrapper(prevColumnStats.getMaxValue()), - (Comparable) unwrapAvroValueWrapper(newColumnStats.getMaxValue())) + Comparable maxValue = newValueMetadata.standardizeJavaTypeAndPromote( + (Comparable) Stream.of(prevValueMetadata.unwrapValue(prevColumnStats.getMaxValue()), + (Comparable) newValueMetadata.unwrapValue(newColumnStats.getMaxValue())) .filter(Objects::nonNull) .max(Comparator.naturalOrder()) - .orElse(null); + .orElse(null)); HoodieMetadataColumnStats.Builder columnStatsBuilder = HoodieMetadataColumnStats.newBuilder(HoodieMetadataPayload.METADATA_COLUMN_STATS_BUILDER_STUB.get()) .setFileName(newColumnStats.getFileName()) .setColumnName(newColumnStats.getColumnName()) - .setMinValue(wrapValueIntoAvro(minValue)) - .setMaxValue(wrapValueIntoAvro(maxValue)) + .setMinValue(newValueMetadata.wrapValue(minValue)) + .setMaxValue(newValueMetadata.wrapValue(maxValue)) .setValueCount(prevColumnStats.getValueCount() + newColumnStats.getValueCount()) .setNullCount(prevColumnStats.getNullCount() + newColumnStats.getNullCount()) .setTotalSize(prevColumnStats.getTotalSize() + newColumnStats.getTotalSize()) .setTotalUncompressedSize(prevColumnStats.getTotalUncompressedSize() + newColumnStats.getTotalUncompressedSize()) + .setValueType(newColumnStats.getValueType()) .setIsDeleted(newColumnStats.getIsDeleted()); if (newColumnStats.hasField(COLUMN_STATS_FIELD_IS_TIGHT_BOUND)) { columnStatsBuilder.setIsTightBound(newColumnStats.getIsTightBound()); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java index 42633a2150209..6aa9857b075ea 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java @@ -31,6 +31,7 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.index.expression.HoodieExpressionIndex; +import org.apache.hudi.stats.ValueMetadata; import org.apache.avro.generic.GenericRecord; @@ -44,8 +45,6 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.hudi.avro.HoodieAvroWrapperUtils.unwrapAvroValueWrapper; -import static org.apache.hudi.avro.HoodieAvroWrapperUtils.wrapValueIntoAvro; import static org.apache.hudi.common.util.TypeUtils.unsafeCast; import static org.apache.hudi.common.util.ValidationUtils.checkArgument; import static org.apache.hudi.common.util.ValidationUtils.checkState; @@ -305,14 +304,16 @@ private static void constructColumnStatsMetadataPayload(HoodieMetadataPayload pa checkArgument(record.getSchema().getField(SCHEMA_FIELD_ID_COLUMN_STATS) == null, String.format("Valid %s record expected for type: %s", SCHEMA_FIELD_ID_COLUMN_STATS, MetadataPartitionType.COLUMN_STATS.getRecordType())); } else { + ValueMetadata valueMetadata = ValueMetadata.getValueMetadata(columnStatsRecord); HoodieMetadataColumnStats.Builder columnStatsBuilder = HoodieMetadataColumnStats.newBuilder(METADATA_COLUMN_STATS_BUILDER_STUB.get()) .setFileName(columnStatsRecord.get(COLUMN_STATS_FIELD_FILE_NAME).toString()) .setColumnName(columnStatsRecord.get(COLUMN_STATS_FIELD_COLUMN_NAME).toString()) // AVRO-2377 1.9.2 Modified the type of org.apache.avro.Schema#FIELD_RESERVED to Collections.unmodifiableSet. // This causes Kryo to fail when deserializing a GenericRecord, See HUDI-5484. // We should avoid using GenericRecord and convert GenericRecord into a serializable type. - .setMinValue(wrapValueIntoAvro(unwrapAvroValueWrapper(columnStatsRecord.get(COLUMN_STATS_FIELD_MIN_VALUE)))) - .setMaxValue(wrapValueIntoAvro(unwrapAvroValueWrapper(columnStatsRecord.get(COLUMN_STATS_FIELD_MAX_VALUE)))) + .setMinValue(valueMetadata.wrapValue(valueMetadata.unwrapValue(columnStatsRecord.get(COLUMN_STATS_FIELD_MIN_VALUE)))) + .setMaxValue(valueMetadata.wrapValue(valueMetadata.unwrapValue(columnStatsRecord.get(COLUMN_STATS_FIELD_MAX_VALUE)))) + .setValueType(valueMetadata.getValueTypeInfo()) .setValueCount((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_VALUE_COUNT)) .setNullCount((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_NULL_COUNT)) .setTotalSize((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_TOTAL_SIZE)) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieColumnRangeMetadata.java b/hudi-common/src/main/java/org/apache/hudi/stats/HoodieColumnRangeMetadata.java similarity index 69% rename from hudi-common/src/main/java/org/apache/hudi/common/model/HoodieColumnRangeMetadata.java rename to hudi-common/src/main/java/org/apache/hudi/stats/HoodieColumnRangeMetadata.java index e5325d7424a1f..0ddc3184771d9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieColumnRangeMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/stats/HoodieColumnRangeMetadata.java @@ -7,26 +7,28 @@ * "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 + * 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. + * 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.common.model; +package org.apache.hudi.stats; import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.metadata.HoodieIndexVersion; import javax.annotation.Nullable; import java.io.Serializable; import java.util.Objects; -import static org.apache.hudi.avro.HoodieAvroWrapperUtils.unwrapAvroValueWrapper; +import static org.apache.hudi.stats.ValueMetadata.getEmptyValueMetadata; /** * Hoodie metadata for the column range of data stored in columnar format (like Parquet) @@ -47,6 +49,7 @@ public class HoodieColumnRangeMetadata implements Serializ private final long valueCount; private final long totalSize; private final long totalUncompressedSize; + private final ValueMetadata valueMetadata; private HoodieColumnRangeMetadata(String filePath, String columnName, @@ -55,7 +58,8 @@ private HoodieColumnRangeMetadata(String filePath, long nullCount, long valueCount, long totalSize, - long totalUncompressedSize) { + long totalUncompressedSize, + ValueMetadata valueMetadata) { this.filePath = filePath; this.columnName = columnName; this.minValue = minValue; @@ -64,6 +68,7 @@ private HoodieColumnRangeMetadata(String filePath, this.valueCount = valueCount; this.totalSize = totalSize; this.totalUncompressedSize = totalUncompressedSize; + this.valueMetadata = valueMetadata; } public String getFilePath() { @@ -79,11 +84,19 @@ public T getMinValue() { return this.minValue; } + public Object getMinValueWrapped() { + return getValueMetadata().wrapValue(getMinValue()); + } + @Nullable public T getMaxValue() { return this.maxValue; } + public Object getMaxValueWrapped() { + return getValueMetadata().wrapValue(getMaxValue()); + } + public long getNullCount() { return nullCount; } @@ -100,6 +113,10 @@ public long getTotalUncompressedSize() { return totalUncompressedSize; } + public ValueMetadata getValueMetadata() { + return valueMetadata; + } + @Override public boolean equals(final Object o) { if (this == o) { @@ -145,29 +162,40 @@ public static > HoodieColumnRangeMetadata create(Stri long nullCount, long valueCount, long totalSize, - long totalUncompressedSize) { - return new HoodieColumnRangeMetadata<>(filePath, columnName, minValue, maxValue, nullCount, valueCount, totalSize, totalUncompressedSize); + long totalUncompressedSize, + ValueMetadata valueMetadata) throws IllegalArgumentException { + valueMetadata.validate(minValue, maxValue); + return new HoodieColumnRangeMetadata<>(filePath, columnName, minValue, maxValue, nullCount, valueCount, totalSize, totalUncompressedSize, valueMetadata); } /** * Converts instance of {@link HoodieMetadataColumnStats} to {@link HoodieColumnRangeMetadata} */ public static HoodieColumnRangeMetadata fromColumnStats(HoodieMetadataColumnStats columnStats) { + ValueMetadata valueMetadata = ValueMetadata.getValueMetadata(columnStats.getValueType()); return HoodieColumnRangeMetadata.create( columnStats.getFileName(), columnStats.getColumnName(), - unwrapAvroValueWrapper(columnStats.getMinValue()), // misses for special handling. - unwrapAvroValueWrapper(columnStats.getMaxValue()), // misses for special handling. + valueMetadata.unwrapValue(columnStats.getMinValue()), + valueMetadata.unwrapValue(columnStats.getMaxValue()), columnStats.getNullCount(), columnStats.getValueCount(), columnStats.getTotalSize(), - columnStats.getTotalUncompressedSize()); + columnStats.getTotalUncompressedSize(), + valueMetadata); } @SuppressWarnings("rawtype") public static HoodieColumnRangeMetadata stub(String filePath, - String columnName) { - return new HoodieColumnRangeMetadata<>(filePath, columnName, null, null, -1, -1, -1, -1); + String columnName, + HoodieIndexVersion indexVersion) { + return new HoodieColumnRangeMetadata<>(filePath, columnName, null, null, -1, -1, -1, -1, getEmptyValueMetadata(indexVersion)); + } + + public static HoodieColumnRangeMetadata createEmpty(String filePath, + String columnName, + HoodieIndexVersion indexVersion) { + return new HoodieColumnRangeMetadata(filePath, columnName, null, null, 0L, 0L, 0L, 0L, getEmptyValueMetadata(indexVersion)); } /** @@ -180,6 +208,16 @@ public static > HoodieColumnRangeMetadata merge( return left == null ? right : left; } + if (left.getValueMetadata().getValueType() != right.getValueMetadata().getValueType()) { + throw new IllegalArgumentException("Value types should be the same for merging column ranges"); + } else if (left.getValueMetadata().getValueType() != ValueType.V1) { + if (left.minValue != null && right.minValue != null && left.minValue.getClass() != right.minValue.getClass()) { + throw new IllegalArgumentException("Value types should be the same for merging column ranges"); + } else if (left.maxValue != null && right.maxValue != null && left.maxValue.getClass() != right.maxValue.getClass()) { + throw new IllegalArgumentException("Value types should be the same for merging column ranges"); + } + } + ValidationUtils.checkArgument(left.getColumnName().equals(right.getColumnName()), "Column names should be the same for merging column ranges"); String filePath = left.getFilePath(); @@ -190,7 +228,8 @@ public static > HoodieColumnRangeMetadata merge( long valueCount = left.getValueCount() + right.getValueCount(); long totalSize = left.getTotalSize() + right.getTotalSize(); long totalUncompressedSize = left.getTotalUncompressedSize() + right.getTotalUncompressedSize(); - return create(filePath, columnName, min, max, nullCount, valueCount, totalSize, totalUncompressedSize); + + return new HoodieColumnRangeMetadata<>(filePath, columnName, min, max, nullCount, valueCount, totalSize, totalUncompressedSize, left.getValueMetadata()); } private static > T minVal(T val1, T val2) { diff --git a/hudi-common/src/main/java/org/apache/hudi/stats/ValueMetadata.java b/hudi-common/src/main/java/org/apache/hudi/stats/ValueMetadata.java new file mode 100644 index 0000000000000..5c1e087b326f8 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/stats/ValueMetadata.java @@ -0,0 +1,266 @@ +/* + * 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.stats; + +import org.apache.hudi.ParquetAdapter; +import org.apache.hudi.avro.model.HoodieValueTypeInfo; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.metadata.HoodieIndexVersion; + +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.Serializable; + +import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema; +import static org.apache.hudi.metadata.HoodieMetadataPayload.COLUMN_STATS_FIELD_VALUE_TYPE; +import static org.apache.hudi.metadata.HoodieMetadataPayload.COLUMN_STATS_FIELD_VALUE_TYPE_ADDITIONAL_INFO; +import static org.apache.hudi.metadata.HoodieMetadataPayload.COLUMN_STATS_FIELD_VALUE_TYPE_ORDINAL; + +/** + * Holder for VaueType and additional info + * Used for wrapping and unwrapping col stat values + * as well as for type promotion + */ +public class ValueMetadata implements Serializable { + + private static final ParquetAdapter PARQUET_ADAPTER = ParquetAdapter.getAdapter(); + + private final ValueType valueType; + + protected ValueMetadata(ValueType valueType) { + this.valueType = valueType; + } + + public ValueType getValueType() { + return valueType; + } + + public HoodieValueTypeInfo getValueTypeInfo() { + return HoodieValueTypeInfo.newBuilder() + .setTypeOrdinal(valueType.ordinal()) + .setAdditionalInfo(getAdditionalInfo()) + .build(); + } + + String getAdditionalInfo() { + return null; + } + + public Comparable standardizeJavaTypeAndPromote(Object val) { + return this.getValueType().standardizeJavaTypeAndPromote(val, this); + } + + public Object wrapValue(Comparable value) { + return this.getValueType().wrapValue(value, this); + } + + public Comparable unwrapValue(Object value) { + return this.getValueType().unwrapValue(value, this); + } + + public void validate(Object minVal, Object maxVal) { + if (getValueType() == ValueType.V1) { + return; + } + this.getValueType().validate(minVal); + this.getValueType().validate(maxVal); + } + + public boolean isV1() { + return this.getValueType() == ValueType.V1; + } + + public static class V1EmptyMetadata extends ValueMetadata { + private static final V1EmptyMetadata V1_EMPTY_METADATA = new V1EmptyMetadata(); + public static V1EmptyMetadata get() { + return V1_EMPTY_METADATA; + } + + private V1EmptyMetadata() { + super(ValueType.V1); + } + + @Override + public HoodieValueTypeInfo getValueTypeInfo() { + // V1 should never be persisted to the MDT. It is only for in memory + return null; + } + } + + public static final ValueMetadata NULL_METADATA = new ValueMetadata(ValueType.NULL); + + /** + * decimal is encoded as a string in the format "precision,scale" in the extra info field + */ + interface DecimalValueMetadata { + + int getPrecision(); + + int getScale(); + + /** + * Do not change how we encode decimal without + * handling upgrade/downgrade and backwards compatibility + */ + static String encodeData(DecimalValueMetadata decimalValueMetadata) { + return String.format("%d,%d", decimalValueMetadata.getPrecision(), decimalValueMetadata.getScale()); + } + + /** + * Do not change how we encode decimal without + * handling upgrade/downgrade and backwards compatibility + */ + static Pair decodeData(String data) { + String[] splits = data.split(","); + return Pair.of(Integer.parseInt(splits[0]), Integer.parseInt(splits[1])); + } + } + + static class DecimalMetadata extends ValueMetadata implements DecimalValueMetadata { + + static DecimalMetadata create(String additionalInfo) { + if (additionalInfo == null) { + throw new IllegalArgumentException("additionalInfo cannot be null"); + } + Pair data = DecimalValueMetadata.decodeData(additionalInfo); + return new DecimalMetadata(data.getLeft(), data.getRight()); + } + + static DecimalMetadata create(LogicalTypes.Decimal decimal) { + return new DecimalMetadata(decimal.getPrecision(), decimal.getScale()); + } + + static DecimalMetadata create(PrimitiveType primitiveType) { + return new DecimalMetadata(PARQUET_ADAPTER.getPrecision(primitiveType), PARQUET_ADAPTER.getScale(primitiveType)); + } + + static DecimalMetadata create(int precision, int scale) { + return new DecimalMetadata(precision, scale); + } + + private final int precision; + private final int scale; + + private DecimalMetadata(int precision, int scale) { + super(ValueType.DECIMAL); + this.precision = precision; + this.scale = scale; + } + + @Override + public int getPrecision() { + return precision; + } + + @Override + public int getScale() { + return scale; + } + + @Override + String getAdditionalInfo() { + return DecimalValueMetadata.encodeData(this); + } + } + + public static ValueMetadata getEmptyValueMetadata(HoodieIndexVersion indexVersion) { + if (indexVersion.lowerThan(HoodieIndexVersion.V2)) { + return V1EmptyMetadata.get(); + } + return NULL_METADATA; + } + + public static ValueMetadata getValueMetadata(HoodieValueTypeInfo valueTypeInfo) { + // valueTypeInfo will always be null when version is v1 + if (valueTypeInfo == null) { + return V1EmptyMetadata.get(); + } + + ValueType valueType = ValueType.fromOrdinal(valueTypeInfo.getTypeOrdinal()); + if (valueType == ValueType.V1) { + return V1EmptyMetadata.get(); + } else if (valueType == ValueType.DECIMAL) { + return DecimalMetadata.create(valueTypeInfo.getAdditionalInfo()); + } else { + return new ValueMetadata(valueType); + } + } + + public static ValueMetadata getValueMetadata(GenericRecord columnStatsRecord) { + if (columnStatsRecord == null) { + // NOTE: Only legitimate reason for {@code ColumnStatsMetadata} to not be present is when + // it's not been read from the storage (ie it's not been a part of projected schema). + // Otherwise, it has to be present or the record would be considered invalid + throw new IllegalStateException("ColumnStatsMetadata is null. Handling should happen in the caller."); + } + + GenericRecord valueTypeInfo = (GenericRecord) columnStatsRecord.get(COLUMN_STATS_FIELD_VALUE_TYPE); + if (valueTypeInfo == null) { + return V1EmptyMetadata.get(); + } + + ValueType valueType = ValueType.fromOrdinal((Integer) valueTypeInfo.get(COLUMN_STATS_FIELD_VALUE_TYPE_ORDINAL)); + if (valueType == ValueType.V1) { + throw new IllegalArgumentException("Unsupported value type: " + valueTypeInfo.get(COLUMN_STATS_FIELD_VALUE_TYPE_ORDINAL)); + } else if (valueType == ValueType.DECIMAL) { + return DecimalMetadata.create((String) valueTypeInfo.get(COLUMN_STATS_FIELD_VALUE_TYPE_ADDITIONAL_INFO)); + } else { + return new ValueMetadata(valueType); + } + } + + public static ValueMetadata getValueMetadata(Schema fieldSchema, HoodieIndexVersion indexVersion) { + if (indexVersion.lowerThan(HoodieIndexVersion.V2)) { + return V1EmptyMetadata.get(); + } + if (fieldSchema == null) { + throw new IllegalArgumentException("Field schema cannot be null"); + } + Schema valueSchema = resolveNullableSchema(fieldSchema); + ValueType valueType = ValueType.fromSchema(valueSchema); + if (valueType == ValueType.V1) { + throw new IllegalArgumentException("Unsupported logical type for: " + valueSchema.getLogicalType()); + } else if (valueType == ValueType.DECIMAL) { + return DecimalMetadata.create((LogicalTypes.Decimal) valueSchema.getLogicalType()); + } else { + return new ValueMetadata(valueType); + } + } + + public static ValueMetadata getValueMetadata(PrimitiveType primitiveType, HoodieIndexVersion indexVersion) { + if (indexVersion.lowerThan(HoodieIndexVersion.V2)) { + return V1EmptyMetadata.get(); + } + if (primitiveType == null) { + throw new IllegalArgumentException("Primitive type cannot be null"); + } + ValueType valueType = ValueType.fromParquetPrimitiveType(primitiveType); + if (valueType == ValueType.V1) { + throw new IllegalStateException("Returned ValueType should never be V1 here. Primitive type: " + primitiveType); + } else if (valueType == ValueType.DECIMAL) { + return DecimalMetadata.create(primitiveType); + } else { + return new ValueMetadata(valueType); + } + } +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/stats/ValueType.java b/hudi-common/src/main/java/org/apache/hudi/stats/ValueType.java new file mode 100644 index 0000000000000..9b6ec143f39ae --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/stats/ValueType.java @@ -0,0 +1,665 @@ +/* + * 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.stats; + +import org.apache.hudi.ParquetAdapter; +import org.apache.hudi.avro.AvroSchemaUtils; +import org.apache.hudi.avro.HoodieAvroWrapperUtils; +import org.apache.hudi.common.util.DateTimeUtils; + +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.util.Utf8; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.PrimitiveType; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.math.MathContext; +import java.math.RoundingMode; +import java.nio.ByteBuffer; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneOffset; +import java.util.Objects; +import java.util.UUID; +import java.util.function.BiFunction; +import java.util.function.Function; + +import static org.apache.hudi.avro.HoodieAvroUtils.convertBytesToBigDecimal; +import static org.apache.hudi.common.util.DateTimeUtils.instantToMicros; +import static org.apache.hudi.common.util.DateTimeUtils.instantToNanos; +import static org.apache.hudi.common.util.DateTimeUtils.microsToInstant; +import static org.apache.hudi.common.util.DateTimeUtils.nanosToInstant; +import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; + +public enum ValueType { + V1(HoodieAvroWrapperUtils.PrimitiveWrapperType.V1.getClazz(), HoodieAvroWrapperUtils.PrimitiveWrapperType.V1, + ValueType::passThrough, ValueType::passThrough, ValueType::passThrough), + NULL(HoodieAvroWrapperUtils.PrimitiveWrapperType.NULL.getClazz(), HoodieAvroWrapperUtils.PrimitiveWrapperType.NULL, + ValueType::passThrough, ValueType::passThrough, ValueType::passThrough), + BOOLEAN(HoodieAvroWrapperUtils.PrimitiveWrapperType.BOOLEAN, ValueType::castToBoolean), + INT(HoodieAvroWrapperUtils.PrimitiveWrapperType.INT, ValueType::castToInteger), + LONG(HoodieAvroWrapperUtils.PrimitiveWrapperType.LONG, ValueType::castToLong), + FLOAT(HoodieAvroWrapperUtils.PrimitiveWrapperType.FLOAT, ValueType::castToFloat), + DOUBLE(HoodieAvroWrapperUtils.PrimitiveWrapperType.DOUBLE, ValueType::castToDouble), + STRING(HoodieAvroWrapperUtils.PrimitiveWrapperType.STRING, ValueType::castToString), + BYTES(HoodieAvroWrapperUtils.PrimitiveWrapperType.BYTES, ValueType::castToBytes), + FIXED(HoodieAvroWrapperUtils.PrimitiveWrapperType.BYTES, ValueType::castToFixed), + DECIMAL(BigDecimal.class, HoodieAvroWrapperUtils.PrimitiveWrapperType.BYTES, + ValueType::castToDecimal, ValueType::toDecimal, ValueType::fromDecimal), + UUID(UUID.class, HoodieAvroWrapperUtils.PrimitiveWrapperType.STRING, + ValueType::castToUUID, ValueType::toUUID, ValueType::fromUUID), + DATE(LocalDate.class, HoodieAvroWrapperUtils.PrimitiveWrapperType.INT, + ValueType::castToDate, ValueType::toDate, ValueType::fromDate), + TIME_MILLIS(LocalTime.class, HoodieAvroWrapperUtils.PrimitiveWrapperType.INT, + ValueType::castToTimeMillis, ValueType::toTimeMillis, ValueType::fromTimeMillis), + TIME_MICROS(LocalTime.class, HoodieAvroWrapperUtils.PrimitiveWrapperType.LONG, + ValueType::castToTimeMicros, ValueType::toTimeMicros, ValueType::fromTimeMicros), + TIMESTAMP_MILLIS(Instant.class, HoodieAvroWrapperUtils.PrimitiveWrapperType.LONG, + ValueType::castToTimestampMillis, ValueType::toTimestampMillis, ValueType::fromTimestampMillis), + TIMESTAMP_MICROS(Instant.class, HoodieAvroWrapperUtils.PrimitiveWrapperType.LONG, + ValueType::castToTimestampMicros, ValueType::toTimestampMicros, ValueType::fromTimestampMicros), + TIMESTAMP_NANOS(Instant.class, HoodieAvroWrapperUtils.PrimitiveWrapperType.LONG, + ValueType::castToTimestampNanos, ValueType::toTimestampNanos, ValueType::fromTimestampNanos), + LOCAL_TIMESTAMP_MILLIS(LocalDateTime.class, HoodieAvroWrapperUtils.PrimitiveWrapperType.LONG, + ValueType::castToLocalTimestampMillis, ValueType::toLocalTimestampMillis, ValueType::fromLocalTimestampMillis), + LOCAL_TIMESTAMP_MICROS(LocalDateTime.class, HoodieAvroWrapperUtils.PrimitiveWrapperType.LONG, + ValueType::castToLocalTimestampMicros, ValueType::toLocalTimestampMicros, ValueType::fromLocalTimestampMicros), + LOCAL_TIMESTAMP_NANOS(LocalDateTime.class, HoodieAvroWrapperUtils.PrimitiveWrapperType.LONG, + ValueType::castToLocalTimestampNanos, ValueType::toLocalTimestampNanos, ValueType::fromLocalTimestampNanos); + + // java type to represent this data type while in memory + private final Class internalType; + + // primitive wrapper type to represent this data type when we write to the MDT + private final HoodieAvroWrapperUtils.PrimitiveWrapperType primitiveWrapperType; + + // converts to the internal type if it is something else, and will also type promote + // if the column has been promoted since it was written + private final BiFunction> standardize; + + // converts primitive value read from the mdt to the internalType java class + private final BiFunction, ValueMetadata, Comparable> toComposite; + + // converts value from internalType java class to primitive type for writing to MDT + private final BiFunction, ValueMetadata, Comparable> toPrimitive; + + ValueType(HoodieAvroWrapperUtils.PrimitiveWrapperType primitiveWrapperType, Function single) { + this(primitiveWrapperType.getClazz(), + primitiveWrapperType, + (val, meta) -> (Comparable) single.apply(val), + ValueType::passThrough, + ValueType::passThrough); + } + + ValueType(Class internalType, + HoodieAvroWrapperUtils.PrimitiveWrapperType primitiveWrapperType, + BiFunction> standardize, + BiFunction, ValueMetadata, Comparable> toComposite, + BiFunction, ValueMetadata, Comparable> toPrimitive) { + this.internalType = internalType; + this.primitiveWrapperType = primitiveWrapperType; + this.standardize = standardize; + this.toComposite = toComposite; + this.toPrimitive = toPrimitive; + } + + Comparable standardizeJavaTypeAndPromote(Object val, ValueMetadata meta) { + if (val == null) { + return null; + } + return standardize.apply(val, meta); + } + + private Comparable convertIntoPrimitive(Comparable val, ValueMetadata meta) { + if (val == null) { + return null; + } + return toPrimitive.apply(val, meta); + } + + private Comparable convertIntoComplex(Comparable val, ValueMetadata meta) { + if (val == null) { + return null; + } + return toComposite.apply(val, meta); + } + + void validate(Object val) { + if (val == null) { + return; + } + + if (!internalType.isInstance(val)) { + throw new IllegalArgumentException(String.format( + "should be %s, but got %s", + internalType.getSimpleName(), + val.getClass().getSimpleName() + )); + } + } + + public Object wrapValue(Comparable val, ValueMetadata meta) { + if (meta.getValueType() == V1) { + return primitiveWrapperType.wrap(val); + } + + if (val == null) { + return null; + } + if (!this.internalType.isInstance(val)) { + throw new IllegalArgumentException(String.format( + "should be %s, but got %s", + this.internalType.getSimpleName(), + val.getClass().getSimpleName() + )); + } + return primitiveWrapperType.wrap(convertIntoPrimitive(val, meta)); + } + + public Comparable unwrapValue(Object val, ValueMetadata meta) { + if (meta.getValueType() == V1) { + return primitiveWrapperType.unwrap(val); + } + + if (val == null) { + return null; + } + + if (!primitiveWrapperType.getWrapperClass().isInstance(val)) { + if (!(val instanceof GenericRecord)) { + throw new IllegalArgumentException(String.format( + "should be %s, but got %s", + primitiveWrapperType.getWrapperClass().getSimpleName(), + val.getClass().getSimpleName() + )); + } else if (((GenericRecord) val).getSchema().getField("value") != null) { + return standardizeJavaTypeAndPromote(HoodieAvroWrapperUtils.unwrapGenericRecord(val), meta); + } else { + throw new IllegalArgumentException(String.format( + "should be %s, but got %s", + primitiveWrapperType.getWrapperClass().getSimpleName(), + val.getClass().getSimpleName() + )); + } + } + return convertIntoComplex(primitiveWrapperType.unwrap(val), meta); + } + + private static ValueType[] myEnumValues; + + public static ValueType fromOrdinal(int i) { + if (ValueType.myEnumValues == null) { + ValueType.myEnumValues = ValueType.values(); + } + return ValueType.myEnumValues[i]; + } + + private static final ParquetAdapter PARQUET_ADAPTER = ParquetAdapter.getAdapter(); + + public static ValueType fromParquetPrimitiveType(PrimitiveType primitiveType) { + if (PARQUET_ADAPTER.hasAnnotation(primitiveType)) { + return PARQUET_ADAPTER.getValueTypeFromAnnotation(primitiveType); + } + switch (primitiveType.getPrimitiveTypeName()) { + case INT64: + return ValueType.LONG; + case INT32: + return ValueType.INT; + case BOOLEAN: + return ValueType.BOOLEAN; + case BINARY: + return ValueType.BYTES; + case FLOAT: + return ValueType.FLOAT; + case DOUBLE: + return ValueType.DOUBLE; + case FIXED_LEN_BYTE_ARRAY: + return ValueType.FIXED; + default: + throw new IllegalArgumentException("Unsupported primitive type: " + primitiveType.getPrimitiveTypeName()); + } + } + + public static ValueType fromSchema(Schema schema) { + switch (schema.getType()) { + case NULL: + if (schema.getLogicalType() == null) { + return ValueType.NULL; + } + throw new IllegalArgumentException("Unsupported logical type for Null: " + schema.getLogicalType()); + case BOOLEAN: + if (schema.getLogicalType() == null) { + return ValueType.BOOLEAN; + } + throw new IllegalArgumentException("Unsupported logical type for Boolean: " + schema.getLogicalType()); + case INT: + if (schema.getLogicalType() == null) { + return ValueType.INT; + } else if (schema.getLogicalType() instanceof LogicalTypes.Date) { + return ValueType.DATE; + } else if (schema.getLogicalType() instanceof LogicalTypes.TimeMillis) { + return ValueType.TIME_MILLIS; + } + throw new IllegalArgumentException("Unsupported logical type for Int: " + schema.getLogicalType()); + case LONG: + if (schema.getLogicalType() == null) { + return ValueType.LONG; + } else if (schema.getLogicalType() instanceof LogicalTypes.TimeMicros) { + return ValueType.TIME_MICROS; + } else if (schema.getLogicalType() instanceof LogicalTypes.TimestampMillis) { + return ValueType.TIMESTAMP_MILLIS; + } else if (schema.getLogicalType() instanceof LogicalTypes.TimestampMicros) { + return ValueType.TIMESTAMP_MICROS; + } else if (schema.getLogicalType() instanceof LogicalTypes.LocalTimestampMillis) { + return ValueType.LOCAL_TIMESTAMP_MILLIS; + } else if (schema.getLogicalType() instanceof LogicalTypes.LocalTimestampMicros) { + return ValueType.LOCAL_TIMESTAMP_MICROS; + } + throw new IllegalArgumentException("Unsupported logical type for Long: " + schema.getLogicalType()); + case FLOAT: + if (schema.getLogicalType() == null) { + return ValueType.FLOAT; + } + throw new IllegalArgumentException("Unsupported logical type for Float: " + schema.getLogicalType()); + case DOUBLE: + if (schema.getLogicalType() == null) { + return ValueType.DOUBLE; + } + throw new IllegalArgumentException("Unsupported logical type for Double: " + schema.getLogicalType()); + case BYTES: + if (schema.getLogicalType() == null) { + return ValueType.BYTES; + } else if (schema.getLogicalType() instanceof LogicalTypes.Decimal) { + return ValueType.DECIMAL; + } + throw new IllegalArgumentException("Unsupported logical type for Bytes: " + schema.getLogicalType()); + case STRING: + if (schema.getLogicalType() == null) { + return ValueType.STRING; + } else if (Objects.equals(schema.getLogicalType().getName(), LogicalTypes.uuid().getName())) { + return ValueType.UUID; + } + throw new IllegalArgumentException("Unsupported logical type for String: " + schema.getLogicalType()); + case FIXED: + if (schema.getLogicalType() == null) { + return ValueType.FIXED; + } else if (schema.getLogicalType() instanceof LogicalTypes.Decimal) { + return ValueType.DECIMAL; + } + throw new IllegalArgumentException("Unsupported logical type for Fixed: " + schema.getLogicalType()); + case UNION: + return fromSchema(AvroSchemaUtils.resolveNullableSchema(schema)); + default: + throw new IllegalArgumentException("Unsupported type: " + schema.getType()); + } + } + + // Casting to standardize types and also type promotion + private static Comparable passThrough(Object val, ValueMetadata meta) { + return (Comparable) val; + } + + private static Boolean castToBoolean(Object val) { + if (val instanceof Boolean) { + return (Boolean) val; + } else { + throw new UnsupportedOperationException("Unable to convert boolean: " + val.getClass()); + } + } + + private static Integer castToInteger(Object val) { + if (val == null) { + return null; + } + if (val instanceof Integer) { + return (Integer) val; + } else if (val instanceof Boolean) { + return ((Boolean) val) ? 1 : 0; + } else { + // best effort casting + return Integer.parseInt(val.toString()); + } + } + + private static Long castToLong(Object val) { + if (val == null) { + return null; + } + if (val instanceof Integer) { + return ((Integer) val).longValue(); + } else if (val instanceof Long) { + return ((Long) val); + } else if (val instanceof Boolean) { + return ((Boolean) val) ? 1L : 0L; + } else { + // best effort casting + return Long.parseLong(val.toString()); + } + } + + private static Float castToFloat(Object val) { + if (val == null) { + return null; + } + if (val instanceof Integer) { + return ((Integer) val).floatValue(); + } else if (val instanceof Long) { + return ((Long) val).floatValue(); + } else if (val instanceof Float) { + return (Float) val; + } else if (val instanceof Boolean) { + return (Boolean) val ? 1.0f : 0.0f; + } else { + // best effort casting + return Float.parseFloat(val.toString()); + } + } + + private static Double castToDouble(Object val) { + if (val == null) { + return null; + } + if (val instanceof Integer) { + return ((Integer) val).doubleValue(); + } else if (val instanceof Long) { + return ((Long) val).doubleValue(); + } else if (val instanceof Float) { + return Double.valueOf(val + ""); + } else if (val instanceof Double) { + return (Double) val; + } else if (val instanceof Boolean) { + return (Boolean) val ? 1.0d : 0.0d; + } else { + // best effort casting + return Double.parseDouble(val.toString()); + } + } + + public static String castToString(Object val) { + if (val instanceof String) { + return (String) val; + } else if (val instanceof Utf8 || val instanceof Boolean || val instanceof Integer || val instanceof Long || val instanceof Float || val instanceof Double) { + return val.toString(); + } else if (val instanceof Binary) { + return ((Binary) val).toStringUsingUTF8(); + } else { + throw new UnsupportedOperationException("Unable to convert string: " + val.getClass()); + } + } + + public static ByteBuffer castToBytes(Object val) { + if (val instanceof ByteBuffer) { + return (ByteBuffer) val; + } else if (val instanceof GenericData.Fixed) { + return ByteBuffer.wrap(((GenericData.Fixed) val).bytes()); + } else if (val instanceof byte[]) { + return ByteBuffer.wrap((byte[]) val); + } else if (val instanceof Binary) { + return ((Binary) val).toByteBuffer(); + } else if (val instanceof String) { + return ByteBuffer.wrap(getUTF8Bytes(val.toString())); + } else { + throw new UnsupportedOperationException("Unable to convert bytes: " + val.getClass()); + } + } + + public static ByteBuffer castToFixed(Object val) { + if (val instanceof ByteBuffer) { + return (ByteBuffer) val; + } else if (val instanceof GenericData.Fixed) { + return ByteBuffer.wrap(((GenericData.Fixed) val).bytes()); + } else if (val instanceof byte[]) { + return ByteBuffer.wrap((byte[]) val); + } else if (val instanceof Binary) { + return ((Binary) val).toByteBuffer(); + } else { + throw new UnsupportedOperationException("Unable to convert fixed: " + val.getClass()); + } + } + + public static BigDecimal castToDecimal(Object val, ValueMetadata meta) { + ValueMetadata.DecimalValueMetadata decimalMetadata = (ValueMetadata.DecimalValueMetadata) meta; + int precision = decimalMetadata.getPrecision(); + int scale = decimalMetadata.getScale(); + if (val instanceof BigDecimal) { + return (BigDecimal) val; + } else if (val instanceof GenericData.Fixed) { + return convertBytesToBigDecimal(((GenericData.Fixed) val).bytes(), precision, scale); + } else if (val instanceof ByteBuffer) { + return convertBytesToBigDecimal(((ByteBuffer) val).array(), precision, scale); + } else if (val instanceof byte[]) { + return convertBytesToBigDecimal((byte[]) val, precision, scale); + } else if (val instanceof Integer) { + return BigDecimal.valueOf((Integer) val, scale).round(new MathContext(precision, RoundingMode.HALF_UP)); + } else if (val instanceof Long) { + return BigDecimal.valueOf((Long) val, scale).round(new MathContext(precision, RoundingMode.HALF_UP)); + } else if (val instanceof Binary) { + return new BigDecimal(new BigInteger(((Binary) val).getBytesUnsafe()), scale, new MathContext(precision, RoundingMode.HALF_UP)); + } else { + throw new UnsupportedOperationException("Unable to convert decimal: " + val.getClass()); + } + } + + public static UUID castToUUID(Object val, ValueMetadata meta) { + if (val instanceof UUID) { + return (UUID) val; + } else if (val instanceof String) { + return java.util.UUID.fromString((String) val); + } else { + throw new UnsupportedOperationException("Unable to convert UUID: " + val.getClass()); + } + } + + public static LocalDate castToDate(Object val, ValueMetadata meta) { + if (val instanceof LocalDate) { + return (LocalDate) val; + } else if (val instanceof java.sql.Date) { + return ((java.sql.Date) val).toLocalDate(); + } else if (val instanceof Integer) { + return LocalDate.ofEpochDay((Integer) val); + } else { + throw new UnsupportedOperationException("Unable to convert date: " + val.getClass()); + } + } + + public static LocalTime castToTimeMillis(Object val, ValueMetadata meta) { + if (val instanceof LocalTime) { + return (LocalTime) val; + } else if (val instanceof Integer) { + return LocalTime.ofNanoOfDay((Integer) val * 1_000_000L); + } else { + throw new UnsupportedOperationException("Unable to convert time millis: " + val.getClass()); + } + } + + public static LocalTime castToTimeMicros(Object val, ValueMetadata meta) { + if (val instanceof LocalTime) { + return (LocalTime) val; + } else if (val instanceof Long) { + return LocalTime.ofNanoOfDay((Long) val * 1000); + } else { + throw new UnsupportedOperationException("Unable to convert time micros: " + val.getClass()); + } + } + + public static Instant castToTimestampMillis(Object val, ValueMetadata meta) { + if (val instanceof Instant) { + return (Instant) val; + } else if (val instanceof Timestamp) { + return ((Timestamp) val).toInstant(); + } else if (val instanceof Long) { + return Instant.ofEpochMilli((Long) val); + } else { + throw new UnsupportedOperationException("Unable to convert timestamp millis: " + val.getClass()); + } + } + + public static Instant castToTimestampMicros(Object val, ValueMetadata meta) { + if (val instanceof Instant) { + return (Instant) val; + } else if (val instanceof Timestamp) { + return ((Timestamp) val).toInstant(); + } else if (val instanceof Long) { + return DateTimeUtils.microsToInstant((Long) val); + } else { + throw new UnsupportedOperationException("Unable to convert timestamp micros: " + val.getClass()); + } + } + + public static Instant castToTimestampNanos(Object val, ValueMetadata meta) { + if (val instanceof Instant) { + return (Instant) val; + } else if (val instanceof Long) { + return nanosToInstant((Long) val); + } else { + throw new UnsupportedOperationException("Unable to convert timestamp nanos: " + val.getClass()); + } + } + + public static LocalDateTime castToLocalTimestampMillis(Object val, ValueMetadata meta) { + if (val instanceof LocalDateTime) { + return (LocalDateTime) val; + } else if (val instanceof Long) { + return LocalDateTime.ofInstant(Instant.ofEpochMilli((Long) val), ZoneOffset.UTC); + } else { + throw new UnsupportedOperationException("Unable to convert local timestamp millis: " + val.getClass()); + } + } + + public static LocalDateTime castToLocalTimestampMicros(Object val, ValueMetadata meta) { + if (val instanceof LocalDateTime) { + return (LocalDateTime) val; + } else if (val instanceof Long) { + return LocalDateTime.ofInstant(microsToInstant((Long) val), ZoneOffset.UTC); + } else { + throw new UnsupportedOperationException("Unable to convert local timestamp micros: " + val.getClass()); + } + } + + public static LocalDateTime castToLocalTimestampNanos(Object val, ValueMetadata meta) { + if (val instanceof LocalDateTime) { + return (LocalDateTime) val; + } else if (val instanceof Long) { + return LocalDateTime.ofInstant(nanosToInstant((Long) val), ZoneOffset.UTC); + } else { + throw new UnsupportedOperationException("Unable to convert local timestamp nanos: " + val.getClass()); + } + } + + // Conversion to and from primitive types and complex types + + public static BigDecimal toDecimal(Comparable val, ValueMetadata meta) { + ValueMetadata.DecimalMetadata decimalMeta = (ValueMetadata.DecimalMetadata) meta; + return convertBytesToBigDecimal(((ByteBuffer) val).array(), decimalMeta.getPrecision(), decimalMeta.getScale()); + } + + public static ByteBuffer fromDecimal(Comparable val, ValueMetadata meta) { + return ByteBuffer.wrap(((BigDecimal) val).unscaledValue().toByteArray()); + } + + public static UUID toUUID(Comparable val, ValueMetadata meta) { + return java.util.UUID.fromString((String) val); + } + + public static String fromUUID(Comparable val, ValueMetadata meta) { + return ((UUID) val).toString(); + } + + public static LocalDate toDate(Comparable val, ValueMetadata meta) { + return LocalDate.ofEpochDay((Integer) val); + } + + public static Integer fromDate(Comparable val, ValueMetadata meta) { + return ((Long) ((LocalDate) val).toEpochDay()).intValue(); + } + + public static LocalTime toTimeMillis(Comparable val, ValueMetadata meta) { + return LocalTime.ofNanoOfDay((Integer) val * 1_000_000L); + } + + public static Integer fromTimeMillis(Comparable val, ValueMetadata meta) { + return ((LocalTime) val).toSecondOfDay() * 1000 + (((LocalTime) val).getNano() / 1_000_000); + } + + public static LocalTime toTimeMicros(Comparable val, ValueMetadata meta) { + return LocalTime.ofNanoOfDay((Long) val * 1000); + } + + public static Long fromTimeMicros(Comparable val, ValueMetadata meta) { + return ((LocalTime) val).toSecondOfDay() * 1_000_000L + (((LocalTime) val).getNano() / 1_000); + } + + public static Instant toTimestampMillis(Comparable val, ValueMetadata meta) { + return Instant.ofEpochMilli((Long) val); + } + + public static Long fromTimestampMillis(Comparable val, ValueMetadata meta) { + return ((Instant) val).toEpochMilli(); + } + + public static Instant toTimestampMicros(Comparable val, ValueMetadata meta) { + return microsToInstant((Long) val); + } + + public static Long fromTimestampMicros(Comparable val, ValueMetadata meta) { + return instantToMicros((Instant) val); + } + + public static Instant toTimestampNanos(Comparable val, ValueMetadata meta) { + return nanosToInstant((Long) val); + } + + public static Long fromTimestampNanos(Comparable val, ValueMetadata meta) { + return instantToNanos((Instant) val); + } + + public static LocalDateTime toLocalTimestampMillis(Comparable val, ValueMetadata meta) { + return LocalDateTime.ofInstant(Instant.ofEpochMilli((Long) val), ZoneOffset.UTC); + } + + public static Long fromLocalTimestampMillis(Comparable val, ValueMetadata meta) { + return ((LocalDateTime) val).toInstant(ZoneOffset.UTC).toEpochMilli(); + } + + public static LocalDateTime toLocalTimestampMicros(Comparable val, ValueMetadata meta) { + return LocalDateTime.ofInstant(microsToInstant((Long) val), ZoneOffset.UTC); + } + + public static Long fromLocalTimestampMicros(Comparable val, ValueMetadata meta) { + return instantToMicros(((LocalDateTime) val).toInstant(ZoneOffset.UTC)); + } + + public static LocalDateTime toLocalTimestampNanos(Comparable val, ValueMetadata meta) { + return LocalDateTime.ofInstant(nanosToInstant((Long) val), ZoneOffset.UTC); + } + + public static Long fromLocalTimestampNanos(Comparable val, ValueMetadata meta) { + return instantToNanos(((LocalDateTime) val).toInstant(ZoneOffset.UTC)); + } +} + + diff --git a/hudi-common/src/main/java/org/apache/parquet/schema/LogicalTypeParquetAdapter.java b/hudi-common/src/main/java/org/apache/parquet/schema/LogicalTypeParquetAdapter.java new file mode 100644 index 0000000000000..14582c64ba33e --- /dev/null +++ b/hudi-common/src/main/java/org/apache/parquet/schema/LogicalTypeParquetAdapter.java @@ -0,0 +1,92 @@ +/* + * 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.parquet.schema; + +import org.apache.hudi.ParquetAdapter; +import org.apache.hudi.stats.ValueType; + +/** + * Uses LogicalTypeAnnotation to extract value type, precision, and scale + */ +public class LogicalTypeParquetAdapter implements ParquetAdapter { + @Override + public boolean hasAnnotation(PrimitiveType primitiveType) { + return primitiveType.getLogicalTypeAnnotation() != null; + } + + @Override + public ValueType getValueTypeFromAnnotation(PrimitiveType primitiveType) { + LogicalTypeAnnotation annotation = primitiveType.getLogicalTypeAnnotation(); + if (annotation instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation) { + return ValueType.STRING; + } else if (annotation instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { + return ValueType.DECIMAL; + } else if (annotation instanceof LogicalTypeAnnotation.DateLogicalTypeAnnotation) { + return ValueType.DATE; + } else if (annotation instanceof LogicalTypeAnnotation.TimeLogicalTypeAnnotation) { + // TODO: decide if we need to support adjusted to UTC + LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeAnnotation = (LogicalTypeAnnotation.TimeLogicalTypeAnnotation) annotation; + if (timeAnnotation.getUnit() == LogicalTypeAnnotation.TimeUnit.MILLIS) { + return ValueType.TIME_MILLIS; + } else if (timeAnnotation.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + return ValueType.TIME_MICROS; + } else { + throw new IllegalArgumentException("Unsupported time unit: " + timeAnnotation.getUnit()); + } + } else if (annotation instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { + LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampAnnotation = (LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) annotation; + if (timestampAnnotation.getUnit() == LogicalTypeAnnotation.TimeUnit.MILLIS) { + return timestampAnnotation.isAdjustedToUTC() ? ValueType.TIMESTAMP_MILLIS : ValueType.LOCAL_TIMESTAMP_MILLIS; + } else if (timestampAnnotation.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + return timestampAnnotation.isAdjustedToUTC() ? ValueType.TIMESTAMP_MICROS : ValueType.LOCAL_TIMESTAMP_MICROS; + } else if (timestampAnnotation.getUnit() == LogicalTypeAnnotation.TimeUnit.NANOS) { + return timestampAnnotation.isAdjustedToUTC() ? ValueType.TIMESTAMP_NANOS : ValueType.LOCAL_TIMESTAMP_NANOS; + } else { + throw new IllegalArgumentException("Unsupported timestamp unit: " + timestampAnnotation.getUnit()); + } + } else if (annotation instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { + return ValueType.UUID; + } + + throw new IllegalArgumentException("Unsupported logical type annotation: " + annotation); + } + + private static void validatePrimitiveType(PrimitiveType primitiveType) { + if (primitiveType.getLogicalTypeAnnotation() == null) { + throw new IllegalArgumentException("Unsupported primitive type: " + primitiveType.getPrimitiveTypeName()); + } + + if (!(primitiveType.getLogicalTypeAnnotation() instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation)) { + throw new IllegalArgumentException("Unsupported logical type annotation: " + primitiveType.getLogicalTypeAnnotation()); + } + } + + @Override + public int getPrecision(PrimitiveType primitiveType) { + validatePrimitiveType(primitiveType); + return ((LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) primitiveType.getLogicalTypeAnnotation()).getPrecision(); + } + + @Override + public int getScale(PrimitiveType primitiveType) { + validatePrimitiveType(primitiveType); + return ((LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) primitiveType.getLogicalTypeAnnotation()).getScale(); + } +} diff --git a/hudi-common/src/main/java/org/apache/parquet/schema/OriginalTypeParquetAdapter.java b/hudi-common/src/main/java/org/apache/parquet/schema/OriginalTypeParquetAdapter.java new file mode 100644 index 0000000000000..7ecb9cd507081 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/parquet/schema/OriginalTypeParquetAdapter.java @@ -0,0 +1,83 @@ +/* + * 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.parquet.schema; + +import org.apache.hudi.ParquetAdapter; +import org.apache.hudi.stats.ValueType; + +/** + * Uses OriginalType to extract value type, precision, and scale + */ +public class OriginalTypeParquetAdapter implements ParquetAdapter { + @Override + public boolean hasAnnotation(PrimitiveType primitiveType) { + return primitiveType.getOriginalType() != null; + } + + @Override + public ValueType getValueTypeFromAnnotation(PrimitiveType primitiveType) { + switch (primitiveType.getOriginalType()) { + case UTF8: + return ValueType.STRING; + case DECIMAL: + return ValueType.DECIMAL; + case DATE: + return ValueType.DATE; + case TIME_MILLIS: + return ValueType.TIME_MILLIS; + case TIME_MICROS: + return ValueType.TIME_MICROS; + case TIMESTAMP_MILLIS: + if (primitiveType.toString().contains("(TIMESTAMP(MILLIS,false))")) { + return ValueType.LOCAL_TIMESTAMP_MILLIS; + } + return ValueType.TIMESTAMP_MILLIS; + case TIMESTAMP_MICROS: + if (primitiveType.toString().contains("(TIMESTAMP(MICROS,false))")) { + return ValueType.LOCAL_TIMESTAMP_MICROS; + } + return ValueType.TIMESTAMP_MICROS; + default: + throw new IllegalArgumentException("Unsupported original type: " + primitiveType.getOriginalType()); + } + } + + private static void validatePrimitiveType(PrimitiveType primitiveType) { + if (primitiveType.getOriginalType() == null) { + throw new IllegalArgumentException("Unsupported primitive type: " + primitiveType.getPrimitiveTypeName()); + } + + if (primitiveType.getOriginalType() != OriginalType.DECIMAL) { + throw new IllegalArgumentException("Unsupported original type: " + primitiveType.getOriginalType()); + } + } + + @Override + public int getPrecision(PrimitiveType primitiveType) { + validatePrimitiveType(primitiveType); + return primitiveType.getDecimalMetadata().getPrecision(); + } + + @Override + public int getScale(PrimitiveType primitiveType) { + validatePrimitiveType(primitiveType); + return primitiveType.getDecimalMetadata().getScale(); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java index 17bbd73ddc39e..6206392c30729 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java @@ -23,6 +23,8 @@ import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.Functions; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; +import org.apache.hudi.stats.ValueMetadata; import org.apache.hudi.storage.StoragePath; import org.apache.avro.util.Utf8; @@ -141,7 +143,7 @@ public void testRecordsStats() { minMaxValues.forEach(entry -> { String colName = targetColNamePrefix + "_" + (finalCounter1.getAndIncrement()); columnRangeMetadataMap.put(colName, HoodieColumnRangeMetadata.create(fileName, colName, - entry.getKey(), entry.getValue(), 5, 1000, 123456, 123456)); + entry.getKey(), entry.getValue(), 5, 1000, 123456, 123456, ValueMetadata.V1EmptyMetadata.get())); }); Map> clonedInput = new HashMap<>(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java index 39c8ba797ba89..ce772e40e5765 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java @@ -65,8 +65,10 @@ import java.math.RoundingMode; import java.nio.ByteBuffer; import java.time.Instant; +import java.time.LocalDate; import java.time.LocalDateTime; import java.time.ZoneOffset; +import java.time.ZonedDateTime; import java.util.ArrayList; import java.util.Arrays; import java.util.Base64; @@ -80,6 +82,7 @@ import java.util.Random; import java.util.Set; import java.util.UUID; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; import java.util.function.Supplier; @@ -151,6 +154,31 @@ public class HoodieTestDataGenerator implements AutoCloseable { + "{\"name\":\"current_ts\",\"type\": {\"type\": \"long\"}}," + "{\"name\":\"height\",\"type\":{\"type\":\"fixed\",\"name\":\"abc\",\"size\":5,\"logicalType\":\"decimal\",\"precision\":10,\"scale\":6}},"; + public static final String EXTENDED_LOGICAL_TYPES_SCHEMA_V6 = "{\"name\":\"ts_millis\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}}," + + "{\"name\":\"ts_micros\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-micros\"}}," + + "{\"name\":\"local_ts_millis\",\"type\":{\"type\":\"long\",\"logicalType\":\"local-timestamp-millis\"}}," + + "{\"name\":\"local_ts_micros\",\"type\":{\"type\":\"long\",\"logicalType\":\"local-timestamp-micros\"}}," + + "{\"name\":\"event_date\",\"type\":{\"type\":\"int\",\"logicalType\":\"date\"}}," + + "{\"name\":\"dec_fixed_small\",\"type\":{\"type\":\"fixed\",\"name\":\"decFixedSmall\",\"size\":3,\"logicalType\":\"decimal\",\"precision\":5,\"scale\":2}}," + + "{\"name\":\"dec_fixed_large\",\"type\":{\"type\":\"fixed\",\"name\":\"decFixedLarge\",\"size\":8,\"logicalType\":\"decimal\",\"precision\":18,\"scale\":9}},"; + + public static final String EXTENDED_LOGICAL_TYPES_SCHEMA = "{\"name\":\"ts_millis\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}}," + + "{\"name\":\"ts_micros\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-micros\"}}," + + "{\"name\":\"local_ts_millis\",\"type\":{\"type\":\"long\",\"logicalType\":\"local-timestamp-millis\"}}," + + "{\"name\":\"local_ts_micros\",\"type\":{\"type\":\"long\",\"logicalType\":\"local-timestamp-micros\"}}," + + "{\"name\":\"event_date\",\"type\":{\"type\":\"int\",\"logicalType\":\"date\"}}," + + "{\"name\":\"dec_plain_large\",\"type\":{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":20,\"scale\":10}}," + + "{\"name\":\"dec_fixed_small\",\"type\":{\"type\":\"fixed\",\"name\":\"decFixedSmall\",\"size\":3,\"logicalType\":\"decimal\",\"precision\":5,\"scale\":2}}," + + "{\"name\":\"dec_fixed_large\",\"type\":{\"type\":\"fixed\",\"name\":\"decFixedLarge\",\"size\":8,\"logicalType\":\"decimal\",\"precision\":18,\"scale\":9}},"; + + // LTS = Local Timestamp + public static final String EXTENDED_LOGICAL_TYPES_SCHEMA_NO_LTS = "{\"name\":\"ts_millis\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-millis\"}}," + + "{\"name\":\"ts_micros\",\"type\":{\"type\":\"long\",\"logicalType\":\"timestamp-micros\"}}," + + "{\"name\":\"event_date\",\"type\":{\"type\":\"int\",\"logicalType\":\"date\"}}," + + "{\"name\":\"dec_plain_large\",\"type\":{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":20,\"scale\":10}}," + + "{\"name\":\"dec_fixed_small\",\"type\":{\"type\":\"fixed\",\"name\":\"decFixedSmall\",\"size\":3,\"logicalType\":\"decimal\",\"precision\":5,\"scale\":2}}," + + "{\"name\":\"dec_fixed_large\",\"type\":{\"type\":\"fixed\",\"name\":\"decFixedLarge\",\"size\":8,\"logicalType\":\"decimal\",\"precision\":18,\"scale\":9}},"; + public static final String EXTRA_COL_SCHEMA1 = "{\"name\": \"extra_column1\", \"type\": [\"null\", \"string\"], \"default\": null },"; public static final String EXTRA_COL_SCHEMA2 = "{\"name\": \"extra_column2\", \"type\": [\"null\", \"string\"], \"default\": null},"; public static final String EXTRA_COL_SCHEMA_FOR_AWS_DMS_PAYLOAD = "{\"name\": \"Op\", \"type\": [\"null\", \"string\"], \"default\": null},"; @@ -166,6 +194,14 @@ public class HoodieTestDataGenerator implements AutoCloseable { TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + EXTRA_COL_SCHEMA2 + TRIP_SCHEMA_SUFFIX; public static final String TRIP_FLATTENED_SCHEMA = TRIP_SCHEMA_PREFIX + FARE_FLATTENED_SCHEMA + TRIP_SCHEMA_SUFFIX; + public static final String TRIP_LOGICAL_TYPES_SCHEMA_V6 = + TRIP_SCHEMA_PREFIX + EXTENDED_LOGICAL_TYPES_SCHEMA_V6 + TRIP_SCHEMA_SUFFIX; + public static final String TRIP_LOGICAL_TYPES_SCHEMA = + TRIP_SCHEMA_PREFIX + EXTENDED_LOGICAL_TYPES_SCHEMA + TRIP_SCHEMA_SUFFIX; + // LTS = Local Timestamp + public static final String TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS = + TRIP_SCHEMA_PREFIX + EXTENDED_LOGICAL_TYPES_SCHEMA_NO_LTS + TRIP_SCHEMA_SUFFIX; + public static final String TRIP_NESTED_EXAMPLE_SCHEMA = TRIP_SCHEMA_PREFIX + FARE_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX; @@ -194,6 +230,9 @@ public class HoodieTestDataGenerator implements AutoCloseable { HoodieAvroUtils.addMetadataFields(AVRO_SCHEMA); public static final Schema AVRO_SHORT_TRIP_SCHEMA = new Schema.Parser().parse(SHORT_TRIP_SCHEMA); public static final Schema AVRO_TRIP_ENCODED_DECIMAL_SCHEMA = new Schema.Parser().parse(TRIP_ENCODED_DECIMAL_SCHEMA); + public static final Schema AVRO_TRIP_LOGICAL_TYPES_SCHEMA = new Schema.Parser().parse(TRIP_LOGICAL_TYPES_SCHEMA); + public static final Schema AVRO_TRIP_LOGICAL_TYPES_SCHEMA_V6 = new Schema.Parser().parse(TRIP_LOGICAL_TYPES_SCHEMA_V6); + public static final Schema AVRO_TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS = new Schema.Parser().parse(TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS); public static final Schema AVRO_TRIP_SCHEMA = new Schema.Parser().parse(TRIP_SCHEMA); public static final Schema FLATTENED_AVRO_SCHEMA = new Schema.Parser().parse(TRIP_FLATTENED_SCHEMA); @@ -316,20 +355,42 @@ public int getEstimatedFileSizeInBytes(int numOfRecords) { } public IndexedRecord generateRandomValueAsPerSchema(String schemaStr, HoodieKey key, String commitTime, boolean isFlattened, long timestamp) throws IOException { - if (TRIP_FLATTENED_SCHEMA.equals(schemaStr)) { - return generateRandomValue(key, commitTime, true, timestamp); - } else if (TRIP_EXAMPLE_SCHEMA.equals(schemaStr)) { - return generateRandomValue(key, commitTime, isFlattened, timestamp); - } else if (TRIP_ENCODED_DECIMAL_SCHEMA.equals(schemaStr)) { - return generatePayloadForTripEncodedDecimalSchema(key, commitTime, timestamp); - } else if (TRIP_SCHEMA.equals(schemaStr)) { - return generatePayloadForTripSchema(key, commitTime, timestamp); - } else if (SHORT_TRIP_SCHEMA.equals(schemaStr)) { - return generatePayloadForShortTripSchema(key, commitTime, timestamp); - } else if (TRIP_NESTED_EXAMPLE_SCHEMA.equals(schemaStr)) { - return generateNestedExampleRandomValue(key, commitTime, timestamp); - } else if (TRIP_EXAMPLE_SCHEMA_WITH_PAYLOAD_SPECIFIC_COLS.equals(schemaStr)) { - return generateRandomValueWithColumnRequired(key, commitTime); + return generateRandomValueAsPerSchema(schemaStr, key, commitTime, isFlattened, false, timestamp); + } + + public IndexedRecord generateRandomValueAsPerSchema(String schemaStr, HoodieKey key, String commitTime, boolean isFlattened, boolean isDelete, long timestamp) throws IOException { + if (!isDelete) { + if (TRIP_FLATTENED_SCHEMA.equals(schemaStr)) { + return generateRandomValue(key, commitTime, true, timestamp); + } else if (TRIP_EXAMPLE_SCHEMA.equals(schemaStr)) { + return generateRandomValue(key, commitTime, isFlattened, timestamp); + } else if (TRIP_ENCODED_DECIMAL_SCHEMA.equals(schemaStr)) { + return generatePayloadForTripEncodedDecimalSchema(key, commitTime, timestamp); + } else if (TRIP_SCHEMA.equals(schemaStr)) { + return generatePayloadForTripSchema(key, commitTime, timestamp); + } else if (SHORT_TRIP_SCHEMA.equals(schemaStr)) { + return generatePayloadForShortTripSchema(key, commitTime, timestamp); + } else if (TRIP_NESTED_EXAMPLE_SCHEMA.equals(schemaStr)) { + return generateNestedExampleRandomValue(key, commitTime, timestamp); + } else if (TRIP_EXAMPLE_SCHEMA_WITH_PAYLOAD_SPECIFIC_COLS.equals(schemaStr)) { + return generateRandomValueWithColumnRequired(key, commitTime); + } else if (TRIP_LOGICAL_TYPES_SCHEMA.equals(schemaStr)) { + return generatePayloadForLogicalTypesSchema(key, commitTime, false, timestamp); + } else if (TRIP_LOGICAL_TYPES_SCHEMA_V6.equals(schemaStr)) { + return generatePayloadForLogicalTypesSchemaV6(key, commitTime, false, timestamp); + } else if (TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS.equals(schemaStr)) { + return generatePayloadForLogicalTypesSchemaNoLTS(key, commitTime, false, timestamp); + } + } else { + if (TRIP_EXAMPLE_SCHEMA.equals(schemaStr)) { + return generateRandomDeleteValue(key, commitTime, timestamp); + } else if (TRIP_LOGICAL_TYPES_SCHEMA.equals(schemaStr)) { + return generatePayloadForLogicalTypesSchema(key, commitTime, true, timestamp); + } else if (TRIP_LOGICAL_TYPES_SCHEMA_V6.equals(schemaStr)) { + return generatePayloadForLogicalTypesSchemaV6(key, commitTime, true, timestamp); + } else if (TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS.equals(schemaStr)) { + return generatePayloadForLogicalTypesSchemaNoLTS(key, commitTime, true, timestamp); + } } return null; @@ -384,6 +445,21 @@ public IndexedRecord generatePayloadForTripEncodedDecimalSchema(HoodieKey key, S return generateRecordForTripEncodedDecimalSchema(key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, timestamp); } + /** + * LTS = Local Timestamp + */ + public IndexedRecord generatePayloadForLogicalTypesSchemaNoLTS(HoodieKey key, String commitTime, boolean isDelete, long timestamp) { + return generateRecordForTripLogicalTypesSchema(key, "rider-" + commitTime, "driver-" + commitTime, timestamp, isDelete, false, false); + } + + public IndexedRecord generatePayloadForLogicalTypesSchema(HoodieKey key, String commitTime, boolean isDelete, long timestamp) { + return generateRecordForTripLogicalTypesSchema(key, "rider-" + commitTime, "driver-" + commitTime, timestamp, isDelete, false, true); + } + + public IndexedRecord generatePayloadForLogicalTypesSchemaV6(HoodieKey key, String commitTime, boolean isDelete, long timestamp) { + return generateRecordForTripLogicalTypesSchema(key, "rider-" + commitTime, "driver-" + commitTime, timestamp, isDelete, true, true); + } + /** * Generates a new avro record with TRIP_SCHEMA, retaining the key if optionally provided. */ @@ -599,6 +675,96 @@ public GenericRecord generateRecordForTripEncodedDecimalSchema(String rowKey, St return rec; } + public GenericRecord generateRecordForTripLogicalTypesSchema(HoodieKey key, String riderName, String driverName, + long timestamp, boolean isDeleteRecord, boolean v6, boolean hasLTS) { + GenericRecord rec; + if (!hasLTS) { + // LTS = Local Timestamp + rec = new GenericData.Record(AVRO_TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS); + } else if (v6) { + rec = new GenericData.Record(AVRO_TRIP_LOGICAL_TYPES_SCHEMA_V6); + } else { + rec = new GenericData.Record(AVRO_TRIP_LOGICAL_TYPES_SCHEMA); + } + generateTripPrefixValues(rec, key.getRecordKey(), key.getPartitionPath(), riderName, driverName, timestamp); + + int hash = key.getRecordKey().hashCode(); + boolean above = (hash & 1) == 0; // half above, half below threshold + + // ------------------- + // Threshold definitions + // ------------------- + Instant tsMillisThreshold = Instant.parse("2020-01-01T00:00:00Z"); + Instant tsMicrosThreshold = Instant.parse("2020-06-01T12:00:00Z"); + + Instant localTsMillisThreshold = ZonedDateTime.of( + 2015, 5, 20, 12, 34, 56, 0, ZoneOffset.UTC).toInstant(); + Instant localTsMicrosThreshold = ZonedDateTime.of( + 2017, 7, 7, 7, 7, 7, 0, ZoneOffset.UTC).toInstant(); + + LocalDate dateThreshold = LocalDate.of(2000, 1, 1); + + // ------------------- + // Assign edge values + // ------------------- + + // ts_millis + long tsMillisBase = tsMillisThreshold.toEpochMilli(); + rec.put("ts_millis", above ? tsMillisBase + 1 : tsMillisBase - 1); + + // ts_micros + long tsMicrosBase = TimeUnit.SECONDS.toMicros(tsMicrosThreshold.getEpochSecond()) + tsMicrosThreshold.getNano() / 1_000L; + rec.put("ts_micros", above ? tsMicrosBase + 1 : tsMicrosBase - 1); + + if (hasLTS) { + // local_ts_millis + long localTsMillisBase = localTsMillisThreshold.toEpochMilli(); + rec.put("local_ts_millis", above ? localTsMillisBase + 1 : localTsMillisBase - 1); + + // local_ts_micros + long localTsMicrosBase = TimeUnit.SECONDS.toMicros(localTsMicrosThreshold.getEpochSecond()) + localTsMicrosThreshold.getNano() / 1_000L; + rec.put("local_ts_micros", above ? localTsMicrosBase + 1 : localTsMicrosBase - 1); + } + + // event_date + int eventDateBase = (int) dateThreshold.toEpochDay(); + rec.put("event_date", above ? eventDateBase + 1 : eventDateBase - 1); + + + // ------------------- + // Decimal thresholds + // ------------------- + BigDecimal decPlainLargeThreshold = new BigDecimal("1234567890.0987654321"); // precision=20, scale=10 + + BigDecimal decFixedSmallThreshold = new BigDecimal("543.21"); // precision=5, scale=2 + BigDecimal decFixedLargeThreshold = new BigDecimal("987654321.123456789"); // precision=18, scale=9 + + // Increment for just-above/below threshold = smallest possible unit for that scale + BigDecimal incSmallScale2 = new BigDecimal("0.01"); + BigDecimal incLargeScale9 = new BigDecimal("0.000000001"); + BigDecimal incLargeScale10 = new BigDecimal("0.0000000001"); + + // Assign thresholded decimals + if (!v6) { + rec.put("dec_plain_large", ByteBuffer.wrap((above + ? decPlainLargeThreshold.add(incLargeScale10) + : decPlainLargeThreshold.subtract(incLargeScale10)).unscaledValue().toByteArray())); + } + + Conversions.DecimalConversion decimalConversions = new Conversions.DecimalConversion(); + Schema decFixedSmallSchema = AVRO_TRIP_LOGICAL_TYPES_SCHEMA.getField("dec_fixed_small").schema(); + rec.put("dec_fixed_small", decimalConversions.toFixed(above + ? decFixedSmallThreshold.add(incSmallScale2) + : decFixedSmallThreshold.subtract(incSmallScale2), decFixedSmallSchema, LogicalTypes.decimal(5, 2))); + + Schema decFixedLargeSchema = AVRO_TRIP_LOGICAL_TYPES_SCHEMA.getField("dec_fixed_large").schema(); + rec.put("dec_fixed_large", decimalConversions.toFixed(above + ? decFixedLargeThreshold.add(incLargeScale9) + : decFixedLargeThreshold.subtract(incLargeScale9), decFixedLargeSchema, LogicalTypes.decimal(18, 9))); + generateTripSuffixValues(rec, isDeleteRecord); + return rec; + } + private static String getNonzeroEncodedBigDecimal(Random rand, int scale, int precision) { //scale the value because rand.nextDouble() only returns a val that is between 0 and 1 @@ -1154,9 +1320,13 @@ public Stream generateUniqueDeleteStream(Integer n) { * @return stream of hoodie records for delete */ private Stream generateUniqueDeleteRecordStream(String instantTime, Integer n, boolean updatePartition, long timestamp) { + return generateUniqueDeleteRecordStream(instantTime, n, updatePartition, TRIP_EXAMPLE_SCHEMA, timestamp); + } + + public Stream generateUniqueDeleteRecordStream(String instantTime, Integer n, boolean updatePartition, String schemaStr, long timestamp) { final Set used = new HashSet<>(); - Map existingKeys = existingKeysBySchema.get(TRIP_EXAMPLE_SCHEMA); - Integer numExistingKeys = numKeysBySchema.get(TRIP_EXAMPLE_SCHEMA); + Map existingKeys = existingKeysBySchema.get(schemaStr); + Integer numExistingKeys = numKeysBySchema.get(schemaStr); if (n > numExistingKeys) { throw new IllegalArgumentException("Requested unique deletes is greater than number of available keys"); } @@ -1180,12 +1350,12 @@ private Stream generateUniqueDeleteRecordStream(String instantTime key = new HoodieKey(key.getRecordKey(), updatedPartitionPath); } try { - result.add(new HoodieAvroIndexedRecord(key, generateRandomDeleteValue(key, instantTime, timestamp))); + result.add(new HoodieAvroIndexedRecord(key, generateRandomValueAsPerSchema(schemaStr, kp.key, instantTime, false, true, timestamp))); } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); } } - numKeysBySchema.put(TRIP_EXAMPLE_SCHEMA, numExistingKeys); + numKeysBySchema.put(schemaStr, numExistingKeys); return result.stream(); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestBaseFileUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestBaseFileUtils.java index 57cd85f35707a..d188f8d914102 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestBaseFileUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestBaseFileUtils.java @@ -19,7 +19,9 @@ package org.apache.hudi.common.util; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; +import org.apache.hudi.metadata.HoodieIndexVersion; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; +import org.apache.hudi.stats.ValueMetadata; import org.junit.jupiter.api.Test; @@ -36,14 +38,15 @@ public class TestBaseFileUtils { @Test public void testGetColumnRangeInPartition() { + HoodieIndexVersion indexVersion = HoodieIndexVersion.V1; // Step 1: Set Up Test Data HoodieColumnRangeMetadata fileColumnRange1 = HoodieColumnRangeMetadata.create( - "path/to/file1", COLUMN_NAME, 1, 5, 0, 10, 100, 200); + "path/to/file1", COLUMN_NAME, 1, 5, 0, 10, 100, 200, ValueMetadata.V1EmptyMetadata.get()); HoodieColumnRangeMetadata fileColumnRange2 = HoodieColumnRangeMetadata.create( - "path/to/file2", COLUMN_NAME, 3, 8, 1, 15, 120, 250); + "path/to/file2", COLUMN_NAME, 3, 8, 1, 15, 120, 250, ValueMetadata.V1EmptyMetadata.get()); List> fileColumnRanges = Arrays.asList(fileColumnRange1, fileColumnRange2); // Step 2: Call the Method - HoodieColumnRangeMetadata result = FileFormatUtils.getColumnRangeInPartition(PARTITION_PATH, fileColumnRanges, Collections.emptyMap()); + HoodieColumnRangeMetadata result = FileFormatUtils.getColumnRangeInPartition(PARTITION_PATH, COLUMN_NAME, fileColumnRanges, Collections.emptyMap(), indexVersion); // Step 3: Assertions assertEquals(PARTITION_PATH, result.getFilePath()); assertEquals(COLUMN_NAME, result.getColumnName()); @@ -57,15 +60,16 @@ public void testGetColumnRangeInPartition() { @Test public void testGetColumnRangeInPartitionWithNullMinMax() { + HoodieIndexVersion indexVersion = HoodieIndexVersion.V1; // Step 1: Set Up Test Data HoodieColumnRangeMetadata fileColumnRange1 = HoodieColumnRangeMetadata.create( - "path/to/file1", COLUMN_NAME, 1, null, 0, 10, 100, 200); + "path/to/file1", COLUMN_NAME, 1, null, 0, 10, 100, 200, ValueMetadata.V1EmptyMetadata.get()); HoodieColumnRangeMetadata fileColumnRange2 = HoodieColumnRangeMetadata.create( - "path/to/file2", COLUMN_NAME, null, 8, 1, 15, 120, 250); + "path/to/file2", COLUMN_NAME, null, 8, 1, 15, 120, 250, ValueMetadata.V1EmptyMetadata.get()); List> fileColumnRanges = Arrays.asList(fileColumnRange1, fileColumnRange2); // Step 2: Call the Method - HoodieColumnRangeMetadata result = FileFormatUtils.getColumnRangeInPartition(PARTITION_PATH, fileColumnRanges, Collections.emptyMap()); + HoodieColumnRangeMetadata result = FileFormatUtils.getColumnRangeInPartition(PARTITION_PATH, COLUMN_NAME, fileColumnRanges, Collections.emptyMap(), indexVersion); // Step 3: Assertions assertEquals(PARTITION_PATH, result.getFilePath()); assertEquals(COLUMN_NAME, result.getColumnName()); @@ -79,14 +83,15 @@ public void testGetColumnRangeInPartitionWithNullMinMax() { @Test public void testGetColumnRangeInPartitionWithDifferentColumnNameThrowsException() { + HoodieIndexVersion indexVersion = HoodieIndexVersion.V1; // Step 1: Set Up Test Data HoodieColumnRangeMetadata fileColumnRange1 = HoodieColumnRangeMetadata.create( - "path/to/file1", "columnName1", 1, null, 0, 10, 100, 200); + "path/to/file1", "columnName1", 1, null, 0, 10, 100, 200, ValueMetadata.V1EmptyMetadata.get()); HoodieColumnRangeMetadata fileColumnRange2 = HoodieColumnRangeMetadata.create( - "path/to/file2", "columnName2", null, 8, 1, 15, 120, 250); + "path/to/file2", "columnName2", null, 8, 1, 15, 120, 250, ValueMetadata.V1EmptyMetadata.get()); List> fileColumnRanges = Arrays.asList(fileColumnRange1, fileColumnRange2); // Step 2: Call the Method - assertThrows(IllegalArgumentException.class, () -> FileFormatUtils.getColumnRangeInPartition(PARTITION_PATH, fileColumnRanges, - Collections.emptyMap())); + assertThrows(IllegalArgumentException.class, () -> FileFormatUtils.getColumnRangeInPartition(PARTITION_PATH, COLUMN_NAME, fileColumnRanges, + Collections.emptyMap(), indexVersion)); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/metadata/TestHoodieIndexVersion.java b/hudi-common/src/test/java/org/apache/hudi/metadata/TestHoodieIndexVersion.java index a5c6031f4657d..d0ef7a1ececef 100644 --- a/hudi-common/src/test/java/org/apache/hudi/metadata/TestHoodieIndexVersion.java +++ b/hudi-common/src/test/java/org/apache/hudi/metadata/TestHoodieIndexVersion.java @@ -49,11 +49,11 @@ private static Stream getCurrentVersionTestCases() { Arguments.of("RECORD_INDEX", HoodieTableVersion.EIGHT, "record_index", HoodieIndexVersion.V1), Arguments.of("RECORD_INDEX", HoodieTableVersion.NINE, "record_index", HoodieIndexVersion.V1), Arguments.of("COLUMN_STATS", HoodieTableVersion.EIGHT, "column_stats", HoodieIndexVersion.V1), - Arguments.of("COLUMN_STATS", HoodieTableVersion.NINE, "column_stats", HoodieIndexVersion.V1), + Arguments.of("COLUMN_STATS", HoodieTableVersion.NINE, "column_stats", HoodieIndexVersion.V2), Arguments.of("BLOOM_FILTERS", HoodieTableVersion.EIGHT, "bloom_filters", HoodieIndexVersion.V1), Arguments.of("BLOOM_FILTERS", HoodieTableVersion.NINE, "bloom_filters", HoodieIndexVersion.V1), Arguments.of("BLOOM_FILTERS", HoodieTableVersion.EIGHT, "expr_index_idx1", HoodieIndexVersion.V1), - Arguments.of("BLOOM_FILTERS", HoodieTableVersion.NINE, "expr_index_idx1", HoodieIndexVersion.V1), + Arguments.of("BLOOM_FILTERS", HoodieTableVersion.NINE, "expr_index_idx1", HoodieIndexVersion.V2), Arguments.of("BLOOM_FILTERS", HoodieTableVersion.EIGHT, "secondary_index_idx1", HoodieIndexVersion.V1), Arguments.of("BLOOM_FILTERS", HoodieTableVersion.NINE, "secondary_index_idx1", HoodieIndexVersion.V2), Arguments.of("FILES", HoodieTableVersion.EIGHT, "files", HoodieIndexVersion.V1), @@ -61,7 +61,7 @@ private static Stream getCurrentVersionTestCases() { Arguments.of("EXPRESSION INDEX", HoodieTableVersion.EIGHT, "files", HoodieIndexVersion.V1), Arguments.of("EXPRESSION INDEX", HoodieTableVersion.NINE, "files", HoodieIndexVersion.V1), Arguments.of("PARTITION_STATS", HoodieTableVersion.EIGHT, "partition_stats", HoodieIndexVersion.V1), - Arguments.of("PARTITION_STATS", HoodieTableVersion.NINE, "partition_stats", HoodieIndexVersion.V1) + Arguments.of("PARTITION_STATS", HoodieTableVersion.NINE, "partition_stats", HoodieIndexVersion.V2) ); } diff --git a/hudi-common/src/test/java/org/apache/hudi/stats/TestValueMetadata.java b/hudi-common/src/test/java/org/apache/hudi/stats/TestValueMetadata.java new file mode 100644 index 0000000000000..861867aaff4fb --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/stats/TestValueMetadata.java @@ -0,0 +1,60 @@ +/* + * 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.stats; + +import org.apache.hudi.common.util.collection.Pair; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestValueMetadata { + + /** + * Do not change how we encode decimal without + * handling upgrade/downgrade and backwards compatibility + */ + @Test + public void testDecimalEncoding() { + ValueMetadata.DecimalValueMetadata dm = ValueMetadata.DecimalMetadata.create(1,4); + assertEquals("1,4", ValueMetadata.DecimalValueMetadata.encodeData(dm)); + dm = ValueMetadata.DecimalMetadata.create(12,4); + assertEquals("12,4", ValueMetadata.DecimalValueMetadata.encodeData(dm)); + dm = ValueMetadata.DecimalMetadata.create(12,14); + assertEquals("12,14", ValueMetadata.DecimalValueMetadata.encodeData(dm)); + } + + /** + * Do not change how we encode decimal without + * handling upgrade/downgrade and backwards compatibility + */ + @Test + public void testDecimalDecoding() { + Pair p = ValueMetadata.DecimalValueMetadata.decodeData("1,4"); + assertEquals(1, p.getLeft()); + assertEquals(4, p.getRight()); + p = ValueMetadata.DecimalValueMetadata.decodeData("12,4"); + assertEquals(12, p.getLeft()); + assertEquals(4, p.getRight()); + p = ValueMetadata.DecimalValueMetadata.decodeData("12,14"); + assertEquals(12, p.getLeft()); + assertEquals(14, p.getRight()); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/stats/TestValueType.java b/hudi-common/src/test/java/org/apache/hudi/stats/TestValueType.java new file mode 100644 index 0000000000000..d49664525583a --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/stats/TestValueType.java @@ -0,0 +1,57 @@ +/* + * 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.stats; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestValueType { + + @Test + public void testValueTypeNumbering() { + // DO NOT MODIFY THE ORDERING OF THE TYPE NUMBERING + assertEquals(0, ValueType.V1.ordinal()); + assertEquals(1, ValueType.NULL.ordinal()); + assertEquals(2, ValueType.BOOLEAN.ordinal()); + assertEquals(3, ValueType.INT.ordinal()); + assertEquals(4, ValueType.LONG.ordinal()); + assertEquals(5, ValueType.FLOAT.ordinal()); + assertEquals(6, ValueType.DOUBLE.ordinal()); + assertEquals(7, ValueType.STRING.ordinal()); + assertEquals(8, ValueType.BYTES.ordinal()); + assertEquals(9, ValueType.FIXED.ordinal()); + assertEquals(10, ValueType.DECIMAL.ordinal()); + assertEquals(11, ValueType.UUID.ordinal()); + assertEquals(12, ValueType.DATE.ordinal()); + assertEquals(13, ValueType.TIME_MILLIS.ordinal()); + assertEquals(14, ValueType.TIME_MICROS.ordinal()); + assertEquals(15, ValueType.TIMESTAMP_MILLIS.ordinal()); + assertEquals(16, ValueType.TIMESTAMP_MICROS.ordinal()); + assertEquals(17, ValueType.TIMESTAMP_NANOS.ordinal()); + assertEquals(18, ValueType.LOCAL_TIMESTAMP_MILLIS.ordinal()); + assertEquals(19, ValueType.LOCAL_TIMESTAMP_MICROS.ordinal()); + assertEquals(20, ValueType.LOCAL_TIMESTAMP_NANOS.ordinal()); + // IF YOU ADD A NEW TYPE, ADD IT TO THE END AND INCREMENT THE COUNT + // AND ALSO ASSERT IT HERE SO THAT SOMEONE DOESN'T MESS WITH IT + // IN THE FUTURE + assertEquals(21, ValueType.values().length); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestAvroSchemaConverter.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestAvroSchemaConverter.java index 2eaf5ef2e7e25..74b82eee1846e 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestAvroSchemaConverter.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestAvroSchemaConverter.java @@ -34,6 +34,7 @@ * Test cases for {@link org.apache.hudi.util.AvroSchemaConverter}. */ public class TestAvroSchemaConverter { + @Test void testUnionSchemaWithMultipleRecordTypes() { Schema schema = HoodieMetadataRecord.SCHEMA$; @@ -49,7 +50,8 @@ void testUnionSchemaWithMultipleRecordTypes() { + "`totalSize` BIGINT, " + "`totalUncompressedSize` BIGINT, " + "`isDeleted` BOOLEAN NOT NULL, " - + "`isTightBound` BOOLEAN NOT NULL>"; + + "`isTightBound` BOOLEAN NOT NULL, " + + "`valueType` ROW<`typeOrdinal` INT NOT NULL, `additionalInfo` STRING>>"; assertThat(dataType.getChildren().get(pos).toString(), is(expected)); } diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/common/table/ParquetTableSchemaResolver.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/table/ParquetTableSchemaResolver.java index 0b70677f862fa..066d381f23f6e 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/common/table/ParquetTableSchemaResolver.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/table/ParquetTableSchemaResolver.java @@ -21,9 +21,10 @@ import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; -import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.schema.MessageType; +import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; + public class ParquetTableSchemaResolver extends TableSchemaResolver { public ParquetTableSchemaResolver(HoodieTableMetaClient metaClient) { @@ -31,18 +32,15 @@ public ParquetTableSchemaResolver(HoodieTableMetaClient metaClient) { } public static MessageType convertAvroSchemaToParquet(Schema schema, Configuration hadoopConf) { - AvroSchemaConverter avroSchemaConverter = new AvroSchemaConverter(hadoopConf); - return avroSchemaConverter.convert(schema); + return getAvroSchemaConverter(hadoopConf).convert(schema); } private Schema convertParquetSchemaToAvro(MessageType parquetSchema) { - AvroSchemaConverter avroSchemaConverter = new AvroSchemaConverter(metaClient.getStorageConf().unwrapAs(Configuration.class)); - return avroSchemaConverter.convert(parquetSchema); + return getAvroSchemaConverter(metaClient.getStorageConf().unwrapAs(Configuration.class)).convert(parquetSchema); } private MessageType convertAvroSchemaToParquet(Schema schema) { - AvroSchemaConverter avroSchemaConverter = new AvroSchemaConverter(metaClient.getStorageConf().unwrapAs(Configuration.class)); - return avroSchemaConverter.convert(schema); + return getAvroSchemaConverter(metaClient.getStorageConf().unwrapAs(Configuration.class)).convert(schema); } /** diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/OrcUtils.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/OrcUtils.java index 99cfc0d1f9dc0..0ac4fc653e9b7 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/OrcUtils.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/OrcUtils.java @@ -19,7 +19,6 @@ package org.apache.hudi.common.util; import org.apache.hudi.avro.HoodieAvroUtils; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -32,6 +31,8 @@ import org.apache.hudi.hadoop.fs.HadoopFSUtils; import org.apache.hudi.io.hadoop.OrcReaderIterator; import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.metadata.HoodieIndexVersion; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; @@ -257,7 +258,7 @@ public Schema readAvroSchema(HoodieStorage storage, StoragePath filePath) { } @Override - public List> readColumnStatsFromMetadata(HoodieStorage storage, StoragePath filePath, List columnList) { + public List> readColumnStatsFromMetadata(HoodieStorage storage, StoragePath filePath, List columnList, HoodieIndexVersion indexVersion) { throw new UnsupportedOperationException( "Reading column statistics from metadata is not supported for ORC format yet"); } diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java index b06913bfcb417..e8cf21ea85bb1 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java @@ -22,7 +22,6 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.HoodieAvroWriteSupport; import org.apache.hudi.common.config.HoodieConfig; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -34,6 +33,10 @@ import org.apache.hudi.io.storage.HoodieFileWriter; import org.apache.hudi.io.storage.HoodieFileWriterFactory; import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.metadata.HoodieIndexVersion; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; +import org.apache.hudi.stats.ValueMetadata; +import org.apache.hudi.stats.ValueType; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; @@ -43,7 +46,6 @@ import org.apache.hadoop.fs.Path; import org.apache.parquet.avro.AvroParquetReader; import org.apache.parquet.avro.AvroReadSupport; -import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.column.statistics.Statistics; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.ParquetReader; @@ -83,6 +85,7 @@ import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_MAX_FILE_SIZE; import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_PAGE_SIZE; import static org.apache.hudi.hadoop.fs.HadoopFSUtils.convertToStoragePath; +import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; /** * Utility functions involving with parquet. @@ -268,42 +271,47 @@ public Map readFooter(HoodieStorage storage, boolean required, @Override public Schema readAvroSchema(HoodieStorage storage, StoragePath filePath) { MessageType parquetSchema = readSchema(storage, filePath); - return new AvroSchemaConverter(storage.getConf().unwrapAs(Configuration.class)).convert(parquetSchema); + return getAvroSchemaConverter(storage.getConf().unwrapAs(Configuration.class)).convert(parquetSchema); } @Override public List> readColumnStatsFromMetadata(HoodieStorage storage, StoragePath filePath, - List columnList) { + List columnList, + HoodieIndexVersion indexVersion) { ParquetMetadata metadata = readMetadata(storage, filePath); - return readColumnStatsFromMetadata(metadata, filePath.getName(), Option.of(columnList)); + return readColumnStatsFromMetadata(metadata, filePath.getName(), Option.of(columnList), indexVersion); } - public List> readColumnStatsFromMetadata(ParquetMetadata metadata, String filePath, Option> columnList) { + public List> readColumnStatsFromMetadata(ParquetMetadata metadata, String filePath, Option> columnList, HoodieIndexVersion indexVersion) { // Collect stats from all individual Parquet blocks Stream> hoodieColumnRangeMetadataStream = metadata.getBlocks().stream().sequential().flatMap(blockMetaData -> blockMetaData.getColumns().stream() - .filter(f -> !columnList.isPresent() || columnList.get().contains(f.getPath().toDotString())) - .map(columnChunkMetaData -> { - Statistics stats = columnChunkMetaData.getStatistics(); - return (HoodieColumnRangeMetadata) HoodieColumnRangeMetadata.create( - filePath, - columnChunkMetaData.getPath().toDotString(), - convertToNativeJavaType( - columnChunkMetaData.getPrimitiveType(), - stats.genericGetMin()), - convertToNativeJavaType( - columnChunkMetaData.getPrimitiveType(), - stats.genericGetMax()), - // NOTE: In case when column contains only nulls Parquet won't be creating - // stats for it instead returning stubbed (empty) object. In that case - // we have to equate number of nulls to the value count ourselves - stats.isEmpty() ? columnChunkMetaData.getValueCount() : stats.getNumNulls(), - columnChunkMetaData.getValueCount(), - columnChunkMetaData.getTotalSize(), - columnChunkMetaData.getTotalUncompressedSize()); - }) + .filter(f -> !columnList.isPresent() || columnList.get().contains(f.getPath().toDotString())) + .map(columnChunkMetaData -> { + Statistics stats = columnChunkMetaData.getStatistics(); + ValueMetadata valueMetadata = ValueMetadata.getValueMetadata(columnChunkMetaData.getPrimitiveType(), indexVersion); + return (HoodieColumnRangeMetadata) HoodieColumnRangeMetadata.create( + filePath, + columnChunkMetaData.getPath().toDotString(), + convertToNativeJavaType( + columnChunkMetaData.getPrimitiveType(), + stats.genericGetMin(), + valueMetadata), + convertToNativeJavaType( + columnChunkMetaData.getPrimitiveType(), + stats.genericGetMax(), + valueMetadata), + // NOTE: In case when column contains only nulls Parquet won't be creating + // stats for it instead returning stubbed (empty) object. In that case + // we have to equate number of nulls to the value count ourselves + stats.isEmpty() ? columnChunkMetaData.getValueCount() : stats.getNumNulls(), + columnChunkMetaData.getValueCount(), + columnChunkMetaData.getTotalSize(), + columnChunkMetaData.getTotalUncompressedSize(), + valueMetadata); + }) ); return mergeColumnStats(hoodieColumnRangeMetadataStream); @@ -391,11 +399,11 @@ public void writeMetaFile(HoodieStorage storage, @Override public ByteArrayOutputStream serializeRecordsToLogBlock(HoodieStorage storage, - List records, - Schema writerSchema, - Schema readerSchema, - String keyFieldName, - Map paramsMap) throws IOException { + List records, + Schema writerSchema, + Schema readerSchema, + String keyFieldName, + Map paramsMap) throws IOException { if (records.size() == 0) { return new ByteArrayOutputStream(0); } @@ -474,11 +482,15 @@ private > HoodieColumnRangeMetadata getColumnRangeInF .reduce(HoodieColumnRangeMetadata::merge).get(); } - private static Comparable convertToNativeJavaType(PrimitiveType primitiveType, Comparable val) { + private static Comparable convertToNativeJavaType(PrimitiveType primitiveType, Comparable val, ValueMetadata valueMetadata) { if (val == null) { return null; } + if (valueMetadata.getValueType() != ValueType.V1) { + return valueMetadata.standardizeJavaTypeAndPromote(val); + } + if (primitiveType.getOriginalType() == OriginalType.DECIMAL) { return extractDecimal(val, primitiveType.getDecimalMetadata()); } else if (primitiveType.getOriginalType() == OriginalType.DATE) { diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroFileWriterFactory.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroFileWriterFactory.java index 862259d33f274..2b1edb1238c1b 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroFileWriterFactory.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroFileWriterFactory.java @@ -41,7 +41,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.orc.CompressionKind; -import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.parquet.schema.MessageType; @@ -49,6 +48,8 @@ import java.io.OutputStream; import java.util.Properties; +import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; + public class HoodieAvroFileWriterFactory extends HoodieFileWriterFactory { public HoodieAvroFileWriterFactory(HoodieStorage storage) { @@ -122,6 +123,6 @@ private HoodieAvroWriteSupport getHoodieAvroWriteSupport(Schema schema, return (HoodieAvroWriteSupport) ReflectionUtils.loadClass( config.getStringOrDefault(HoodieStorageConfig.HOODIE_AVRO_WRITE_SUPPORT_CLASS), new Class[] {MessageType.class, Schema.class, Option.class, Properties.class}, - new AvroSchemaConverter(storage.getConf().unwrapAs(Configuration.class)).convert(schema), schema, filter, config.getProps()); + getAvroSchemaConverter(storage.getConf().unwrapAs(Configuration.class)).convert(schema), schema, filter, config.getProps()); } } \ No newline at end of file diff --git a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java new file mode 100644 index 0000000000000..c1bda6ef0ffb7 --- /dev/null +++ b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/AvroSchemaConverterWithTimestampNTZ.java @@ -0,0 +1,598 @@ +/* + * 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.parquet.avro; + +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.schema.ConversionPatterns; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.LogicalTypeAnnotation.UUIDLogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Types; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.avro.JsonProperties.NULL_VALUE; +import static org.apache.parquet.avro.AvroReadSupport.READ_INT96_AS_FIXED; +import static org.apache.parquet.avro.AvroReadSupport.READ_INT96_AS_FIXED_DEFAULT; +import static org.apache.parquet.avro.AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE; +import static org.apache.parquet.avro.AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE_DEFAULT; +import static org.apache.parquet.avro.AvroWriteSupport.WRITE_PARQUET_UUID; +import static org.apache.parquet.avro.AvroWriteSupport.WRITE_PARQUET_UUID_DEFAULT; +import static org.apache.parquet.schema.LogicalTypeAnnotation.TimeUnit.MICROS; +import static org.apache.parquet.schema.LogicalTypeAnnotation.TimeUnit.MILLIS; +import static org.apache.parquet.schema.LogicalTypeAnnotation.dateType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.decimalType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.enumType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.stringType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.timeType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.timestampType; +import static org.apache.parquet.schema.LogicalTypeAnnotation.uuidType; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; +import static org.apache.parquet.schema.Type.Repetition.REPEATED; + +/** + *

+ * Converts an Avro schema into a Parquet schema, or vice versa. See package + * documentation for details of the mapping. + *

+ * This was taken from parquet-java 1.13.1 AvroSchemaConverter and modified + * to support local timestamp types by copying a few methods from 1.14.0 AvroSchemaConverter. + */ +@SuppressWarnings("all") +public class AvroSchemaConverterWithTimestampNTZ extends HoodieAvroParquetSchemaConverter { + + public static final String ADD_LIST_ELEMENT_RECORDS = + "parquet.avro.add-list-element-records"; + private static final boolean ADD_LIST_ELEMENT_RECORDS_DEFAULT = true; + + private final boolean assumeRepeatedIsListElement; + private final boolean writeOldListStructure; + private final boolean writeParquetUUID; + private final boolean readInt96AsFixed; + private final Set pathsToInt96; + + public AvroSchemaConverterWithTimestampNTZ() { + this(ADD_LIST_ELEMENT_RECORDS_DEFAULT); + } + + /** + * Constructor used by {@link AvroRecordConverter#isElementType}, which always + * uses the 2-level list conversion. + * + * @param assumeRepeatedIsListElement whether to assume 2-level lists + */ + AvroSchemaConverterWithTimestampNTZ(boolean assumeRepeatedIsListElement) { + this.assumeRepeatedIsListElement = assumeRepeatedIsListElement; + this.writeOldListStructure = WRITE_OLD_LIST_STRUCTURE_DEFAULT; + this.writeParquetUUID = WRITE_PARQUET_UUID_DEFAULT; + this.readInt96AsFixed = READ_INT96_AS_FIXED_DEFAULT; + this.pathsToInt96 = Collections.emptySet(); + } + + public AvroSchemaConverterWithTimestampNTZ(Configuration conf) { + this.assumeRepeatedIsListElement = conf.getBoolean( + ADD_LIST_ELEMENT_RECORDS, ADD_LIST_ELEMENT_RECORDS_DEFAULT); + this.writeOldListStructure = conf.getBoolean( + WRITE_OLD_LIST_STRUCTURE, WRITE_OLD_LIST_STRUCTURE_DEFAULT); + this.writeParquetUUID = conf.getBoolean(WRITE_PARQUET_UUID, WRITE_PARQUET_UUID_DEFAULT); + this.readInt96AsFixed = conf.getBoolean(READ_INT96_AS_FIXED, READ_INT96_AS_FIXED_DEFAULT); + this.pathsToInt96 = new HashSet<>(Arrays.asList(conf.getStrings("parquet.avro.writeFixedAsInt96", new String[0]))); + } + + /** + * Given a schema, check to see if it is a union of a null type and a regular schema, + * and then return the non-null sub-schema. Otherwise, return the given schema. + * + * @param schema The schema to check + * @return The non-null portion of a union schema, or the given schema + */ + public static Schema getNonNull(Schema schema) { + if (schema.getType().equals(Schema.Type.UNION)) { + List schemas = schema.getTypes(); + if (schemas.size() == 2) { + if (schemas.get(0).getType().equals(Schema.Type.NULL)) { + return schemas.get(1); + } else if (schemas.get(1).getType().equals(Schema.Type.NULL)) { + return schemas.get(0); + } else { + return schema; + } + } else { + return schema; + } + } else { + return schema; + } + } + + @Override + public MessageType convert(Schema avroSchema) { + if (!avroSchema.getType().equals(Schema.Type.RECORD)) { + throw new IllegalArgumentException("Avro schema must be a record."); + } + return new MessageType(avroSchema.getFullName(), convertFields(avroSchema.getFields(), "")); + } + + private List convertFields(List fields, String schemaPath) { + List types = new ArrayList(); + for (Schema.Field field : fields) { + if (field.schema().getType().equals(Schema.Type.NULL)) { + continue; // Avro nulls are not encoded, unless they are null unions + } + types.add(convertField(field, appendPath(schemaPath, field.name()))); + } + return types; + } + + private Type convertField(String fieldName, Schema schema, String schemaPath) { + return convertField(fieldName, schema, Type.Repetition.REQUIRED, schemaPath); + } + + @SuppressWarnings("deprecation") + private Type convertField(String fieldName, Schema schema, Type.Repetition repetition, String schemaPath) { + Types.PrimitiveBuilder builder; + Schema.Type type = schema.getType(); + LogicalType logicalType = schema.getLogicalType(); + if (type.equals(Schema.Type.BOOLEAN)) { + builder = Types.primitive(BOOLEAN, repetition); + } else if (type.equals(Schema.Type.INT)) { + builder = Types.primitive(INT32, repetition); + } else if (type.equals(Schema.Type.LONG)) { + builder = Types.primitive(INT64, repetition); + } else if (type.equals(Schema.Type.FLOAT)) { + builder = Types.primitive(FLOAT, repetition); + } else if (type.equals(Schema.Type.DOUBLE)) { + builder = Types.primitive(DOUBLE, repetition); + } else if (type.equals(Schema.Type.BYTES)) { + builder = Types.primitive(BINARY, repetition); + } else if (type.equals(Schema.Type.STRING)) { + if (logicalType != null && logicalType.getName().equals(LogicalTypes.uuid().getName()) && writeParquetUUID) { + builder = Types.primitive(FIXED_LEN_BYTE_ARRAY, repetition) + .length(LogicalTypeAnnotation.UUIDLogicalTypeAnnotation.BYTES); + } else { + builder = Types.primitive(BINARY, repetition).as(stringType()); + } + } else if (type.equals(Schema.Type.RECORD)) { + return new GroupType(repetition, fieldName, convertFields(schema.getFields(), schemaPath)); + } else if (type.equals(Schema.Type.ENUM)) { + builder = Types.primitive(BINARY, repetition).as(enumType()); + } else if (type.equals(Schema.Type.ARRAY)) { + if (writeOldListStructure) { + return ConversionPatterns.listType(repetition, fieldName, + convertField("array", schema.getElementType(), REPEATED, schemaPath)); + } else { + return ConversionPatterns.listOfElements(repetition, fieldName, + convertField(AvroWriteSupport.LIST_ELEMENT_NAME, schema.getElementType(), schemaPath)); + } + } else if (type.equals(Schema.Type.MAP)) { + Type valType = convertField("value", schema.getValueType(), schemaPath); + // avro map key type is always string + return ConversionPatterns.stringKeyMapType(repetition, fieldName, valType); + } else if (type.equals(Schema.Type.FIXED)) { + if (pathsToInt96.contains(schemaPath)) { + if (schema.getFixedSize() != 12) { + throw new IllegalArgumentException( + "The size of the fixed type field " + schemaPath + " must be 12 bytes for INT96 conversion"); + } + builder = Types.primitive(PrimitiveTypeName.INT96, repetition); + } else { + builder = Types.primitive(FIXED_LEN_BYTE_ARRAY, repetition).length(schema.getFixedSize()); + } + } else if (type.equals(Schema.Type.UNION)) { + return convertUnion(fieldName, schema, repetition, schemaPath); + } else { + throw new UnsupportedOperationException("Cannot convert Avro type " + type); + } + + // schema translation can only be done for known logical types because this + // creates an equivalence + if (logicalType != null) { + if (logicalType instanceof LogicalTypes.Decimal) { + LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; + builder = builder.as(decimalType(decimal.getScale(), decimal.getPrecision())); + } else { + LogicalTypeAnnotation annotation = convertLogicalType(logicalType); + if (annotation != null) { + builder.as(annotation); + } + } + } + + return builder.named(fieldName); + } + + private Type convertUnion(String fieldName, Schema schema, Type.Repetition repetition, String schemaPath) { + List nonNullSchemas = new ArrayList(schema.getTypes().size()); + // Found any schemas in the union? Required for the edge case, where the union contains only a single type. + boolean foundNullSchema = false; + for (Schema childSchema : schema.getTypes()) { + if (childSchema.getType().equals(Schema.Type.NULL)) { + foundNullSchema = true; + if (Type.Repetition.REQUIRED == repetition) { + repetition = Type.Repetition.OPTIONAL; + } + } else { + nonNullSchemas.add(childSchema); + } + } + // If we only get a null and one other type then its a simple optional field + // otherwise construct a union container + switch (nonNullSchemas.size()) { + case 0: + throw new UnsupportedOperationException("Cannot convert Avro union of only nulls"); + + case 1: + return foundNullSchema ? convertField(fieldName, nonNullSchemas.get(0), repetition, schemaPath) : + convertUnionToGroupType(fieldName, repetition, nonNullSchemas, schemaPath); + + default: // complex union type + return convertUnionToGroupType(fieldName, repetition, nonNullSchemas, schemaPath); + } + } + + private Type convertUnionToGroupType(String fieldName, Type.Repetition repetition, List nonNullSchemas, + String schemaPath) { + List unionTypes = new ArrayList(nonNullSchemas.size()); + int index = 0; + for (Schema childSchema : nonNullSchemas) { + unionTypes.add( convertField("member" + index++, childSchema, Type.Repetition.OPTIONAL, schemaPath)); + } + return new GroupType(repetition, fieldName, unionTypes); + } + + private Type convertField(Schema.Field field, String schemaPath) { + return convertField(field.name(), field.schema(), schemaPath); + } + + @Override + public Schema convert(MessageType parquetSchema) { + return convertFields(parquetSchema.getName(), parquetSchema.getFields(), new HashMap<>()); + } + + Schema convert(GroupType parquetSchema) { + return convertFields(parquetSchema.getName(), parquetSchema.getFields(), new HashMap<>()); + } + + private Schema convertFields(String name, List parquetFields, Map names) { + List fields = new ArrayList(); + Integer nameCount = names.merge(name, 1, (oldValue, value) -> oldValue + 1); + for (Type parquetType : parquetFields) { + Schema fieldSchema = convertField(parquetType, names); + if (parquetType.isRepetition(REPEATED)) { + throw new UnsupportedOperationException("REPEATED not supported outside LIST or MAP. Type: " + parquetType); + } else if (parquetType.isRepetition(Type.Repetition.OPTIONAL)) { + fields.add(new Schema.Field( + parquetType.getName(), optional(fieldSchema), null, NULL_VALUE)); + } else { // REQUIRED + fields.add(new Schema.Field( + parquetType.getName(), fieldSchema, null, (Object) null)); + } + } + Schema schema = Schema.createRecord(name, null, nameCount > 1 ? name + nameCount : null, false); + schema.setFields(fields); + return schema; + } + + private Schema convertField(final Type parquetType, Map names) { + if (parquetType.isPrimitive()) { + final PrimitiveType asPrimitive = parquetType.asPrimitiveType(); + final PrimitiveTypeName parquetPrimitiveTypeName = + asPrimitive.getPrimitiveTypeName(); + final LogicalTypeAnnotation annotation = parquetType.getLogicalTypeAnnotation(); + Schema schema = parquetPrimitiveTypeName.convert( + new PrimitiveType.PrimitiveTypeNameConverter() { + @Override + public Schema convertBOOLEAN(PrimitiveTypeName primitiveTypeName) { + return Schema.create(Schema.Type.BOOLEAN); + } + @Override + public Schema convertINT32(PrimitiveTypeName primitiveTypeName) { + return Schema.create(Schema.Type.INT); + } + @Override + public Schema convertINT64(PrimitiveTypeName primitiveTypeName) { + return Schema.create(Schema.Type.LONG); + } + @Override + public Schema convertINT96(PrimitiveTypeName primitiveTypeName) { + if (readInt96AsFixed) { + return Schema.createFixed("INT96", "INT96 represented as byte[12]", null, 12); + } + throw new IllegalArgumentException( + "INT96 is deprecated. As interim enable READ_INT96_AS_FIXED flag to read as byte array."); + } + @Override + public Schema convertFLOAT(PrimitiveTypeName primitiveTypeName) { + return Schema.create(Schema.Type.FLOAT); + } + @Override + public Schema convertDOUBLE(PrimitiveTypeName primitiveTypeName) { + return Schema.create(Schema.Type.DOUBLE); + } + @Override + public Schema convertFIXED_LEN_BYTE_ARRAY(PrimitiveTypeName primitiveTypeName) { + if (annotation instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { + return Schema.create(Schema.Type.STRING); + } else { + int size = parquetType.asPrimitiveType().getTypeLength(); + return Schema.createFixed(parquetType.getName(), null, null, size); + } + } + @Override + public Schema convertBINARY(PrimitiveTypeName primitiveTypeName) { + if (annotation instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation || + annotation instanceof LogicalTypeAnnotation.EnumLogicalTypeAnnotation) { + return Schema.create(Schema.Type.STRING); + } else { + return Schema.create(Schema.Type.BYTES); + } + } + }); + + LogicalType logicalType = convertLogicalType(annotation); + if (logicalType != null && (!(annotation instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) || + parquetPrimitiveTypeName == BINARY || + parquetPrimitiveTypeName == FIXED_LEN_BYTE_ARRAY)) { + schema = logicalType.addToSchema(schema); + } + + return schema; + + } else { + GroupType parquetGroupType = parquetType.asGroupType(); + LogicalTypeAnnotation logicalTypeAnnotation = parquetGroupType.getLogicalTypeAnnotation(); + if (logicalTypeAnnotation != null) { + return logicalTypeAnnotation.accept(new LogicalTypeAnnotation.LogicalTypeAnnotationVisitor() { + @Override + public java.util.Optional visit(LogicalTypeAnnotation.ListLogicalTypeAnnotation listLogicalType) { + if (parquetGroupType.getFieldCount()!= 1) { + throw new UnsupportedOperationException("Invalid list type " + parquetGroupType); + } + Type repeatedType = parquetGroupType.getType(0); + if (!repeatedType.isRepetition(REPEATED)) { + throw new UnsupportedOperationException("Invalid list type " + parquetGroupType); + } + if (isElementType(repeatedType, parquetGroupType.getName())) { + // repeated element types are always required + return java.util.Optional.of(Schema.createArray(convertField(repeatedType, names))); + } else { + Type elementType = repeatedType.asGroupType().getType(0); + if (elementType.isRepetition(Type.Repetition.OPTIONAL)) { + return java.util.Optional.of(Schema.createArray(optional(convertField(elementType, names)))); + } else { + return java.util.Optional.of(Schema.createArray(convertField(elementType, names))); + } + } + } + + @Override + // for backward-compatibility + public java.util.Optional visit(LogicalTypeAnnotation.MapKeyValueTypeAnnotation mapKeyValueLogicalType) { + return visitMapOrMapKeyValue(); + } + + @Override + public java.util.Optional visit(LogicalTypeAnnotation.MapLogicalTypeAnnotation mapLogicalType) { + return visitMapOrMapKeyValue(); + } + + private java.util.Optional visitMapOrMapKeyValue() { + if (parquetGroupType.getFieldCount() != 1 || parquetGroupType.getType(0).isPrimitive()) { + throw new UnsupportedOperationException("Invalid map type " + parquetGroupType); + } + GroupType mapKeyValType = parquetGroupType.getType(0).asGroupType(); + if (!mapKeyValType.isRepetition(REPEATED) || + mapKeyValType.getFieldCount()!=2) { + throw new UnsupportedOperationException("Invalid map type " + parquetGroupType); + } + Type keyType = mapKeyValType.getType(0); + if (!keyType.isPrimitive() || + !keyType.asPrimitiveType().getPrimitiveTypeName().equals(BINARY) || + !keyType.getLogicalTypeAnnotation().equals(stringType())) { + throw new IllegalArgumentException("Map key type must be binary (UTF8): " + + keyType); + } + Type valueType = mapKeyValType.getType(1); + if (valueType.isRepetition(Type.Repetition.OPTIONAL)) { + return java.util.Optional.of(Schema.createMap(optional(convertField(valueType, names)))); + } else { + return java.util.Optional.of(Schema.createMap(convertField(valueType, names))); + } + } + + @Override + public java.util.Optional visit(LogicalTypeAnnotation.EnumLogicalTypeAnnotation enumLogicalType) { + return java.util.Optional.of(Schema.create(Schema.Type.STRING)); + } + }).orElseThrow(() -> new UnsupportedOperationException("Cannot convert Parquet type " + parquetType)); + } else { + // if no original type then it's a record + return convertFields(parquetGroupType.getName(), parquetGroupType.getFields(), names); + } + } + } + + private LogicalTypeAnnotation convertLogicalType(LogicalType logicalType) { + if (logicalType == null) { + return null; + } else if (logicalType instanceof LogicalTypes.Decimal) { + LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; + return decimalType(decimal.getScale(), decimal.getPrecision()); + } else if (logicalType instanceof LogicalTypes.Date) { + return dateType(); + } else if (logicalType instanceof LogicalTypes.TimeMillis) { + return timeType(true, MILLIS); + } else if (logicalType instanceof LogicalTypes.TimeMicros) { + return timeType(true, MICROS); + } else if (logicalType instanceof LogicalTypes.TimestampMillis) { + return timestampType(true, MILLIS); + } else if (logicalType instanceof LogicalTypes.TimestampMicros) { + return timestampType(true, MICROS); + } else if (logicalType.getName().equals(LogicalTypes.uuid().getName()) && writeParquetUUID) { + return uuidType(); + } + + if (avroVersionSupportsLocalTimestampTypes()) { + if (logicalType instanceof LogicalTypes.LocalTimestampMillis) { + return timestampType(false, MILLIS); + } else if (logicalType instanceof LogicalTypes.LocalTimestampMicros) { + return timestampType(false, MICROS); + } + } + + return null; + } + + private LogicalType convertLogicalType(LogicalTypeAnnotation annotation) { + if (annotation == null) { + return null; + } + return annotation + .accept(new LogicalTypeAnnotation.LogicalTypeAnnotationVisitor() { + @Override + public java.util.Optional visit( + LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalLogicalType) { + return java.util.Optional.of( + LogicalTypes.decimal(decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); + } + + @Override + public java.util.Optional visit( + LogicalTypeAnnotation.DateLogicalTypeAnnotation dateLogicalType) { + return java.util.Optional.of(LogicalTypes.date()); + } + + @Override + public java.util.Optional visit( + LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeLogicalType) { + LogicalTypeAnnotation.TimeUnit unit = timeLogicalType.getUnit(); + switch (unit) { + case MILLIS: + return java.util.Optional.of(LogicalTypes.timeMillis()); + case MICROS: + return java.util.Optional.of(LogicalTypes.timeMicros()); + } + return java.util.Optional.empty(); + } + + @Override + public java.util.Optional visit( + LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampLogicalType) { + LogicalTypeAnnotation.TimeUnit unit = timestampLogicalType.getUnit(); + boolean isAdjustedToUTC = timestampLogicalType.isAdjustedToUTC(); + + if (isAdjustedToUTC || !avroVersionSupportsLocalTimestampTypes()) { + switch (unit) { + case MILLIS: + return java.util.Optional.of(LogicalTypes.timestampMillis()); + case MICROS: + return java.util.Optional.of(LogicalTypes.timestampMicros()); + } + return java.util.Optional.empty(); + } else { + switch (unit) { + case MILLIS: + return java.util.Optional.of(LogicalTypes.localTimestampMillis()); + case MICROS: + return java.util.Optional.of(LogicalTypes.localTimestampMicros()); + } + return java.util.Optional.empty(); + } + } + + @Override + public java.util.Optional visit(UUIDLogicalTypeAnnotation uuidLogicalType) { + return java.util.Optional.of(LogicalTypes.uuid()); + } + }) + .orElse(null); + } + + /** + * Implements the rules for interpreting existing data from the logical type + * spec for the LIST annotation. This is used to produce the expected schema. + *

+ * The AvroArrayConverter will decide whether the repeated type is the array + * element type by testing whether the element schema and repeated type are + * the same. This ensures that the LIST rules are followed when there is no + * schema and that a schema can be provided to override the default behavior. + */ + private boolean isElementType(Type repeatedType, String parentName) { + return ( + // can't be a synthetic layer because it would be invalid + repeatedType.isPrimitive() || + repeatedType.asGroupType().getFieldCount() > 1 || + repeatedType.asGroupType().getType(0).isRepetition(REPEATED) || + // known patterns without the synthetic layer + repeatedType.getName().equals("array") || + repeatedType.getName().equals(parentName + "_tuple") || + // default assumption + assumeRepeatedIsListElement + ); + } + + private static Schema optional(Schema original) { + // null is first in the union because Parquet's default is always null + return Schema.createUnion(Arrays.asList( + Schema.create(Schema.Type.NULL), + original)); + } + + private static String appendPath(String path, String fieldName) { + if (path == null || path.isEmpty()) { + return fieldName; + } + return path + '.' + fieldName; + } + + /* Avro <= 1.9 does not support conversions to LocalTimestamp{Micros, Millis} classes */ + private static boolean avroVersionSupportsLocalTimestampTypes() { + final String avroVersion = getRuntimeAvroVersion(); + + return avroVersion == null + || !(avroVersion.startsWith("1.7.") + || avroVersion.startsWith("1.8.") + || avroVersion.startsWith("1.9.")); + } + + private static String getRuntimeAvroVersion() { + return Schema.Parser.class.getPackage().getImplementationVersion(); + } +} diff --git a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetSchemaConverter.java b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetSchemaConverter.java new file mode 100644 index 0000000000000..5473afd36d83b --- /dev/null +++ b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetSchemaConverter.java @@ -0,0 +1,57 @@ +/* + * 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.parquet.avro; + +import org.apache.hudi.common.util.ReflectionUtils; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.schema.MessageType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Parquet-Java AvroSchemaConverter doesn't support local timestamp types until version 1.14 + * for this reason we use a modified version of the AvroSchemaConverter that adds support for local timestamp types + * Parquet-Java still supports local timestamp types from version 1.11.0, just that the AvroSchemaConverter + * doesn't work. + *

+ * However, for versions < 1.11.0, local timestamp is not supported at all. Therefore, we just use the + * library AvroSchemaConverter in this case. + * + */ +public abstract class HoodieAvroParquetSchemaConverter { + + private static final Logger LOG = LoggerFactory.getLogger(HoodieAvroParquetSchemaConverter.class); + public static HoodieAvroParquetSchemaConverter getAvroSchemaConverter(Configuration configuration) { + try { + return (HoodieAvroParquetSchemaConverter) ReflectionUtils.loadClass("org.apache.parquet.avro.AvroSchemaConverterWithTimestampNTZ", + new Class[] {Configuration.class}, configuration); + } catch (Throwable t) { + LOG.debug("Failed to load AvroSchemaConverterWithTimestampNTZ, using NativeAvroSchemaConverter instead", t); + return (HoodieAvroParquetSchemaConverter) ReflectionUtils.loadClass("org.apache.parquet.avro.NativeAvroSchemaConverter", + new Class[] {Configuration.class}, configuration); + } + } + + public abstract MessageType convert(Schema schema); + + public abstract Schema convert(MessageType schema); +} diff --git a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/NativeAvroSchemaConverter.java b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/NativeAvroSchemaConverter.java new file mode 100644 index 0000000000000..509a8afb3ea83 --- /dev/null +++ b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/NativeAvroSchemaConverter.java @@ -0,0 +1,46 @@ +/* + * 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.parquet.avro; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.schema.MessageType; + +/** + * uses the native avro schema converter from parquet java + */ +public class NativeAvroSchemaConverter extends HoodieAvroParquetSchemaConverter { + + private final AvroSchemaConverter avroSchemaConverter; + + public NativeAvroSchemaConverter(Configuration configuration) { + this.avroSchemaConverter = new AvroSchemaConverter(configuration); + } + + @Override + public MessageType convert(Schema schema) { + return avroSchemaConverter.convert(schema); + } + + @Override + public Schema convert(MessageType schema) { + return avroSchemaConverter.convert(schema); + } +} diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java index 17e80ab63f95b..c567f3fb4386f 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java @@ -24,7 +24,6 @@ import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.bloom.BloomFilterTypeCode; import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; @@ -32,6 +31,7 @@ import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; import org.apache.hudi.storage.StoragePath; import org.apache.avro.JsonProperties; @@ -64,6 +64,7 @@ import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema; import static org.apache.hudi.avro.HoodieAvroUtils.METADATA_FIELD_SCHEMA; +import static org.apache.hudi.metadata.HoodieIndexVersion.V1; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -278,7 +279,7 @@ public void testReadColumnStatsFromMetadata() throws Exception { columnList.add(dataField); List> columnRangeMetadataList = parquetUtils.readColumnStatsFromMetadata( - HoodieTestUtils.getStorage(filePath), new StoragePath(filePath), columnList) + HoodieTestUtils.getStorage(filePath), new StoragePath(filePath), columnList, V1) .stream() .sorted(Comparator.comparing(HoodieColumnRangeMetadata::getColumnName)) .collect(Collectors.toList()); diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/metadata/TestHoodieMetadataPayload.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/metadata/TestHoodieMetadataPayload.java index 29a27860b762b..f928b46b7c9c2 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/metadata/TestHoodieMetadataPayload.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/metadata/TestHoodieMetadataPayload.java @@ -18,11 +18,12 @@ package org.apache.hudi.metadata; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; +import org.apache.hudi.stats.ValueMetadata; import org.apache.avro.generic.IndexedRecord; import org.junit.jupiter.api.Test; @@ -35,6 +36,7 @@ import java.util.Map; import static org.apache.hudi.common.util.CollectionUtils.createImmutableMap; +import static org.apache.hudi.metadata.HoodieIndexVersion.V1; import static org.apache.hudi.metadata.HoodieMetadataPayload.SECONDARY_INDEX_RECORD_KEY_SEPARATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -184,7 +186,7 @@ public void testColumnStatsPayloadMerging() throws IOException { String targetColName = "c1"; HoodieColumnRangeMetadata c1Metadata = - HoodieColumnRangeMetadata.create(fileName, targetColName, 100, 1000, 5, 1000, 123456, 123456); + HoodieColumnRangeMetadata.create(fileName, targetColName, 100, 1000, 5, 1000, 123456, 123456, ValueMetadata.V1EmptyMetadata.get()); HoodieRecord columnStatsRecord = HoodieMetadataPayload.createColumnStatsRecords(PARTITION_NAME, Collections.singletonList(c1Metadata), false) @@ -197,7 +199,7 @@ public void testColumnStatsPayloadMerging() throws IOException { // NOTE: Column Stats record will only be merged in case existing file will be modified, // which could only happen on storages schemes supporting appends HoodieColumnRangeMetadata c1AppendedBlockMetadata = - HoodieColumnRangeMetadata.create(fileName, targetColName, 0, 500, 0, 100, 12345, 12345); + HoodieColumnRangeMetadata.create(fileName, targetColName, 0, 500, 0, 100, 12345, 12345, ValueMetadata.V1EmptyMetadata.get()); HoodieRecord updatedColumnStatsRecord = HoodieMetadataPayload.createColumnStatsRecords(PARTITION_NAME, Collections.singletonList(c1AppendedBlockMetadata), false) @@ -207,7 +209,7 @@ public void testColumnStatsPayloadMerging() throws IOException { columnStatsRecord.getData().preCombine(updatedColumnStatsRecord.getData()); HoodieColumnRangeMetadata expectedColumnRangeMetadata = - HoodieColumnRangeMetadata.create(fileName, targetColName, 0, 1000, 5, 1100, 135801, 135801); + HoodieColumnRangeMetadata.create(fileName, targetColName, 0, 1000, 5, 1100, 135801, 135801, ValueMetadata.V1EmptyMetadata.get()); HoodieRecord expectedColumnStatsRecord = HoodieMetadataPayload.createColumnStatsRecords(PARTITION_NAME, Collections.singletonList(expectedColumnRangeMetadata), false) @@ -227,7 +229,7 @@ public void testColumnStatsPayloadMerging() throws IOException { //////////////////////////////////////////////////////////////////////// HoodieColumnRangeMetadata c1StubbedMetadata = - HoodieColumnRangeMetadata.stub(fileName, targetColName); + HoodieColumnRangeMetadata.stub(fileName, targetColName, V1); HoodieRecord deletedColumnStatsRecord = HoodieMetadataPayload.createColumnStatsRecords(PARTITION_NAME, Collections.singletonList(c1StubbedMetadata), true) @@ -253,17 +255,17 @@ public void testColumnStatsPayloadMerging() throws IOException { @Test public void testPartitionStatsPayloadMerging() { HoodieColumnRangeMetadata fileColumnRange1 = HoodieColumnRangeMetadata.create( - "path/to/file", "columnName", 1, 5, 0, 10, 100, 200); + "path/to/file", "columnName", 1, 5, 0, 10, 100, 200, ValueMetadata.V1EmptyMetadata.get()); HoodieRecord firstPartitionStatsRecord = HoodieMetadataPayload.createPartitionStatsRecords(PARTITION_NAME, Collections.singletonList(fileColumnRange1), false, false, Option.empty()).findFirst().get(); HoodieColumnRangeMetadata fileColumnRange2 = HoodieColumnRangeMetadata.create( - "path/to/file", "columnName", 3, 8, 1, 15, 120, 250); + "path/to/file", "columnName", 3, 8, 1, 15, 120, 250, ValueMetadata.V1EmptyMetadata.get()); HoodieRecord updatedPartitionStatsRecord = HoodieMetadataPayload.createPartitionStatsRecords(PARTITION_NAME, Collections.singletonList(fileColumnRange2), false, false, Option.empty()).findFirst().get(); HoodieMetadataPayload combinedPartitionStatsRecordPayload = updatedPartitionStatsRecord.getData().preCombine(firstPartitionStatsRecord.getData()); HoodieColumnRangeMetadata expectedColumnRange = HoodieColumnRangeMetadata.create( - "path/to/file", "columnName", 1, 8, 1, 25, 220, 450); + "path/to/file", "columnName", 1, 8, 1, 25, 220, 450, ValueMetadata.V1EmptyMetadata.get()); HoodieMetadataPayload expectedColumnRangeMetadata = (HoodieMetadataPayload) HoodieMetadataPayload.createPartitionStatsRecords( PARTITION_NAME, Collections.singletonList(expectedColumnRange), false, false, Option.empty()).findFirst().get().getData(); assertEquals(expectedColumnRangeMetadata, combinedPartitionStatsRecordPayload); @@ -272,11 +274,11 @@ public void testPartitionStatsPayloadMerging() { @Test public void testPartitionStatsPayloadMergingWithDelete() { HoodieColumnRangeMetadata fileColumnRange1 = HoodieColumnRangeMetadata.create( - "path/to/file", "columnName", 1, 5, 0, 10, 100, 200); + "path/to/file", "columnName", 1, 5, 0, 10, 100, 200, ValueMetadata.V1EmptyMetadata.get()); HoodieRecord firstPartitionStatsRecord = HoodieMetadataPayload.createPartitionStatsRecords(PARTITION_NAME, Collections.singletonList(fileColumnRange1), false, false, Option.empty()).findFirst().get(); HoodieColumnRangeMetadata fileColumnRange2 = HoodieColumnRangeMetadata.create( - "path/to/file", "columnName", 3, 8, 1, 15, 120, 250); + "path/to/file", "columnName", 3, 8, 1, 15, 120, 250, ValueMetadata.V1EmptyMetadata.get()); // create delete payload HoodieRecord deletedPartitionStatsRecord = HoodieMetadataPayload.createPartitionStatsRecords(PARTITION_NAME, Collections.singletonList(fileColumnRange2), true, false, Option.empty()).findFirst().get(); @@ -284,7 +286,7 @@ public void testPartitionStatsPayloadMergingWithDelete() { HoodieMetadataPayload combinedPartitionStatsRecordPayload = deletedPartitionStatsRecord.getData().preCombine(firstPartitionStatsRecord.getData()); HoodieColumnRangeMetadata expectedColumnRange = HoodieColumnRangeMetadata.create( - "path/to/file", "columnName", 3, 8, 1, 15, 120, 250); + "path/to/file", "columnName", 3, 8, 1, 15, 120, 250, ValueMetadata.V1EmptyMetadata.get()); HoodieMetadataPayload expectedColumnRangeMetadata = (HoodieMetadataPayload) HoodieMetadataPayload.createPartitionStatsRecords( PARTITION_NAME, Collections.singletonList(expectedColumnRange), true, false, Option.empty()).findFirst().get().getData(); assertEquals(expectedColumnRangeMetadata, combinedPartitionStatsRecordPayload); diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/metadata/TestHoodieTableMetadataUtil.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/metadata/TestHoodieTableMetadataUtil.java index af34673ad5165..889eae6d32c07 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/metadata/TestHoodieTableMetadataUtil.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/metadata/TestHoodieTableMetadataUtil.java @@ -26,13 +26,13 @@ import org.apache.hudi.common.function.SerializableBiFunction; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.testutils.FileCreateUtilsLegacy; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; @@ -42,6 +42,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.io.storage.HoodieFileWriter; import org.apache.hudi.io.storage.HoodieFileWriterFactory; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.util.Lazy; @@ -381,7 +382,7 @@ public void testGetColumnsToIndex() { List expected = new ArrayList<>(Arrays.asList(HoodieTableMetadataUtil.META_COLS_TO_ALWAYS_INDEX)); addNColumns(expected, HoodieMetadataConfig.COLUMN_STATS_INDEX_MAX_COLUMNS.defaultValue()); assertListEquality(expected, new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(tableConfig, metadataConfig, - Lazy.eagerly(Option.of(getTableSchema(expected))), false).keySet())); + Lazy.eagerly(Option.of(getTableSchema(expected))), false, V1).keySet())); //test with table schema < default int tableSchemaSize = HoodieMetadataConfig.COLUMN_STATS_INDEX_MAX_COLUMNS.defaultValue() - 10; @@ -391,7 +392,7 @@ public void testGetColumnsToIndex() { expected = new ArrayList<>(Arrays.asList(HoodieTableMetadataUtil.META_COLS_TO_ALWAYS_INDEX)); addNColumns(expected, tableSchemaSize); assertListEquality(expected, new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(tableConfig, metadataConfig, - Lazy.eagerly(Option.of(getTableSchema(expected))), false).keySet())); + Lazy.eagerly(Option.of(getTableSchema(expected))), false, V1).keySet())); //test with max val < tableSchema metadataConfig = HoodieMetadataConfig.newBuilder() @@ -403,7 +404,7 @@ public void testGetColumnsToIndex() { expected = new ArrayList<>(Arrays.asList(HoodieTableMetadataUtil.META_COLS_TO_ALWAYS_INDEX)); addNColumns(expected, 3); assertListEquality(expected, new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(tableConfig, metadataConfig, - Lazy.eagerly(Option.of(getTableSchema(expected))), false).keySet())); + Lazy.eagerly(Option.of(getTableSchema(expected))), false, V1).keySet())); //test with max val > tableSchema metadataConfig = HoodieMetadataConfig.newBuilder() @@ -415,7 +416,7 @@ public void testGetColumnsToIndex() { expected = new ArrayList<>(Arrays.asList(HoodieTableMetadataUtil.META_COLS_TO_ALWAYS_INDEX)); addNColumns(expected, tableSchemaSize); assertListEquality(expected, new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(tableConfig, metadataConfig, - Lazy.eagerly(Option.of(getTableSchema(expected))), false).keySet())); + Lazy.eagerly(Option.of(getTableSchema(expected))), false, V1).keySet())); //test with list of cols and a nested field as well. metadataConfig = HoodieMetadataConfig.newBuilder() @@ -443,7 +444,7 @@ public void testGetColumnsToIndex() { .endRecord(); assertListEquality(expected, new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(tableConfig, metadataConfig, - Lazy.eagerly(Option.of(schema)), false).keySet())); + Lazy.eagerly(Option.of(schema)), false, V1).keySet())); //test with list of cols longer than config metadataConfig = HoodieMetadataConfig.newBuilder() @@ -458,7 +459,7 @@ public void testGetColumnsToIndex() { expected.add("col_7"); expected.add("col_11"); assertListEquality(expected, new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(tableConfig, metadataConfig, - Lazy.eagerly(Option.of(getTableSchema(expected))), false).keySet())); + Lazy.eagerly(Option.of(getTableSchema(expected))), false, V1).keySet())); //test with list of cols including meta cols than config metadataConfig = HoodieMetadataConfig.newBuilder() @@ -473,7 +474,7 @@ public void testGetColumnsToIndex() { expected.add("col_11"); expected.add("_hoodie_commit_seqno"); assertListEquality(expected, new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(tableConfig, metadataConfig, - Lazy.eagerly(Option.of(getTableSchema(expected))), false).keySet())); + Lazy.eagerly(Option.of(getTableSchema(expected))), false, V1).keySet())); //test with avro schema schema = new Schema.Parser().parse(SCHEMA_WITH_AVRO_TYPES_STR); @@ -486,7 +487,7 @@ public void testGetColumnsToIndex() { expected.add("decimalField"); expected.add("localTimestampMillisField"); assertListEquality(expected, new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(tableConfig, metadataConfig, - Lazy.eagerly(Option.of(schema)), true).keySet())); + Lazy.eagerly(Option.of(schema)), true, V1).keySet())); //test with avro schema and nested fields and unsupported types schema = new Schema.Parser().parse(SCHEMA_WITH_NESTED_FIELD_STR); @@ -498,7 +499,7 @@ public void testGetColumnsToIndex() { expected.add("firstname"); expected.add("student.lastnameNested"); assertListEquality(expected, new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(tableConfig, metadataConfig, - Lazy.eagerly(Option.of(schema)), false).keySet())); + Lazy.eagerly(Option.of(schema)), false, V1).keySet())); //test with avro schema with max cols set schema = new Schema.Parser().parse(SCHEMA_WITH_AVRO_TYPES_STR); @@ -510,10 +511,10 @@ public void testGetColumnsToIndex() { expected.add("booleanField"); expected.add("intField"); assertListEquality(expected, new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(tableConfig, metadataConfig, - Lazy.eagerly(Option.of(schema)), false).keySet())); + Lazy.eagerly(Option.of(schema)), false, V1).keySet())); //test with avro schema with meta cols assertListEquality(expected, new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(tableConfig, metadataConfig, - Lazy.eagerly(Option.of(HoodieAvroUtils.addMetadataFields(schema))), false).keySet())); + Lazy.eagerly(Option.of(HoodieAvroUtils.addMetadataFields(schema))), false, V1).keySet())); //test with avro schema with type filter metadataConfig = HoodieMetadataConfig.newBuilder() @@ -537,11 +538,11 @@ public void testGetColumnsToIndex() { expected.add("current_ts"); expected.add("_hoodie_is_deleted"); assertListEquality(expected, new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(tableConfig, metadataConfig, - Lazy.eagerly(Option.of(HoodieTestDataGenerator.AVRO_SCHEMA)), false).keySet())); + Lazy.eagerly(Option.of(HoodieTestDataGenerator.AVRO_SCHEMA)), false, V1).keySet())); //test with avro schema with meta cols assertListEquality(expected, new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(tableConfig, metadataConfig, - Lazy.eagerly(Option.of(HoodieAvroUtils.addMetadataFields(HoodieTestDataGenerator.AVRO_SCHEMA))), false).keySet())); + Lazy.eagerly(Option.of(HoodieAvroUtils.addMetadataFields(HoodieTestDataGenerator.AVRO_SCHEMA))), false, V1).keySet())); //test with meta cols disabled tableConfig.setValue(HoodieTableConfig.POPULATE_META_FIELDS.key(), "false"); @@ -553,7 +554,7 @@ public void testGetColumnsToIndex() { expected = new ArrayList<>(); addNColumns(expected, tableSchemaSize); assertListEquality(expected, new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(tableConfig, metadataConfig, - Lazy.eagerly(Option.of(getTableSchema(expected))), false).keySet())); + Lazy.eagerly(Option.of(getTableSchema(expected))), false, V1).keySet())); //test with meta cols disabled with col list metadataConfig = HoodieMetadataConfig.newBuilder() @@ -567,7 +568,7 @@ public void testGetColumnsToIndex() { expected.add("col_7"); expected.add("col_11"); assertListEquality(expected, new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(tableConfig, metadataConfig, - Lazy.eagerly(Option.of(getTableSchema(expected))), false).keySet())); + Lazy.eagerly(Option.of(getTableSchema(expected))), false, V1).keySet())); //test with meta cols disabled with avro schema metadataConfig = HoodieMetadataConfig.newBuilder() @@ -579,7 +580,7 @@ public void testGetColumnsToIndex() { expected.add("decimalField"); expected.add("localTimestampMillisField"); assertListEquality(expected, new ArrayList<>(HoodieTableMetadataUtil.getColumnsToIndex(tableConfig, metadataConfig, - Lazy.eagerly(Option.of(schema)), true).keySet())); + Lazy.eagerly(Option.of(schema)), true, V1).keySet())); } private void assertListEquality(List expected, List actual) { @@ -623,19 +624,19 @@ public void testValidateDataTypeForPartitionStats() { .endRecord(); // Test for primitive fields - assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("stringField").schema(), Option.empty())); - assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("intField").schema(), Option.empty())); - assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("booleanField").schema(), Option.empty())); - assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("floatField").schema(), Option.empty())); - assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("doubleField").schema(), Option.empty())); - assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("longField").schema(), Option.empty())); - assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("unionIntField").schema(), Option.empty())); + assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("stringField").schema(), Option.empty(), V1)); + assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("intField").schema(), Option.empty(), V1)); + assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("booleanField").schema(), Option.empty(), V1)); + assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("floatField").schema(), Option.empty(), V1)); + assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("doubleField").schema(), Option.empty(), V1)); + assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("longField").schema(), Option.empty(), V1)); + assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("unionIntField").schema(), Option.empty(), V1)); // Test for unsupported fields - assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("arrayField").schema(), Option.empty())); - assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("mapField").schema(), Option.empty())); - assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("structField").schema(), Option.empty())); - assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("bytesField").schema(), Option.of(HoodieRecord.HoodieRecordType.SPARK))); + assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("arrayField").schema(), Option.empty(), V1)); + assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("mapField").schema(), Option.empty(), V1)); + assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("structField").schema(), Option.empty(), V1)); + assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("bytesField").schema(), Option.of(HoodieRecord.HoodieRecordType.SPARK), V1)); // Test for logical types Schema dateFieldSchema = LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)); @@ -643,28 +644,28 @@ public void testValidateDataTypeForPartitionStats() { .fields() .name("dateField").type(dateFieldSchema).noDefault() .endRecord(); - assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("dateField").schema(), Option.empty())); + assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema.getField("dateField").schema(), Option.empty(), V1)); // Test for logical decimal type with allowed precision and scale schema = Schema.create(Schema.Type.BYTES); LogicalTypes.Decimal decimalType = LogicalTypes.decimal(30, 15); decimalType.addToSchema(schema); // Expect the column to be supported. - assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema, Option.of(HoodieRecord.HoodieRecordType.AVRO))); + assertTrue(HoodieTableMetadataUtil.isColumnTypeSupported(schema, Option.of(HoodieRecord.HoodieRecordType.AVRO), V1)); // Test for logical decimal type with precision and scale exceeding the limit schema = Schema.create(Schema.Type.BYTES); decimalType = LogicalTypes.decimal(35, 20); decimalType.addToSchema(schema); // Expect the column to be unsupported. - assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema, Option.of(HoodieRecord.HoodieRecordType.AVRO))); + assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema, Option.of(HoodieRecord.HoodieRecordType.AVRO), V1)); // Test for logical decimal type with precision exceeding limit after upscaling schema = Schema.create(Schema.Type.BYTES); decimalType = LogicalTypes.decimal(28, 10); decimalType.addToSchema(schema); // Expect the column to be unsupported. - assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema, Option.of(HoodieRecord.HoodieRecordType.AVRO))); + assertFalse(HoodieTableMetadataUtil.isColumnTypeSupported(schema, Option.of(HoodieRecord.HoodieRecordType.AVRO), V1)); } @Test @@ -719,6 +720,7 @@ public void testGetExpressionIndexPartitionsToInit() { HoodieTableConfig tableConfig = mock(HoodieTableConfig.class); when(metaClient.getTableConfig()).thenReturn(tableConfig); when(tableConfig.getMetadataPartitions()).thenReturn(new HashSet<>(Collections.singleton("expr_index_idx_ts"))); + when(tableConfig.getTableVersion()).thenReturn(HoodieTableVersion.current()); // Build metadata config HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().enable(true) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/SchemaEvolutionContext.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/SchemaEvolutionContext.java index a0e2054fb3ff4..5b4b67b0cc512 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/SchemaEvolutionContext.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/SchemaEvolutionContext.java @@ -319,13 +319,19 @@ private TypeInfo constructHiveSchemaFromType(Type type, TypeInfo typeInfo) { case DOUBLE: case DATE: case TIMESTAMP: + case TIMESTAMP_MILLIS: + case LOCAL_TIMESTAMP_MICROS: + case LOCAL_TIMESTAMP_MILLIS: case STRING: case UUID: case FIXED: case BINARY: case DECIMAL: + case DECIMAL_BYTES: + case DECIMAL_FIXED: return typeInfo; case TIME: + case TIME_MILLIS: throw new UnsupportedOperationException(String.format("cannot convert %s type to hive", type)); default: LOG.error("cannot convert unknown type: {} to Hive", type); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetReader.java index 6ef52f61ec332..6bc7c21ccc02e 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetReader.java @@ -34,8 +34,8 @@ import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.internal.schema.action.InternalSchemaMerger; import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; + import org.apache.parquet.avro.AvroReadSupport; -import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.format.converter.ParquetMetadataConverter; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.ParquetInputSplit; @@ -47,6 +47,7 @@ import java.util.Arrays; import java.util.List; +import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter; public class HoodieAvroParquetReader extends RecordReader { @@ -59,7 +60,7 @@ public HoodieAvroParquetReader(InputSplit inputSplit, Configuration conf, Option ParquetMetadata fileFooter = ParquetFileReader.readFooter(conf, ((ParquetInputSplit) inputSplit).getPath(), ParquetMetadataConverter.NO_FILTER); MessageType messageType = fileFooter.getFileMetaData().getSchema(); - baseSchema = new AvroSchemaConverter(conf).convert(messageType); + baseSchema = getAvroSchemaConverter(conf).convert(messageType); if (internalSchemaOption.isPresent()) { // do schema reconciliation in case there exists read column which is not in the file schema. diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala index 158b88175eb63..8fe2de9520f12 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala @@ -30,8 +30,10 @@ import org.apache.hudi.common.util.BinaryUtil.toBytes import org.apache.hudi.common.util.ValidationUtils.checkState import org.apache.hudi.common.util.collection import org.apache.hudi.data.HoodieJavaRDD -import org.apache.hudi.metadata.{ColumnStatsIndexPrefixRawKey, HoodieMetadataPayload, HoodieTableMetadata, HoodieTableMetadataUtil, MetadataPartitionType} +import org.apache.hudi.metadata.{ColumnStatsIndexPrefixRawKey, HoodieIndexVersion, HoodieMetadataPayload, HoodieTableMetadata, HoodieTableMetadataUtil, MetadataPartitionType} import org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS +import org.apache.hudi.stats.{SparkValueMetadataUtils, ValueMetadata, ValueType} +import org.apache.hudi.stats.ValueMetadata.getValueMetadata import org.apache.hudi.util.JFunction import org.apache.avro.Conversions.DecimalConversion @@ -242,6 +244,7 @@ class ColumnStatsIndexSupport(spark: SparkSession, // of the dataset, and therefore we rely on low-level RDD API to avoid incurring encoding/decoding // penalty of the [[Dataset]], since it's required to adhere to its schema at all times, while // RDDs are not; + val useJava8api = spark.sessionState.conf.datetimeJava8ApiEnabled val transposedRows: HoodieData[Row] = colStatsRecords //TODO: [HUDI-8303] Explicit conversion might not be required for Scala 2.12+ .filter(JFunction.toJavaSerializableFunction(r => sortedTargetColumnsSet.contains(r.getColumnName))) @@ -260,8 +263,9 @@ class ColumnStatsIndexSupport(spark: SparkSession, val colName = r.getColumnName val colType = sortedTargetColDataTypeMap(colName).dataType - val minValue = deserialize(tryUnpackValueWrapper(minValueWrapper), colType) - val maxValue = deserialize(tryUnpackValueWrapper(maxValueWrapper), colType) + val valueMetadata = getValueMetadata(r.getValueType) + val minValue = extractColStatsValue(minValueWrapper, colType, valueMetadata, useJava8api) + val maxValue = extractColStatsValue(maxValueWrapper, colType, valueMetadata, useJava8api) // Update min-/max-value structs w/ unwrapped values in-place r.setMinValue(minValue) @@ -443,6 +447,28 @@ object ColumnStatsIndexSupport { @inline def composeColumnStatStructType(col: String, statName: String, dataType: DataType) = StructField(formatColName(col, statName), dataType, nullable = true, Metadata.empty) + def extractColStatsValue(valueWrapper: AnyRef, dataType: DataType, valueMetadata: ValueMetadata, useJava8api: Boolean): Any = { + valueMetadata.getValueType match { + case ValueType.V1 => extractWrapperValueV1(valueWrapper, dataType) + case _ => extractColStatsValueV2(valueWrapper, dataType, valueMetadata, useJava8api) + } + } + + private def extractColStatsValueV2(valueWrapper: AnyRef, dataType: DataType, valueMetadata: ValueMetadata, useJava8api: Boolean): Any = { + val colStatsValue = SparkValueMetadataUtils.convertJavaTypeToSparkType(SparkValueMetadataUtils.getValueMetadata(dataType, HoodieIndexVersion.V2) + .standardizeJavaTypeAndPromote(valueMetadata.unwrapValue(valueWrapper)), useJava8api) + // TODO: should this be done here? Should we handle this with adding more value types? + // TODO: should this logic be in convertJavaTypeToSparkType? + dataType match { + case ShortType => colStatsValue.asInstanceOf[Int].toShort + case ByteType => colStatsValue.asInstanceOf[Int].toByte + case _ => colStatsValue + } + } + + def extractWrapperValueV1(valueWrapper: AnyRef, dataType: DataType): Any = + deserialize(tryUnpackValueWrapper(valueWrapper), dataType) + def tryUnpackValueWrapper(valueWrapper: AnyRef): Any = { valueWrapper match { case w: BooleanWrapper => w.getValue diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ExpressionIndexSupport.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ExpressionIndexSupport.scala index 7beb7bba680c3..3007607f5e9c9 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ExpressionIndexSupport.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ExpressionIndexSupport.scala @@ -19,7 +19,7 @@ package org.apache.hudi -import org.apache.hudi.ColumnStatsIndexSupport.{composeColumnStatStructType, deserialize, tryUnpackValueWrapper} +import org.apache.hudi.ColumnStatsIndexSupport.{composeColumnStatStructType, extractWrapperValueV1} import org.apache.hudi.ExpressionIndexSupport._ import org.apache.hudi.HoodieCatalystUtils.{withPersistedData, withPersistedDataset} import org.apache.hudi.HoodieConversionUtils.{toJavaOption, toScalaOption} @@ -34,6 +34,8 @@ import org.apache.hudi.common.util.ValidationUtils.checkState import org.apache.hudi.data.HoodieJavaRDD import org.apache.hudi.index.expression.HoodieExpressionIndex import org.apache.hudi.metadata.{ColumnStatsIndexPrefixRawKey, HoodieMetadataPayload, HoodieTableMetadataUtil, MetadataPartitionType} +import org.apache.hudi.stats.{SparkValueMetadataUtils, ValueMetadata, ValueType} +import org.apache.hudi.stats.ValueMetadata.getValueMetadata import org.apache.hudi.util.JFunction import org.apache.spark.sql.{DataFrame, Row, SparkSession} @@ -241,6 +243,7 @@ class ExpressionIndexSupport(spark: SparkSession, // of the dataset, and therefore we rely on low-level RDD API to avoid incurring encoding/decoding // penalty of the [[Dataset]], since it's required to adhere to its schema at all times, while // RDDs are not; + val useJava8api = spark.sessionState.conf.datetimeJava8ApiEnabled val transposedRows: HoodieData[Row] = colStatsRecords //TODO: [HUDI-8303] Explicit conversion might not be required for Scala 2.12+ .filter(JFunction.toJavaSerializableFunction(r => sortedTargetColumnsSet.contains(r.getColumnName))) @@ -259,8 +262,9 @@ class ExpressionIndexSupport(spark: SparkSession, val colName = r.getColumnName val colType = tableSchemaFieldMap(colName).dataType - val minValue = deserialize(tryUnpackValueWrapper(minValueWrapper), colType) - val maxValue = deserialize(tryUnpackValueWrapper(maxValueWrapper), colType) + val valueMetadata = getValueMetadata(r.getValueType) + val minValue = extractExpressionIndexValue(minValueWrapper, colType, valueMetadata, useJava8api) + val maxValue = extractExpressionIndexValue(maxValueWrapper, colType, valueMetadata, useJava8api) // Update min-/max-value structs w/ unwrapped values in-place r.setMinValue(minValue) @@ -611,6 +615,18 @@ class ExpressionIndexSupport(spark: SparkSession, } object ExpressionIndexSupport { + + def extractExpressionIndexValue(valueWrapper: AnyRef, dataType: DataType, valueMetadata: ValueMetadata, useJava8api: Boolean): Any = { + valueMetadata.getValueType match { + case ValueType.V1 => extractWrapperValueV1(valueWrapper, dataType) + case _ => extractExpressionIndexValueV2(valueWrapper, valueMetadata, useJava8api) + } + } + + private def extractExpressionIndexValueV2(valueWrapper: AnyRef, valueMetadata: ValueMetadata, useJava8api: Boolean): Any = { + SparkValueMetadataUtils.convertJavaTypeToSparkType(valueMetadata.unwrapValue(valueWrapper), useJava8api) + } + val INDEX_NAME = "EXPRESSION" /** * Target Column Stats Index columns which internally are mapped onto fields of the corresponding diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala index 9430fcd08ce3d..de54189bd78b1 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.avro import org.apache.avro.{LogicalTypes, Schema, SchemaBuilder} -import org.apache.avro.LogicalTypes.{Date, Decimal, TimestampMicros, TimestampMillis} +import org.apache.avro.LogicalTypes.{Date, Decimal, LocalTimestampMicros, LocalTimestampMillis, TimestampMicros, TimestampMillis} import org.apache.avro.Schema.Type._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.types._ @@ -77,6 +77,7 @@ private[sql] object SchemaConverters { case FLOAT => SchemaType(FloatType, nullable = false) case LONG => avroSchema.getLogicalType match { case _: TimestampMillis | _: TimestampMicros => SchemaType(TimestampType, nullable = false) + case _: LocalTimestampMillis | _: LocalTimestampMicros => SchemaType(TimestampNTZType, nullable = false) case _ => SchemaType(LongType, nullable = false) } @@ -165,6 +166,8 @@ private[sql] object SchemaConverters { LogicalTypes.date().addToSchema(builder.intType()) case TimestampType => LogicalTypes.timestampMicros().addToSchema(builder.longType()) + case TimestampNTZType => + LogicalTypes.localTimestampMicros().addToSchema(builder.longType()) case FloatType => builder.floatType() case DoubleType => builder.doubleType() diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala index 98f640dc191ac..6179dc9317081 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala @@ -35,6 +35,7 @@ import org.apache.hudi.io.IOUtils import org.apache.hudi.storage.StorageConfiguration import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration +import org.apache.avro.Schema import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce.Job @@ -84,6 +85,8 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, hoodieFileFormat: HoodieFileFormat) extends ParquetFileFormat with SparkAdapterSupport with HoodieFormatTrait with Logging with Serializable { + private lazy val avroTableSchema = new Schema.Parser().parse(tableSchema.avroSchemaStr) + override def shortName(): String = "HudiFileGroup" override def toString: String = "HoodieFileGroupReaderBasedFileFormat" @@ -192,9 +195,12 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, val fixedPartitionIndexes = fixedPartitionIndexesArr.toSet // schema that we want fg reader to output to us + val exclusionFields = new java.util.HashSet[String]() + exclusionFields.add("op") + partitionSchema.fields.foreach(f => exclusionFields.add(f.name)) val requestedSchema = StructType(requiredSchema.fields ++ partitionSchema.fields.filter(f => mandatoryFields.contains(f.name))) - val requestedAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(requestedSchema, sanitizedTableName) - val dataAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(dataSchema, sanitizedTableName) + val requestedAvroSchema = AvroSchemaUtils.pruneDataSchema(avroTableSchema, AvroConversionUtils.convertStructTypeToAvroSchema(requestedSchema, sanitizedTableName), exclusionFields) + val dataAvroSchema = AvroSchemaUtils.pruneDataSchema(avroTableSchema, AvroConversionUtils.convertStructTypeToAvroSchema(dataSchema, sanitizedTableName), exclusionFields) val baseFileReader = spark.sparkContext.broadcast(buildBaseFileReader(spark, options, augmentedStorageConf.unwrap(), dataSchema, supportVectorizedRead)) val fileGroupBaseFileReader = if (isMOR && supportVectorizedRead) { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReaderBase.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReaderBase.scala index 219dfccf83c20..5fb471f2aec52 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReaderBase.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReaderBase.scala @@ -74,7 +74,7 @@ abstract class SparkParquetReaderBase(enableVectorizedReader: Boolean, conf.setBoolean(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, false) if (HoodieSparkUtils.gteqSpark3_4) { // PARQUET_INFER_TIMESTAMP_NTZ_ENABLED is required from Spark 3.4.0 or above - conf.setBoolean("spark.sql.parquet.inferTimestampNTZ.enabled", false) + conf.setBooleanIfUnset("spark.sql.parquet.inferTimestampNTZ.enabled", true) } ParquetWriteSupport.setSchema(requiredSchema, conf) diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/testutils/LogFileColStatsTestUtil.java b/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/testutils/LogFileColStatsTestUtil.java index 9272eef601229..49718d224ceaa 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/testutils/LogFileColStatsTestUtil.java +++ b/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/testutils/LogFileColStatsTestUtil.java @@ -19,12 +19,12 @@ package org.apache.hudi.testutils; import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; import org.apache.hudi.storage.StoragePath; import org.apache.avro.Schema; diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala index 246c266d4673c..1344a738c9565 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/SparkHelpers.scala @@ -31,7 +31,7 @@ import org.apache.hudi.storage.{HoodieStorage, StorageConfiguration, StoragePath import org.apache.avro.Schema import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem -import org.apache.parquet.avro.AvroSchemaConverter +import org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter import org.apache.parquet.hadoop.metadata.CompressionCodecName import org.apache.spark.sql.{DataFrame, SQLContext} @@ -55,7 +55,7 @@ object SparkHelpers { val filter: BloomFilter = BloomFilterFactory.createBloomFilter( BLOOM_FILTER_NUM_ENTRIES_VALUE.defaultValue.toInt, BLOOM_FILTER_FPP_VALUE.defaultValue.toDouble, BLOOM_FILTER_DYNAMIC_MAX_ENTRIES.defaultValue.toInt, BLOOM_FILTER_TYPE.defaultValue); - val writeSupport: HoodieAvroWriteSupport[_] = new HoodieAvroWriteSupport(new AvroSchemaConverter(conf.unwrap()).convert(schema), + val writeSupport: HoodieAvroWriteSupport[_] = new HoodieAvroWriteSupport(getAvroSchemaConverter(conf.unwrap()).convert(schema), schema, Option.of(filter), new Properties()) val parquetConfig: HoodieParquetConfig[HoodieAvroWriteSupport[_]] = new HoodieParquetConfig( diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/ColumnStatsIndexHelper.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/ColumnStatsIndexHelper.java index fe8427e7caa72..221af780ef9ac 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/ColumnStatsIndexHelper.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/ColumnStatsIndexHelper.java @@ -17,10 +17,13 @@ package org.apache.hudi; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.util.ParquetUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.metadata.HoodieIndexVersion; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.hadoop.HoodieHadoopStorage; @@ -181,7 +184,8 @@ public static Dataset buildColumnStatsTableFor( return utils.readColumnStatsFromMetadata( storage, new StoragePath(path), - columnNames + columnNames, + HoodieIndexVersion.getCurrentVersion(HoodieTableVersion.current(), MetadataPartitionType.COLUMN_STATS.getPartitionPath()) ) .stream(); } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java index 8d7c9403d445b..98793ed4c33d3 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java @@ -23,7 +23,6 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.DefaultHoodieRecordPayload; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -35,6 +34,8 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner; import org.apache.hudi.metadata.HoodieMetadataPayload; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; +import org.apache.hudi.stats.ValueMetadata; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.testutils.HoodieClientTestBase; @@ -293,7 +294,7 @@ public void testSerHoodieMetadataPayload() throws IOException { String targetColName = "c1"; HoodieColumnRangeMetadata columnStatsRecord = - HoodieColumnRangeMetadata.create(fileName, targetColName, 0, 500, 0, 100, 12345, 12345); + HoodieColumnRangeMetadata.create(fileName, targetColName, 0, 500, 0, 100, 12345, 12345, ValueMetadata.V1EmptyMetadata.get()); HoodieRecord hoodieMetadataPayload = HoodieMetadataPayload.createColumnStatsRecords(partitionPath, Collections.singletonList(columnStatsRecord), false) diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestColStatsRecordWithMetadataRecord.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestColStatsRecordWithMetadataRecord.java index 43bae063ba0ac..2fb2205c4d4cd 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestColStatsRecordWithMetadataRecord.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestColStatsRecordWithMetadataRecord.java @@ -27,7 +27,6 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.EngineProperty; import org.apache.hudi.common.engine.TaskContextSupplier; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieWriteStat; @@ -46,6 +45,8 @@ import org.apache.hudi.io.storage.HoodieSeekingFileReader; import org.apache.hudi.metadata.HoodieMetadataPayload; import org.apache.hudi.metadata.HoodieMetadataWriteUtils; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; +import org.apache.hudi.stats.ValueMetadata; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; @@ -81,6 +82,7 @@ import static org.apache.hudi.common.util.ConfigUtils.DEFAULT_HUDI_CONFIG_FOR_READER; import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; +import static org.apache.hudi.metadata.HoodieIndexVersion.V1; import static org.apache.hudi.metadata.MetadataPartitionType.COLUMN_STATS; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -169,7 +171,7 @@ public void testColsStatsSerDe() throws Exception { AtomicInteger finalCounter1 = counter; minMaxValues.forEach(entry -> { columnRangeMetadata.add(HoodieColumnRangeMetadata.create(fileName, targetColNamePrefix + "_" + (finalCounter1.getAndIncrement()), - entry.getKey(), entry.getValue(), 5, 1000, 123456, 123456)); + entry.getKey(), entry.getValue(), 5, 1000, 123456, 123456, ValueMetadata.V1EmptyMetadata.get())); }); // create mdt records @@ -320,24 +322,34 @@ public void testGetColumnRangeInPartition() { long totalUncompressedSize = 1000; // Integer vals - HoodieColumnRangeMetadata aIntegerVal = HoodieColumnRangeMetadata.create(fileName, colName, (Integer)1, (Integer)1000, nullCount, valueCount, totalSize, totalUncompressedSize); - HoodieColumnRangeMetadata bIntegerVal = HoodieColumnRangeMetadata.create(fileName, colName, (Integer)(-1), (Integer)10000, nullCount, valueCount, totalSize, totalUncompressedSize); + HoodieColumnRangeMetadata aIntegerVal = HoodieColumnRangeMetadata.create(fileName, colName, (Integer)1, (Integer)1000, nullCount, valueCount, totalSize, totalUncompressedSize, + ValueMetadata.V1EmptyMetadata.get()); + HoodieColumnRangeMetadata bIntegerVal = HoodieColumnRangeMetadata.create(fileName, colName, (Integer)(-1), (Integer)10000, nullCount, valueCount, totalSize, totalUncompressedSize, + ValueMetadata.V1EmptyMetadata.get()); // Long vals - HoodieColumnRangeMetadata aLongVal = HoodieColumnRangeMetadata.create(fileName, colName, (Long)1L, (Long)1000L, nullCount, valueCount, totalSize, totalUncompressedSize); - HoodieColumnRangeMetadata bLongVal = HoodieColumnRangeMetadata.create(fileName, colName, (Long)(-1L), (Long)10000L, nullCount, valueCount, totalSize, totalUncompressedSize); + HoodieColumnRangeMetadata aLongVal = HoodieColumnRangeMetadata.create(fileName, colName, (Long)1L, (Long)1000L, nullCount, valueCount, totalSize, totalUncompressedSize, + ValueMetadata.V1EmptyMetadata.get()); + HoodieColumnRangeMetadata bLongVal = HoodieColumnRangeMetadata.create(fileName, colName, (Long)(-1L), (Long)10000L, nullCount, valueCount, totalSize, totalUncompressedSize, + ValueMetadata.V1EmptyMetadata.get()); // Float vals - HoodieColumnRangeMetadata aFloatVal = HoodieColumnRangeMetadata.create(fileName, colName, new Float(1), new Float(1000.0), nullCount, valueCount, totalSize, totalUncompressedSize); - HoodieColumnRangeMetadata bFloatVal = HoodieColumnRangeMetadata.create(fileName, colName, new Float(-1.0), new Float(10000.0), nullCount, valueCount, totalSize, totalUncompressedSize); + HoodieColumnRangeMetadata aFloatVal = HoodieColumnRangeMetadata.create(fileName, colName, new Float(1), new Float(1000.0), nullCount, valueCount, totalSize, totalUncompressedSize, + ValueMetadata.V1EmptyMetadata.get()); + HoodieColumnRangeMetadata bFloatVal = HoodieColumnRangeMetadata.create(fileName, colName, new Float(-1.0), new Float(10000.0), nullCount, valueCount, totalSize, totalUncompressedSize, + ValueMetadata.V1EmptyMetadata.get()); // Double vals - HoodieColumnRangeMetadata aDoubleVal = HoodieColumnRangeMetadata.create(fileName, colName, new Double(0.1), new Double(1000.0), nullCount, valueCount, totalSize, totalUncompressedSize); - HoodieColumnRangeMetadata bDoubleVal = HoodieColumnRangeMetadata.create(fileName, colName, new Double(-1.0), new Double(10000.0), nullCount, valueCount, totalSize, totalUncompressedSize); + HoodieColumnRangeMetadata aDoubleVal = HoodieColumnRangeMetadata.create(fileName, colName, new Double(0.1), new Double(1000.0), nullCount, valueCount, totalSize, totalUncompressedSize, + ValueMetadata.V1EmptyMetadata.get()); + HoodieColumnRangeMetadata bDoubleVal = HoodieColumnRangeMetadata.create(fileName, colName, new Double(-1.0), new Double(10000.0), nullCount, valueCount, totalSize, totalUncompressedSize, + ValueMetadata.V1EmptyMetadata.get()); // String vals - HoodieColumnRangeMetadata aStringVal = HoodieColumnRangeMetadata.create(fileName, colName, new String("1"), new String("1000"), nullCount, valueCount, totalSize, totalUncompressedSize); - HoodieColumnRangeMetadata bStringVal = HoodieColumnRangeMetadata.create(fileName, colName, new String("-1"), new String("10000"), nullCount, valueCount, totalSize, totalUncompressedSize); + HoodieColumnRangeMetadata aStringVal = HoodieColumnRangeMetadata.create(fileName, colName, new String("1"), new String("1000"), nullCount, valueCount, totalSize, totalUncompressedSize, + ValueMetadata.V1EmptyMetadata.get()); + HoodieColumnRangeMetadata bStringVal = HoodieColumnRangeMetadata.create(fileName, colName, new String("-1"), new String("10000"), nullCount, valueCount, totalSize, totalUncompressedSize, + ValueMetadata.V1EmptyMetadata.get()); // Merging Integer and Integer. HoodieColumnRangeMetadata actualColumnRange = mergeAndAssert(aIntegerVal, bIntegerVal, relativePartitionPath, colName, nullCount, totalSize, totalUncompressedSize, @@ -439,7 +451,7 @@ private HoodieColumnRangeMetadata mergeAndAssert(HoodieColumnRangeMetadata colsToIndexSchemaMap = new HashMap<>(); colsToIndexSchemaMap.put(colName, Schema.create(schemaType)); - HoodieColumnRangeMetadata actualColumnRange = FileFormatUtils.getColumnRangeInPartition(relativePartitionPath, fileColumnRanges, colsToIndexSchemaMap); + HoodieColumnRangeMetadata actualColumnRange = FileFormatUtils.getColumnRangeInPartition(relativePartitionPath, colName, fileColumnRanges, colsToIndexSchemaMap, V1); validateColumnRangeMetadata(actualColumnRange, relativePartitionPath, colName, nullCount, totalSize, totalUncompressedSize); return actualColumnRange; @@ -473,7 +485,7 @@ private void generateNColStatsEntriesAndValidateMerge(Functions.Function1> columnRangeMetadata = new ArrayList<>(); minMaxValues.forEach(entry -> { columnRangeMetadata.add(HoodieColumnRangeMetadata.create(fileName, colName, - entry.getKey(), entry.getValue(), 5, 1000, 123456, 123456)); + entry.getKey(), entry.getValue(), 5, 1000, 123456, 123456, ValueMetadata.V1EmptyMetadata.get())); }); HoodieColumnRangeMetadata mergedColStatsRangeMetadata = (HoodieColumnRangeMetadata) columnRangeMetadata.stream() diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieBackedMetadata.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieBackedMetadata.java index 43fde892946c3..1be6c3d1218fc 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieBackedMetadata.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieBackedMetadata.java @@ -43,7 +43,6 @@ import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCleaningPolicy; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; @@ -109,6 +108,7 @@ import org.apache.hudi.metadata.RawKey; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.metrics.Metrics; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StorageConfiguration; import org.apache.hudi.storage.StoragePath; @@ -188,6 +188,7 @@ import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.config.HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS; import static org.apache.hudi.io.storage.HoodieSparkIOFactory.getHoodieSparkIOFactory; +import static org.apache.hudi.metadata.HoodieIndexVersion.V1; import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP; import static org.apache.hudi.metadata.HoodieTableMetadata.getMetadataTableBasePath; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.deleteMetadataTable; @@ -3223,7 +3224,7 @@ public void testColStatsMultipleColumns() throws Exception { Map> parquetStatsMap = HoodieIOFactory.getIOFactory(metaClient.getStorage()) .getFileFormatUtils(HoodieFileFormat.PARQUET) - .readColumnStatsFromMetadata(metaClient.getStorage(), fullFilePath, columns) + .readColumnStatsFromMetadata(metaClient.getStorage(), fullFilePath, columns, V1) .stream() .collect(Collectors.toMap(HoodieColumnRangeMetadata::getColumnName, Function.identity())); Map columnStatsMap = stats.stream().collect(Collectors.toMap(HoodieMetadataColumnStats::getColumnName, Function.identity())); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkSortAndSizeClustering.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkSortAndSizeClustering.java index 19bf008002807..5c0f18fab7ee9 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkSortAndSizeClustering.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkSortAndSizeClustering.java @@ -41,6 +41,7 @@ import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; +import org.apache.hudi.stats.ValueType; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.cluster.ClusteringPlanPartitionFilterMode; @@ -188,9 +189,9 @@ private void validateDateAndTimestampFields(List rows, long ts) { } if (!HoodieSparkUtils.gteqSpark4_0()) { if (!row.isNullAt(tsLocalMillisFieldIndex)) { - assertEquals(ts, row.get(tsLocalMillisFieldIndex)); + assertEquals(ValueType.toLocalTimestampMillis(ts, null), row.get(tsLocalMillisFieldIndex)); } - assertEquals(ts * 1000L, row.get(tsLocalMicrosFieldIndex)); + assertEquals(ValueType.toLocalTimestampMicros(Math.multiplyExact(ts, 1000L), null), row.get(tsLocalMicrosFieldIndex)); } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/schema_with_logical_types.avsc b/hudi-spark-datasource/hudi-spark/src/test/resources/schema_with_logical_types.avsc index bf8459d7ad8fa..7ca7b761d2bb7 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/resources/schema_with_logical_types.avsc +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/schema_with_logical_types.avsc @@ -30,12 +30,11 @@ }, { "name": "ts", - "type": ["long", "null"] + "type": ["null", "long"] }, { "name": "_hoodie_is_deleted", - "type": ["boolean", "null"], - "default" : false + "type": ["null", "boolean"] }, { "name": "double_field", diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/ColumnStatIndexTestBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/ColumnStatIndexTestBase.scala index 96c67d3bb82ad..80cb2f8f8a50d 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/ColumnStatIndexTestBase.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/ColumnStatIndexTestBase.scala @@ -25,7 +25,7 @@ import org.apache.hudi.avro.model.DecimalWrapper import org.apache.hudi.client.common.HoodieSparkEngineContext import org.apache.hudi.common.config.{HoodieMetadataConfig, HoodieReaderConfig, HoodieStorageConfig} import org.apache.hudi.common.model.{HoodieBaseFile, HoodieFileGroup, HoodieLogFile, HoodieTableType} -import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} +import org.apache.hudi.common.table.{HoodieTableMetaClient, HoodieTableVersion, TableSchemaResolver} import org.apache.hudi.common.table.view.FileSystemViewManager import org.apache.hudi.config.{HoodieCompactionConfig, HoodieWriteConfig} import org.apache.hudi.functional.ColumnStatIndexTestBase.{ColumnStatsTestCase, ColumnStatsTestParams} @@ -125,6 +125,10 @@ class ColumnStatIndexTestBase extends HoodieSparkClientTestBase { dfList = dfList :+ inputDF metaClient = HoodieTableMetaClient.builder().setBasePath(basePath).setConf(storageConf).build() + if (writeOptions.contains(HoodieWriteConfig.WRITE_TABLE_VERSION.key())) { + assertEquals(HoodieTableVersion.fromVersionCode(writeOptions(HoodieWriteConfig.WRITE_TABLE_VERSION.key()).toInt), + metaClient.getTableConfig.getTableVersion) + } if (params.shouldValidateColStats) { // Currently, routine manually validating the column stats (by actually reading every column of every file) @@ -334,16 +338,16 @@ class ColumnStatIndexTestBase extends HoodieSparkClientTestBase { val metadataConfig = HoodieMetadataConfig.newBuilder() .fromProperties(toProperties(metadataOpts)) .build() - - val pStatsIndex = new PartitionStatsIndexSupport(spark, sourceTableSchema, metadataConfig, metaClient) - val schemaUtil = new TableSchemaResolver(metaClient) val tableSchema = schemaUtil.getTableAvroSchema(false) + val localSourceTableSchema = AvroConversionUtils.convertAvroSchemaToStructType(tableSchema) + + val pStatsIndex = new PartitionStatsIndexSupport(spark, localSourceTableSchema, metadataConfig, metaClient) val indexedColumnswithMeta: Set[String] = metaClient.getIndexMetadata.get().getIndexDefinitions.get(PARTITION_NAME_COLUMN_STATS).getSourceFields.asScala.toSet val pIndexedColumns = indexedColumnswithMeta.filter(colName => !HoodieTableMetadataUtil.META_COL_SET_TO_INDEX.contains(colName)) .toSeq.sorted - val (pExpectedColStatsSchema, _) = composeIndexSchema(pIndexedColumns, pIndexedColumns, sourceTableSchema) + val (pExpectedColStatsSchema, _) = composeIndexSchema(pIndexedColumns, pIndexedColumns, localSourceTableSchema) val pValidationSortColumns = if (pIndexedColumns.contains("c5")) { Seq("c1_maxValue", "c1_minValue", "c2_maxValue", "c2_minValue", "c3_maxValue", "c3_minValue", "c5_maxValue", "c5_minValue") @@ -351,7 +355,7 @@ class ColumnStatIndexTestBase extends HoodieSparkClientTestBase { Seq("c1_maxValue", "c1_minValue", "c2_maxValue", "c2_minValue", "c3_maxValue", "c3_minValue") } - pStatsIndex.loadTransposed(sourceTableSchema.fieldNames, testCase.shouldReadInMemory) { pTransposedColStatsDF => + pStatsIndex.loadTransposed(localSourceTableSchema.fieldNames, testCase.shouldReadInMemory) { pTransposedColStatsDF => // Match against expected column stats table val pExpectedColStatsIndexTableDf = { spark.read @@ -443,52 +447,128 @@ object ColumnStatIndexTestBase { case class ColumnStatsTestCase(tableType: HoodieTableType, shouldReadInMemory: Boolean, tableVersion: Int) + // General providers (both in-memory and on-disk variants) def testMetadataColumnStatsIndexParams: java.util.stream.Stream[Arguments] = { - java.util.stream.Stream.of(HoodieTableType.values().toStream.flatMap(tableType => - Seq(Arguments.arguments(ColumnStatsTestCase(tableType, shouldReadInMemory = true, tableVersion = 6)), - Arguments.arguments(ColumnStatsTestCase(tableType, shouldReadInMemory = false, tableVersion = 6)), - Arguments.arguments(ColumnStatsTestCase(tableType, shouldReadInMemory = true, tableVersion = 8)), - Arguments.arguments(ColumnStatsTestCase(tableType, shouldReadInMemory = false, tableVersion = 8)) - ) - ): _*) + testMetadataColumnStatsIndexParams(true) } + def testMetadataPartitionStatsIndexParams: java.util.stream.Stream[Arguments] = { + testMetadataColumnStatsIndexParams(false) + } + + def testMetadataColumnStatsIndexParams(testV6: Boolean): java.util.stream.Stream[Arguments] = { + val currentVersionCode = HoodieTableVersion.current().versionCode() + java.util.stream.Stream.of( + HoodieTableType.values().toStream.flatMap { tableType => + val v6Seq = if (testV6) { + Seq( + Arguments.arguments(ColumnStatsTestCase(tableType, shouldReadInMemory = true, tableVersion = 6)), + Arguments.arguments(ColumnStatsTestCase(tableType, shouldReadInMemory = false, tableVersion = 6)) + ) + } else { + Seq.empty + } + + v6Seq ++ Seq( + Arguments.arguments(ColumnStatsTestCase(tableType, shouldReadInMemory = true, tableVersion = 8)), + Arguments.arguments(ColumnStatsTestCase(tableType, shouldReadInMemory = false, tableVersion = 8)), + Arguments.arguments(ColumnStatsTestCase(tableType, shouldReadInMemory = true, tableVersion = currentVersionCode)), + Arguments.arguments(ColumnStatsTestCase(tableType, shouldReadInMemory = false, tableVersion = currentVersionCode)) + ) + }: _* + ) + } + + // In-memory providers (only shouldReadInMemory = true) def testMetadataColumnStatsIndexParamsInMemory: java.util.stream.Stream[Arguments] = { - java.util.stream.Stream.of(HoodieTableType.values().toStream.flatMap(tableType => - Seq(Arguments.arguments(ColumnStatsTestCase(tableType, shouldReadInMemory = true, tableVersion = 6)), - Arguments.arguments(ColumnStatsTestCase(tableType, shouldReadInMemory = true, tableVersion = 8)) - ) - ): _*) + testMetadataColumnStatsIndexParamsInMemory(true) + } + + def testMetadataPartitionStatsIndexParamsInMemory: java.util.stream.Stream[Arguments] = { + testMetadataColumnStatsIndexParamsInMemory(false) + } + + def testMetadataColumnStatsIndexParamsInMemory(testV6: Boolean): java.util.stream.Stream[Arguments] = { + val currentVersionCode = HoodieTableVersion.current().versionCode() + java.util.stream.Stream.of( + HoodieTableType.values().toStream.flatMap { tableType => + val v6Seq = if (testV6) { + Seq(Arguments.arguments(ColumnStatsTestCase(tableType, shouldReadInMemory = true, tableVersion = 6))) + } else { + Seq.empty + } + + v6Seq ++ Seq( + Arguments.arguments(ColumnStatsTestCase(tableType, shouldReadInMemory = true, tableVersion = 8)), + Arguments.arguments(ColumnStatsTestCase(tableType, shouldReadInMemory = true, tableVersion = currentVersionCode)) + ) + }: _* + ) } + + // MOR-only providers def testMetadataColumnStatsIndexParamsForMOR: java.util.stream.Stream[Arguments] = { + testMetadataColumnStatsIndexParamsForMOR(true) + } + + def testMetadataPartitionStatsIndexParamsForMOR: java.util.stream.Stream[Arguments] = { + testMetadataColumnStatsIndexParamsForMOR(false) + } + + def testMetadataColumnStatsIndexParamsForMOR(testV6: Boolean): java.util.stream.Stream[Arguments] = { + val currentVersionCode = HoodieTableVersion.current().versionCode() java.util.stream.Stream.of( - Seq(Arguments.arguments(ColumnStatsTestCase(HoodieTableType.MERGE_ON_READ, shouldReadInMemory = true, tableVersion = 6)), - Arguments.arguments(ColumnStatsTestCase(HoodieTableType.MERGE_ON_READ, shouldReadInMemory = false, tableVersion = 6)), + (if (testV6) Seq( + Arguments.arguments(ColumnStatsTestCase(HoodieTableType.MERGE_ON_READ, shouldReadInMemory = true, tableVersion = 6)), + Arguments.arguments(ColumnStatsTestCase(HoodieTableType.MERGE_ON_READ, shouldReadInMemory = false, tableVersion = 6)) + ) else Seq.empty) ++ Seq( Arguments.arguments(ColumnStatsTestCase(HoodieTableType.MERGE_ON_READ, shouldReadInMemory = true, tableVersion = 8)), - Arguments.arguments(ColumnStatsTestCase(HoodieTableType.MERGE_ON_READ, shouldReadInMemory = false, tableVersion = 8)) - ) - : _*) + Arguments.arguments(ColumnStatsTestCase(HoodieTableType.MERGE_ON_READ, shouldReadInMemory = false, tableVersion = 8)), + Arguments.arguments(ColumnStatsTestCase(HoodieTableType.MERGE_ON_READ, shouldReadInMemory = true, tableVersion = currentVersionCode)), + Arguments.arguments(ColumnStatsTestCase(HoodieTableType.MERGE_ON_READ, shouldReadInMemory = false, tableVersion = currentVersionCode)) + ): _* + ) } + + // TableType / partition column providers (tableVersion encoded as String like your original) def testTableTypePartitionTypeParams: java.util.stream.Stream[Arguments] = { - java.util.stream.Stream.of( + testTableTypePartitionTypeParams(true) + } + + def testTableTypePartitionTypeParamsNoV6: java.util.stream.Stream[Arguments] = { + testTableTypePartitionTypeParams(false) + } + + def testTableTypePartitionTypeParams(testV6: Boolean): java.util.stream.Stream[Arguments] = { + val currentVersionCode = HoodieTableVersion.current().versionCode().toString + val v6Seq = if (testV6) { Seq( - // Table version 6 Arguments.arguments(HoodieTableType.COPY_ON_WRITE, "c8", "6"), - // empty partition col represents non-partitioned table. Arguments.arguments(HoodieTableType.COPY_ON_WRITE, "", "6"), Arguments.arguments(HoodieTableType.MERGE_ON_READ, "c8", "6"), - Arguments.arguments(HoodieTableType.MERGE_ON_READ, "", "6"), + Arguments.arguments(HoodieTableType.MERGE_ON_READ, "", "6") + ) + } else { + Seq.empty + } + java.util.stream.Stream.of( + (v6Seq ++ Seq( // Table version 8 Arguments.arguments(HoodieTableType.COPY_ON_WRITE, "c8", "8"), - // empty partition col represents non-partitioned table. Arguments.arguments(HoodieTableType.COPY_ON_WRITE, "", "8"), Arguments.arguments(HoodieTableType.MERGE_ON_READ, "c8", "8"), - Arguments.arguments(HoodieTableType.MERGE_ON_READ, "", "8") - ) - : _*) + Arguments.arguments(HoodieTableType.MERGE_ON_READ, "", "8"), + + // Table version current + Arguments.arguments(HoodieTableType.COPY_ON_WRITE, "c8", currentVersionCode), + Arguments.arguments(HoodieTableType.COPY_ON_WRITE, "", currentVersionCode), + Arguments.arguments(HoodieTableType.MERGE_ON_READ, "c8", currentVersionCode), + Arguments.arguments(HoodieTableType.MERGE_ON_READ, "", currentVersionCode) + )): _* + ) } trait WrapperCreator { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala index db80c93dff6bd..018bfcefe9c97 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala @@ -35,6 +35,7 @@ import org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERA import org.apache.hudi.common.util.{ParquetUtils, StringUtils} import org.apache.hudi.config.{HoodieCleanConfig, HoodieCompactionConfig, HoodieWriteConfig} import org.apache.hudi.functional.ColumnStatIndexTestBase.{ColumnStatsTestCase, ColumnStatsTestParams, WrapperCreator} +import org.apache.hudi.metadata.HoodieIndexVersion import org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS import org.apache.hudi.storage.StoragePath import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration @@ -50,7 +51,7 @@ import org.apache.spark.sql.types._ import org.junit.jupiter.api._ import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertTrue} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.{CsvSource, MethodSource} +import org.junit.jupiter.params.provider.{Arguments, CsvSource, MethodSource} import java.math.{BigDecimal => JBigDecimal} import java.nio.ByteBuffer @@ -504,7 +505,7 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase { } @ParameterizedTest - @CsvSource(value = Array("6", "8")) + @MethodSource(Array("testMORDeleteBlocksParams")) def testMORDeleteBlocks(tableVersion: Int): Unit = { val tableType: HoodieTableType = HoodieTableType.MERGE_ON_READ val partitionCol = "c8" @@ -1073,7 +1074,7 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase { val parquetFilePath = new StoragePath( fs.listStatus(path).filter(fs => fs.getPath.getName.endsWith(".parquet")).toSeq.head.getPath.toUri) - val ranges = utils.readColumnStatsFromMetadata(storage, parquetFilePath, Seq("c1", "c2", "c3a", "c3b", "c3c", "c4", "c5", "c6", "c7", "c8").asJava) + val ranges = utils.readColumnStatsFromMetadata(storage, parquetFilePath, Seq("c1", "c2", "c3a", "c3b", "c3c", "c4", "c5", "c6", "c7", "c8").asJava, HoodieIndexVersion.V1) ranges.asScala.foreach(r => { // NOTE: Unfortunately Parquet can't compute statistics for Timestamp column, hence we @@ -1113,3 +1114,15 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase { s"Decimal value from $description does not match") } } + +object TestColumnStatsIndex { + def testMORDeleteBlocksParams: java.util.stream.Stream[Arguments] = { + val currentVersionCode = HoodieTableVersion.current().versionCode().toString + java.util.stream.Stream.of(Seq( + Arguments.arguments("6"), + Arguments.arguments("8"), + Arguments.arguments(currentVersionCode) + ) + : _*) + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala index 2990e980db45d..207cb9018eb2d 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala @@ -22,7 +22,6 @@ import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions} import org.apache.hudi.avro.HoodieAvroWrapperUtils.unwrapAvroValueWrapper import org.apache.hudi.client.common.HoodieSparkEngineContext import org.apache.hudi.common.config.HoodieMetadataConfig -import org.apache.hudi.common.model.HoodieColumnRangeMetadata import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.common.table.timeline.HoodieTimeline import org.apache.hudi.common.table.view.FileSystemViewManager @@ -31,7 +30,8 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator.recordsToStrings import org.apache.hudi.common.util.ParquetUtils import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.hadoop.fs.HadoopFSUtils -import org.apache.hudi.metadata.{HoodieBackedTableMetadata, HoodieTableMetadata} +import org.apache.hudi.metadata.{HoodieBackedTableMetadata, HoodieIndexVersion, HoodieTableMetadata} +import org.apache.hudi.stats.HoodieColumnRangeMetadata import org.apache.hudi.storage.StoragePath import org.apache.hudi.storage.hadoop.HoodieHadoopStorage import org.apache.hudi.testutils.SparkClientFunctionalTestHarness @@ -161,7 +161,7 @@ class TestMetadataTableWithSparkDataSource extends SparkClientFunctionalTestHarn val colRangeMetadataList: java.util.List[HoodieColumnRangeMetadata[Comparable[_]]] = new ParquetUtils() .readColumnStatsFromMetadata( new HoodieHadoopStorage(fileStatuses.get(0).getPath, HadoopFSUtils.getStorageConf(jsc().hadoopConfiguration())), - fileStatuses.get(0).getPath, Collections.singletonList("begin_lat")) + fileStatuses.get(0).getPath, Collections.singletonList("begin_lat"), HoodieIndexVersion.V1) val columnRangeMetadata = colRangeMetadataList.get(0) assertEquals(metadataColStats.getValueCount, columnRangeMetadata.getValueCount) @@ -219,7 +219,7 @@ class TestMetadataTableWithSparkDataSource extends SparkClientFunctionalTestHarn val colRangeMetadataList: java.util.List[HoodieColumnRangeMetadata[Comparable[_]]] = new ParquetUtils() .readColumnStatsFromMetadata( new HoodieHadoopStorage(fileStatuses.get(0).getPath, HadoopFSUtils.getStorageConf(jsc().hadoopConfiguration())), - fileStatuses.get(0).getPath, Collections.singletonList("begin_lat")) + fileStatuses.get(0).getPath, Collections.singletonList("begin_lat"), HoodieIndexVersion.V1) val columnRangeMetadata = colRangeMetadataList.get(0) assertEquals(metadataColStats.getValueCount, columnRangeMetadata.getValueCount) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsPruning.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsPruning.scala index fd8d4ddc1035c..2e07b702592da 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsPruning.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsPruning.scala @@ -43,9 +43,8 @@ class TestPartitionStatsPruning extends ColumnStatIndexTestBase { HoodieRecord.PARTITION_PATH_METADATA_FIELD, "c1","c2","c3","c4","c5","c6","c7","c8") @ParameterizedTest - @MethodSource(Array("testMetadataColumnStatsIndexParamsInMemory")) + @MethodSource(Array("testMetadataPartitionStatsIndexParamsInMemory")) def testMetadataPSISimple(testCase: ColumnStatsTestCase): Unit = { - val metadataOpts = Map( HoodieMetadataConfig.ENABLE.key -> "true", HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true", @@ -57,6 +56,7 @@ class TestPartitionStatsPruning extends ColumnStatIndexTestBase { "hoodie.upsert.shuffle.parallelism" -> "4", HoodieWriteConfig.TBL_NAME.key -> "hoodie_test", DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString, + HoodieWriteConfig.WRITE_TABLE_VERSION.key() -> testCase.tableVersion.toString, RECORDKEY_FIELD.key -> "c1", HoodieTableConfig.ORDERING_FIELDS.key -> "c1", PARTITIONPATH_FIELD.key() -> "c8", @@ -73,7 +73,7 @@ class TestPartitionStatsPruning extends ColumnStatIndexTestBase { } @ParameterizedTest - @MethodSource(Array("testMetadataColumnStatsIndexParamsInMemory")) + @MethodSource(Array("testMetadataPartitionStatsIndexParamsInMemory")) def testMetadataColumnStatsIndex(testCase: ColumnStatsTestCase): Unit = { val metadataOpts = Map( HoodieMetadataConfig.ENABLE.key -> "true", @@ -86,6 +86,7 @@ class TestPartitionStatsPruning extends ColumnStatIndexTestBase { "hoodie.upsert.shuffle.parallelism" -> "4", HoodieWriteConfig.TBL_NAME.key -> "hoodie_test", DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString, + HoodieWriteConfig.WRITE_TABLE_VERSION.key() -> testCase.tableVersion.toString, RECORDKEY_FIELD.key -> "c1", HoodieTableConfig.ORDERING_FIELDS.key -> "c1", PARTITIONPATH_FIELD.key() -> "c8", @@ -180,7 +181,7 @@ class TestPartitionStatsPruning extends ColumnStatIndexTestBase { } @ParameterizedTest - @MethodSource(Array("testMetadataColumnStatsIndexParamsInMemory")) + @MethodSource(Array("testMetadataPartitionStatsIndexParamsInMemory")) def testMetadataColumnStatsIndexInitializationWithUpserts(testCase: ColumnStatsTestCase): Unit = { val partitionCol : String = "c8" val metadataOpts = Map( @@ -194,6 +195,7 @@ class TestPartitionStatsPruning extends ColumnStatIndexTestBase { "hoodie.upsert.shuffle.parallelism" -> "1", HoodieWriteConfig.TBL_NAME.key -> "hoodie_test", DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString, + HoodieWriteConfig.WRITE_TABLE_VERSION.key() -> testCase.tableVersion.toString, RECORDKEY_FIELD.key -> "c1", HoodieTableConfig.ORDERING_FIELDS.key -> "c1", PARTITIONPATH_FIELD.key -> partitionCol, @@ -292,7 +294,7 @@ class TestPartitionStatsPruning extends ColumnStatIndexTestBase { } @ParameterizedTest - @MethodSource(Array("testMetadataColumnStatsIndexParamsInMemory")) + @MethodSource(Array("testMetadataPartitionStatsIndexParamsInMemory")) def testMetadataColumnStatsIndexInitializationWithRollbacks(testCase: ColumnStatsTestCase): Unit = { val partitionCol : String ="c8" val metadataOpts = Map( @@ -305,6 +307,7 @@ class TestPartitionStatsPruning extends ColumnStatIndexTestBase { "hoodie.upsert.shuffle.parallelism" -> "1", HoodieWriteConfig.TBL_NAME.key -> "hoodie_test", DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString, + HoodieWriteConfig.WRITE_TABLE_VERSION.key() -> testCase.tableVersion.toString, RECORDKEY_FIELD.key -> "c1", HoodieTableConfig.ORDERING_FIELDS.key -> "c1", PARTITIONPATH_FIELD.key() -> partitionCol, diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/TestExpressionIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/TestExpressionIndex.scala index 193e44086e82e..48b41c646ace4 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/TestExpressionIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/feature/index/TestExpressionIndex.scala @@ -22,6 +22,7 @@ package org.apache.spark.sql.hudi.feature.index import org.apache.hudi.{DataSourceReadOptions, ExpressionIndexSupport, HoodieFileIndex, HoodieSparkUtils, SparkAdapterSupport} import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.HoodieConversionUtils.toProperties +import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.avro.model.HoodieMetadataBloomFilter import org.apache.hudi.client.SparkRDDWriteClient import org.apache.hudi.client.common.HoodieSparkEngineContext @@ -40,6 +41,7 @@ import org.apache.hudi.index.HoodieIndex import org.apache.hudi.index.expression.HoodieExpressionIndex import org.apache.hudi.metadata.{HoodieBackedTableMetadata, HoodieIndexVersion, HoodieMetadataPayload, MetadataPartitionType} import org.apache.hudi.metadata.HoodieTableMetadataUtil.getPartitionStatsIndexKey +import org.apache.hudi.stats.{SparkValueMetadataUtils, ValueType} import org.apache.hudi.storage.StoragePath import org.apache.hudi.sync.common.HoodieSyncConfig.{META_SYNC_BASE_PATH, META_SYNC_DATABASE_NAME, META_SYNC_NO_PARTITION_METADATA, META_SYNC_TABLE_NAME} import org.apache.hudi.testutils.HoodieClientTestUtils.createMetaClient @@ -1172,7 +1174,7 @@ class TestExpressionIndex extends HoodieSparkSqlTestBase with SparkAdapterSuppor spark.sql(s"create index idx_to_date on $tableName using column_stats(date) options(expr='to_date', format='yyyy-MM-dd')") metaClient = HoodieTableMetaClient.reload(metaClient) val toDate = resolveExpr(spark, sparkAdapter.getExpressionFromColumn(functions.to_date(functions.col("date"), "yyyy-MM-dd")), tableSchema) - dataFilter = EqualTo(toDate, sparkAdapter.getExpressionFromColumn(lit(18230))) + dataFilter = EqualTo(toDate, sparkAdapter.getExpressionFromColumn(lit(HoodieAvroUtils.toJavaDate(18230)))) verifyPartitionPruning(opts, Seq(), Seq(dataFilter), metaClient, isDataSkippingExpected = true) spark.sql(s"drop index idx_to_date on $tableName") } @@ -1532,14 +1534,14 @@ class TestExpressionIndex extends HoodieSparkSqlTestBase with SparkAdapterSuppor spark.sql(s"create index idx_to_date on $tableName using column_stats(date) options(expr='to_date', format='yyyy-MM-dd')") metaClient = HoodieTableMetaClient.reload(metaClient) val toDate = resolveExpr(spark, sparkAdapter.getExpressionFromColumn(functions.to_date(functions.col("date"), "yyyy-MM-dd")), tableSchema) - dataFilter = EqualTo(toDate, sparkAdapter.getExpressionFromColumn(lit(18596))) + dataFilter = EqualTo(toDate, sparkAdapter.getExpressionFromColumn(lit(HoodieAvroUtils.toJavaDate(18596)))) verifyFilePruning(opts, dataFilter, metaClient, isDataSkippingExpected = true) spark.sql(s"drop index idx_to_date on $tableName") spark.sql(s"create index idx_to_date_default on $tableName using column_stats(date) options(expr='to_date')") metaClient = HoodieTableMetaClient.reload(metaClient) val toDateDefault = resolveExpr(spark, sparkAdapter.getExpressionFromColumn(functions.to_date(functions.col("date"))), tableSchema) - dataFilter = EqualTo(toDateDefault, sparkAdapter.getExpressionFromColumn(lit(18596))) + dataFilter = EqualTo(toDateDefault, sparkAdapter.getExpressionFromColumn(lit(HoodieAvroUtils.toJavaDate(18596)))) verifyFilePruning(opts, dataFilter, metaClient, isDataSkippingExpected = true) spark.sql(s"drop index idx_to_date_default on $tableName") @@ -1553,28 +1555,28 @@ class TestExpressionIndex extends HoodieSparkSqlTestBase with SparkAdapterSuppor spark.sql(s"create index idx_to_timestamp_default on $tableName using column_stats(date) options(expr='to_timestamp')") metaClient = HoodieTableMetaClient.reload(metaClient) val toTimestampDefault = resolveExpr(spark, sparkAdapter.getExpressionFromColumn(functions.to_timestamp(functions.col("date"))), tableSchema) - dataFilter = EqualTo(toTimestampDefault, sparkAdapter.getExpressionFromColumn(lit(1732924800000000L))) + dataFilter = EqualTo(toTimestampDefault, sparkAdapter.getExpressionFromColumn(lit(SparkValueMetadataUtils.convertJavaTypeToSparkType(ValueType.toTimestampMicros(1732924800000000L, null), false)))) verifyFilePruning(opts, dataFilter, metaClient, isDataSkippingExpected = true) spark.sql(s"drop index idx_to_timestamp_default on $tableName") spark.sql(s"create index idx_to_timestamp on $tableName using column_stats(date) options(expr='to_timestamp', format='yyyy-MM-dd')") metaClient = HoodieTableMetaClient.reload(metaClient) val toTimestamp = resolveExpr(spark, sparkAdapter.getExpressionFromColumn(functions.to_timestamp(functions.col("date"), "yyyy-MM-dd")), tableSchema) - dataFilter = EqualTo(toTimestamp, sparkAdapter.getExpressionFromColumn(lit(1732924800000000L))) + dataFilter = EqualTo(toTimestamp, sparkAdapter.getExpressionFromColumn(lit(SparkValueMetadataUtils.convertJavaTypeToSparkType(ValueType.toTimestampMicros(1732924800000000L, null), false)))) verifyFilePruning(opts, dataFilter, metaClient, isDataSkippingExpected = true) spark.sql(s"drop index idx_to_timestamp on $tableName") spark.sql(s"create index idx_date_add on $tableName using column_stats(date) options(expr='date_add', days='10')") metaClient = HoodieTableMetaClient.reload(metaClient) val dateAdd = resolveExpr(spark, sparkAdapter.getExpressionFromColumn(functions.date_add(functions.col("date"), 10)), tableSchema) - dataFilter = EqualTo(dateAdd, sparkAdapter.getExpressionFromColumn(lit(18606))) + dataFilter = EqualTo(dateAdd, sparkAdapter.getExpressionFromColumn(lit(HoodieAvroUtils.toJavaDate(18606)))) verifyFilePruning(opts, dataFilter, metaClient, isDataSkippingExpected = true) spark.sql(s"drop index idx_date_add on $tableName") spark.sql(s"create index idx_date_sub on $tableName using column_stats(date) options(expr='date_sub', days='10')") metaClient = HoodieTableMetaClient.reload(metaClient) val dateSub = resolveExpr(spark, sparkAdapter.getExpressionFromColumn(functions.date_sub(functions.col("date"), 10)), tableSchema) - dataFilter = EqualTo(dateSub, sparkAdapter.getExpressionFromColumn(lit(18586))) + dataFilter = EqualTo(dateSub, sparkAdapter.getExpressionFromColumn(lit(HoodieAvroUtils.toJavaDate(18586)))) verifyFilePruning(opts, dataFilter, metaClient, isDataSkippingExpected = true) spark.sql(s"drop index idx_date_sub on $tableName") } diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala index 33f8ed4153c67..a135aba228088 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala @@ -164,5 +164,7 @@ class Spark3_3Adapter extends BaseSpark3Adapter { value == LegacyBehaviorPolicy.LEGACY } - override def isTimestampNTZType(dataType: DataType): Boolean = false + override def isTimestampNTZType(dataType: DataType): Boolean = { + dataType.getClass.getSimpleName.startsWith("TimestampNTZType") + } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java index 9bb9449b11c76..b9f3fd5e0004f 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java @@ -41,12 +41,11 @@ import org.apache.hudi.common.model.BaseFile; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieFileGroup; -import org.apache.hudi.common.model.HoodieIndexDefinition; import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieIndexDefinition; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; @@ -69,6 +68,7 @@ import org.apache.hudi.common.util.ConfigUtils; import org.apache.hudi.common.util.FileFormatUtils; import org.apache.hudi.common.util.FileIOUtils; +import org.apache.hudi.common.util.HoodieDataUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; @@ -84,13 +84,15 @@ import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieIOFactory; import org.apache.hudi.metadata.HoodieBackedTableMetadata; +import org.apache.hudi.metadata.HoodieIndexVersion; import org.apache.hudi.metadata.HoodieMetadataPayload; import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.utilities.util.BloomFilterData; -import org.apache.hudi.common.util.HoodieDataUtils; import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; @@ -143,6 +145,7 @@ import static org.apache.hudi.io.storage.HoodieSparkIOFactory.getHoodieSparkIOFactory; import static org.apache.hudi.metadata.HoodieTableMetadata.getMetadataTableBasePath; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.META_COL_SET_TO_INDEX; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_SECONDARY_INDEX; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getLocationFromRecordIndexInfo; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getLogFileColumnRangeMetadata; @@ -1097,7 +1100,8 @@ static TreeSet> aggregateColumnStats(Strin TreeSet> aggregatedColumnStats = new TreeSet<>(Comparator.comparing(HoodieColumnRangeMetadata::getColumnName)); for (HoodieColumnRangeMetadata colStat : colStats) { HoodieColumnRangeMetadata partitionStat = HoodieColumnRangeMetadata.create(partitionPath, colStat.getColumnName(), - colStat.getMinValue(), colStat.getMaxValue(), colStat.getNullCount(), colStat.getValueCount(), colStat.getTotalSize(), colStat.getTotalUncompressedSize()); + colStat.getMinValue(), colStat.getMaxValue(), colStat.getNullCount(), colStat.getValueCount(), colStat.getTotalSize(), colStat.getTotalUncompressedSize(), + colStat.getValueMetadata()); HoodieColumnRangeMetadata storedPartitionStat = aggregatedColumnStats.floor(partitionStat); if (storedPartitionStat == null || !storedPartitionStat.getColumnName().equals(partitionStat.getColumnName())) { aggregatedColumnStats.add(partitionStat); @@ -1914,12 +1918,13 @@ public List> getSortedColumnStatsList(Stri .collect(Collectors.toList()); } else { FileFormatUtils formatUtils = HoodieIOFactory.getIOFactory(metaClient.getStorage()).getFileFormatUtils(HoodieFileFormat.PARQUET); + HoodieIndexVersion indexVersion = HoodieTableMetadataUtil.existingIndexVersionOrDefault(PARTITION_NAME_COLUMN_STATS, metaClient); return (List>) fileNames.stream().flatMap(filename -> { if (filename.endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { return formatUtils.readColumnStatsFromMetadata( metaClient.getStorage(), new StoragePath(FSUtils.constructAbsolutePath(metaClient.getBasePath(), partitionPath), filename), - allColumnNameList + allColumnNameList, indexVersion ).stream(); } else { StoragePath storagePartitionPath = new StoragePath(metaClient.getBasePath(), partitionPath); @@ -1930,7 +1935,7 @@ public List> getSortedColumnStatsList(Stri .stream() // We need to convert file path and use only the file name instead of the complete file path .map(m -> (HoodieColumnRangeMetadata) HoodieColumnRangeMetadata.create(filename, m.getColumnName(), m.getMinValue(), m.getMaxValue(), - m.getNullCount(), m.getValueCount(), m.getTotalSize(), m.getTotalUncompressedSize())); + m.getNullCount(), m.getValueCount(), m.getTotalSize(), m.getTotalUncompressedSize(), m.getValueMetadata())); } catch (IOException e) { throw new HoodieIOException(String.format("Failed to get column stats for file: %s", filePath), e); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/MercifulJsonToRowConverter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/MercifulJsonToRowConverter.java index f7a0bae0509c5..2c6e18897e68d 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/MercifulJsonToRowConverter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/MercifulJsonToRowConverter.java @@ -25,11 +25,14 @@ import org.apache.hudi.avro.processors.EnumTypeProcessor; import org.apache.hudi.avro.processors.FixedTypeProcessor; import org.apache.hudi.avro.processors.JsonFieldProcessor; +import org.apache.hudi.avro.processors.LocalTimestampMicroLogicalTypeProcessor; +import org.apache.hudi.avro.processors.LocalTimestampMilliLogicalTypeProcessor; import org.apache.hudi.avro.processors.Parser; import org.apache.hudi.avro.processors.TimestampMicroLogicalTypeProcessor; import org.apache.hudi.avro.processors.TimestampMilliLogicalTypeProcessor; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.stats.ValueType; import org.apache.hudi.utilities.exception.HoodieJsonToRowConversionException; import com.fasterxml.jackson.databind.DeserializationFeature; @@ -46,6 +49,7 @@ import java.math.BigDecimal; import java.sql.Timestamp; import java.time.Instant; +import java.time.LocalDateTime; import java.time.temporal.ChronoField; import java.util.ArrayList; import java.util.Collections; @@ -245,6 +249,16 @@ protected JsonFieldProcessor generateTimestampMicroLogicalTypeHandler() { return new TimestampMicroToRowLogicalTypeProcessor(); } + @Override + protected JsonFieldProcessor generateLocalTimeStampMicroLogicalTypeHandler() { + return new LocalTimestampMicroToRowLogicalTypeProcessor(); + } + + @Override + protected JsonFieldProcessor generateLocalTimeStampMilliLogicalTypeHandler() { + return new LocalTimestampMilliToRowLogicalTypeProcessor(); + } + private static class TimestampMicroToRowLogicalTypeProcessor extends TimestampMicroLogicalTypeProcessor { @Override public Pair convert( @@ -268,6 +282,87 @@ public Pair handleStringValue(String value) { } } + private static class LocalTimestampMicroToRowLogicalTypeProcessor extends LocalTimestampMicroLogicalTypeProcessor { + + @Override + public Pair convert( + Object value, String name, Schema schema) { + return convertCommon( + new Parser.LongParser() { + @Override + public Pair handleNumberValue(Number value) { + Pair result = super.handleNumberValue(value); + if (result.getLeft()) { + return Pair.of(true, ValueType.castToLocalTimestampMicros(result.getRight(), null)); + } + return result; + } + + @Override + public Pair handleStringNumber(String value) { + Pair result = super.handleStringNumber(value); + if (result.getLeft()) { + return Pair.of(true, ValueType.castToLocalTimestampMicros(result.getRight(), null)); + } + return result; + } + + @Override + public Pair handleStringValue(String value) { + if (!isWellFormedDateTime(value)) { + return Pair.of(false, null); + } + Pair result = convertToLocalDateTime(value); + if (!result.getLeft()) { + return Pair.of(false, null); + } + return Pair.of(true, result.getRight()); + } + }, + value, schema); + } + } + + private static class LocalTimestampMilliToRowLogicalTypeProcessor extends LocalTimestampMilliLogicalTypeProcessor { + @Override + public Pair convert( + Object value, String name, Schema schema) { + return convertCommon( + new Parser.LongParser() { + @Override + public Pair handleNumberValue(Number value) { + Pair result = super.handleNumberValue(value); + if (result.getLeft()) { + return Pair.of(true, ValueType.castToLocalTimestampMillis(result.getRight(), null)); + } + return result; + } + + @Override + public Pair handleStringNumber(String value) { + Pair result = super.handleStringNumber(value); + if (result.getLeft()) { + return Pair.of(true, ValueType.castToLocalTimestampMillis(result.getRight(), null)); + } + return result; + } + + @Override + public Pair handleStringValue(String value) { + if (!isWellFormedDateTime(value)) { + return Pair.of(false, null); + } + Pair result = convertToLocalDateTime(value); + if (!result.getLeft()) { + return Pair.of(false, null); + } + return Pair.of(true, result.getRight()); + } + }, + value, schema); + } + } + @Override protected JsonFieldProcessor generateArrayTypeHandler() { return new JsonFieldProcessor() { diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java index e4e380fd60a9a..19cd9473f1284 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java @@ -19,9 +19,9 @@ package org.apache.hudi.utilities; import org.apache.hudi.DataSourceWriteOptions; -import org.apache.hudi.client.WriteClientTestUtils; import org.apache.hudi.avro.model.HoodieInstantInfo; import org.apache.hudi.avro.model.HoodieRollbackPlan; +import org.apache.hudi.client.WriteClientTestUtils; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.HoodieTimeGeneratorConfig; @@ -29,7 +29,6 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; @@ -61,6 +60,8 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieValidationException; import org.apache.hudi.hadoop.fs.HadoopFSUtils; +import org.apache.hudi.stats.HoodieColumnRangeMetadata; +import org.apache.hudi.stats.ValueMetadata; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.StoragePathInfo; @@ -112,9 +113,9 @@ import static org.apache.hudi.common.testutils.SchemaTestUtil.getSimpleSchema; import static org.apache.hudi.common.util.StringUtils.toStringWithThreshold; import static org.apache.hudi.common.util.TestStringUtils.generateRandomString; +import static org.apache.hudi.utilities.HoodieMetadataTableValidator.computeDiffSummary; import static org.apache.spark.sql.types.DataTypes.IntegerType; import static org.apache.spark.sql.types.DataTypes.StringType; -import static org.apache.hudi.utilities.HoodieMetadataTableValidator.computeDiffSummary; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -158,13 +159,13 @@ public void testValidationWithoutDataTable() throws Exception { @Test public void testAggregateColumnStats() { HoodieColumnRangeMetadata fileColumn1Range1 = HoodieColumnRangeMetadata.create( - "path/to/file1", "col1", 1, 5, 0, 10, 100, 200); + "path/to/file1", "col1", 1, 5, 0, 10, 100, 200, ValueMetadata.V1EmptyMetadata.get()); HoodieColumnRangeMetadata fileColumn1Range2 = HoodieColumnRangeMetadata.create( - "path/to/file1", "col1", 1, 10, 5, 10, 100, 200); + "path/to/file1", "col1", 1, 10, 5, 10, 100, 200, ValueMetadata.V1EmptyMetadata.get()); HoodieColumnRangeMetadata fileColumn2Range1 = HoodieColumnRangeMetadata.create( - "path/to/file1", "col2", 3, 8, 1, 15, 120, 250); + "path/to/file1", "col2", 3, 8, 1, 15, 120, 250, ValueMetadata.V1EmptyMetadata.get()); HoodieColumnRangeMetadata fileColumn2Range2 = HoodieColumnRangeMetadata.create( - "path/to/file1", "col2", 5, 9, 4, 5, 80, 150); + "path/to/file1", "col2", 5, 9, 4, 5, 80, 150, ValueMetadata.V1EmptyMetadata.get()); List> colStats = new ArrayList<>(); colStats.add(fileColumn1Range1); colStats.add(fileColumn1Range2); @@ -1230,34 +1231,34 @@ private Pair generateRando HoodieColumnRangeMetadata intMetadata = HoodieColumnRangeMetadata.create( generateRandomString(30), generateRandomString(5), RANDOM.nextInt() % 30, RANDOM.nextInt() % 1000_000_000 + 30, - count / 3L, count, size, size / 8L); + count / 3L, count, size, size / 8L, ValueMetadata.V1EmptyMetadata.get()); return Pair.of(intMetadata, HoodieColumnRangeMetadata.create( new String(intMetadata.getFilePath()), new String(intMetadata.getColumnName()), (int) intMetadata.getMinValue(), (int) intMetadata.getMaxValue(), - count / 3L, count, size, size / 8L)); + count / 3L, count, size, size / 8L, ValueMetadata.V1EmptyMetadata.get())); case 1: HoodieColumnRangeMetadata longMetadata = HoodieColumnRangeMetadata.create( generateRandomString(30), generateRandomString(5), RANDOM.nextLong() % 30L, RANDOM.nextInt() % 1000_000_000_000_000L + 30L, - count / 3L, count, size, size / 8L); + count / 3L, count, size, size / 8L, ValueMetadata.V1EmptyMetadata.get()); return Pair.of(longMetadata, HoodieColumnRangeMetadata.create( new String(longMetadata.getFilePath()), new String(longMetadata.getColumnName()), (long) longMetadata.getMinValue(), (long) longMetadata.getMaxValue(), - count / 3L, count, size, size / 8L)); + count / 3L, count, size, size / 8L, ValueMetadata.V1EmptyMetadata.get())); default: String stringValue1 = generateRandomString(20); String stringValue2 = generateRandomString(20); HoodieColumnRangeMetadata stringMetadata = HoodieColumnRangeMetadata.create( generateRandomString(30), generateRandomString(5), stringValue1, stringValue2, - count / 3L, count, size, size / 8L); + count / 3L, count, size, size / 8L, ValueMetadata.V1EmptyMetadata.get()); return Pair.of(stringMetadata, HoodieColumnRangeMetadata.create( new String(stringMetadata.getFilePath()), new String(stringMetadata.getColumnName()), new String(stringValue1), new String(stringValue2), - count / 3L, count, size, size / 8L)); + count / 3L, count, size, size / 8L, ValueMetadata.V1EmptyMetadata.get())); } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerTestBase.java index 7edd11804c911..8a0add29b516a 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerTestBase.java @@ -196,7 +196,7 @@ protected static void prepareInitialConfigs(HoodieStorage storage, String dfsBas UtilitiesTestBase.Helpers.copyToDFS("streamer-config/source-flattened.avsc", storage, dfsBasePath + "/source-flattened.avsc"); UtilitiesTestBase.Helpers.copyToDFS("streamer-config/target.avsc", storage, dfsBasePath + "/target.avsc"); UtilitiesTestBase.Helpers.copyToDFS("streamer-config/target-flattened.avsc", storage, dfsBasePath + "/target-flattened.avsc"); - + UtilitiesTestBase.Helpers.copyToDFS("streamer-config/source-timestamp-millis.avsc", storage, dfsBasePath + "/source-timestamp-millis.avsc"); UtilitiesTestBase.Helpers.copyToDFS("streamer-config/source_short_trip_uber.avsc", storage, dfsBasePath + "/source_short_trip_uber.avsc"); UtilitiesTestBase.Helpers.copyToDFS("streamer-config/source_uber.avsc", storage, dfsBasePath + "/source_uber.avsc"); UtilitiesTestBase.Helpers.copyToDFS( diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java index 76aaa1d8592aa..41062605fa683 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java @@ -22,6 +22,7 @@ import org.apache.hudi.DataSourceReadOptions; import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.DefaultSparkRecordMerger; +import org.apache.hudi.HoodieSparkUtils; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.heartbeat.HoodieHeartbeatClient; import org.apache.hudi.client.transaction.lock.InProcessLockProvider; @@ -109,17 +110,20 @@ import org.apache.hudi.utilities.sources.SqlSource; import org.apache.hudi.utilities.sources.TestDataSource; import org.apache.hudi.utilities.sources.TestParquetDFSSourceEmptyBatch; +import org.apache.hudi.utilities.sources.helpers.TestMercifulJsonToRowConverter; import org.apache.hudi.utilities.streamer.HoodieStreamer; import org.apache.hudi.utilities.streamer.NoNewDataTerminationStrategy; import org.apache.hudi.utilities.streamer.StreamSync; import org.apache.hudi.utilities.streamer.StreamerCheckpointUtils; import org.apache.hudi.utilities.testutils.JdbcTestUtils; import org.apache.hudi.utilities.testutils.UtilitiesTestBase; +import org.apache.hudi.utilities.testutils.sources.AbstractBaseTestSource; import org.apache.hudi.utilities.testutils.sources.DistributedTestDataSource; import org.apache.hudi.utilities.transform.SqlQueryBasedTransformer; import org.apache.hudi.utilities.transform.Transformer; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -154,6 +158,8 @@ import java.io.IOException; import java.io.InputStream; +import java.net.URI; +import java.nio.file.Paths; import java.sql.Connection; import java.sql.DriverManager; import java.time.Instant; @@ -670,6 +676,256 @@ public void testSchemaEvolution(String tableType, boolean useUserProvidedSchema, defaultSchemaProviderClassName = FilebasedSchemaProvider.class.getName(); } + @Test + public void testTimestampMillis() throws Exception { + String tableBasePath = basePath + "/testTimestampMillis"; + defaultSchemaProviderClassName = FilebasedSchemaProvider.class.getName(); + // Insert data produced with Schema A, pass Schema A + HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT, Collections.singletonList(TestIdentityTransformer.class.getName()), + PROPS_FILENAME_TEST_SOURCE, false, true, false, null, HoodieTableType.MERGE_ON_READ.name()); + cfg.payloadClassName = DefaultHoodieRecordPayload.class.getName(); + cfg.recordMergeStrategyId = HoodieRecordMerger.EVENT_TIME_BASED_MERGE_STRATEGY_UUID; + cfg.recordMergeMode = RecordMergeMode.EVENT_TIME_ORDERING; + cfg.configs.add("hoodie.streamer.schemaprovider.source.schema.file=" + basePath + "/source-timestamp-millis.avsc"); + cfg.configs.add("hoodie.streamer.schemaprovider.target.schema.file=" + basePath + "/source-timestamp-millis.avsc"); + cfg.configs.add(String.format("%s=%s", HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT.key(), "0")); + cfg.configs.add("hoodie.datasource.write.row.writer.enable=false"); + + + new HoodieDeltaStreamer(cfg, jsc).sync(); + assertUseV2Checkpoint(HoodieTestUtils.createMetaClient(storage, tableBasePath)); + assertRecordCount(1000, tableBasePath, sqlContext); + TestHelpers.assertCommitMetadata("00000", tableBasePath, 1); + TableSchemaResolver tableSchemaResolver = new TableSchemaResolver( + HoodieTestUtils.createMetaClient(storage, tableBasePath)); + Schema tableSchema = tableSchemaResolver.getTableAvroSchema(false); + assertEquals("timestamp-millis", tableSchema.getField("current_ts").schema().getLogicalType().getName()); + assertEquals(1000, sqlContext.read().options(hudiOpts).format("org.apache.hudi").load(tableBasePath).filter("current_ts > '1980-01-01'").count()); + + cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, Collections.singletonList(TestIdentityTransformer.class.getName()), + PROPS_FILENAME_TEST_SOURCE, false, true, false, null, HoodieTableType.MERGE_ON_READ.name()); + cfg.payloadClassName = DefaultHoodieRecordPayload.class.getName(); + cfg.recordMergeStrategyId = HoodieRecordMerger.EVENT_TIME_BASED_MERGE_STRATEGY_UUID; + cfg.recordMergeMode = RecordMergeMode.EVENT_TIME_ORDERING; + cfg.configs.add("hoodie.streamer.schemaprovider.source.schema.file=" + basePath + "/source-timestamp-millis.avsc"); + cfg.configs.add("hoodie.streamer.schemaprovider.target.schema.file=" + basePath + "/source-timestamp-millis.avsc"); + cfg.configs.add(String.format("%s=%s", HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT.key(), "0")); + cfg.configs.add("hoodie.datasource.write.row.writer.enable=false"); + + new HoodieDeltaStreamer(cfg, jsc).sync(); + assertUseV2Checkpoint(HoodieTestUtils.createMetaClient(storage, tableBasePath)); + assertRecordCount(1450, tableBasePath, sqlContext); + TestHelpers.assertCommitMetadata("00001", tableBasePath, 2); + tableSchemaResolver = new TableSchemaResolver( + HoodieTestUtils.createMetaClient(storage, tableBasePath)); + tableSchema = tableSchemaResolver.getTableAvroSchema(false); + assertEquals("timestamp-millis", tableSchema.getField("current_ts").schema().getLogicalType().getName()); + sqlContext.clearCache(); + assertEquals(1450, sqlContext.read().options(hudiOpts).format("org.apache.hudi").load(tableBasePath).filter("current_ts > '1980-01-01'").count()); + assertEquals(1450, sqlContext.read().options(hudiOpts).format("org.apache.hudi").load(tableBasePath).filter("current_ts < '2080-01-01'").count()); + assertEquals(0, sqlContext.read().options(hudiOpts).format("org.apache.hudi").load(tableBasePath).filter("current_ts < '1980-01-01'").count()); + } + + @Test + public void testLogicalTypes() throws Exception { + try { + String tableBasePath = basePath + "/testTimestampMillis"; + defaultSchemaProviderClassName = TestHoodieDeltaStreamerSchemaEvolutionBase.TestSchemaProvider.class.getName(); + + if (HoodieSparkUtils.isSpark3_3()) { + TestHoodieDeltaStreamerSchemaEvolutionBase.TestSchemaProvider.sourceSchema = HoodieTestDataGenerator.AVRO_TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS; + TestHoodieDeltaStreamerSchemaEvolutionBase.TestSchemaProvider.targetSchema = HoodieTestDataGenerator.AVRO_TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS; + AbstractBaseTestSource.schemaStr = HoodieTestDataGenerator.TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS; + AbstractBaseTestSource.avroSchema = HoodieTestDataGenerator.AVRO_TRIP_LOGICAL_TYPES_SCHEMA_NO_LTS; + } else { + TestHoodieDeltaStreamerSchemaEvolutionBase.TestSchemaProvider.sourceSchema = HoodieTestDataGenerator.AVRO_TRIP_LOGICAL_TYPES_SCHEMA; + TestHoodieDeltaStreamerSchemaEvolutionBase.TestSchemaProvider.targetSchema = HoodieTestDataGenerator.AVRO_TRIP_LOGICAL_TYPES_SCHEMA; + AbstractBaseTestSource.schemaStr = HoodieTestDataGenerator.TRIP_LOGICAL_TYPES_SCHEMA; + AbstractBaseTestSource.avroSchema = HoodieTestDataGenerator.AVRO_TRIP_LOGICAL_TYPES_SCHEMA; + } + + // Insert data produced with Schema A, pass Schema A + HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT, Collections.singletonList(TestIdentityTransformer.class.getName()), + PROPS_FILENAME_TEST_SOURCE, false, true, false, null, HoodieTableType.MERGE_ON_READ.name()); + cfg.payloadClassName = DefaultHoodieRecordPayload.class.getName(); + cfg.recordMergeStrategyId = HoodieRecordMerger.EVENT_TIME_BASED_MERGE_STRATEGY_UUID; + cfg.recordMergeMode = RecordMergeMode.EVENT_TIME_ORDERING; + cfg.configs.add(String.format("%s=%s", HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT.key(), "0")); + cfg.configs.add("hoodie.datasource.write.row.writer.enable=false"); + + new HoodieDeltaStreamer(cfg, jsc).sync(); + assertUseV2Checkpoint(HoodieTestUtils.createMetaClient(storage, tableBasePath)); + assertRecordCount(1000, tableBasePath, sqlContext); + TestHelpers.assertCommitMetadata("00000", tableBasePath, 1); + TableSchemaResolver tableSchemaResolver = new TableSchemaResolver( + HoodieTestUtils.createMetaClient(storage, tableBasePath)); + Schema tableSchema = tableSchemaResolver.getTableAvroSchema(false); + Map hudiOpts = new HashMap<>(); + hudiOpts.put("hoodie.datasource.write.recordkey.field", "id"); + logicalAssertions(tableSchema, tableBasePath, hudiOpts, HoodieTableVersion.current().versionCode()); + + + cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, Collections.singletonList(TestIdentityTransformer.class.getName()), + PROPS_FILENAME_TEST_SOURCE, false, true, false, null, HoodieTableType.MERGE_ON_READ.name()); + cfg.payloadClassName = DefaultHoodieRecordPayload.class.getName(); + cfg.recordMergeStrategyId = HoodieRecordMerger.EVENT_TIME_BASED_MERGE_STRATEGY_UUID; + cfg.recordMergeMode = RecordMergeMode.EVENT_TIME_ORDERING; + cfg.configs.add(String.format("%s=%s", HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT.key(), "0")); + cfg.configs.add("hoodie.datasource.write.row.writer.enable=false"); + + new HoodieDeltaStreamer(cfg, jsc).sync(); + assertUseV2Checkpoint(HoodieTestUtils.createMetaClient(storage, tableBasePath)); + assertRecordCount(1450, tableBasePath, sqlContext); + TestHelpers.assertCommitMetadata("00001", tableBasePath, 2); + tableSchemaResolver = new TableSchemaResolver( + HoodieTestUtils.createMetaClient(storage, tableBasePath)); + tableSchema = tableSchemaResolver.getTableAvroSchema(false); + logicalAssertions(tableSchema, tableBasePath, hudiOpts, HoodieTableVersion.current().versionCode()); + } finally { + defaultSchemaProviderClassName = FilebasedSchemaProvider.class.getName(); + AbstractBaseTestSource.schemaStr = HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; + AbstractBaseTestSource.avroSchema = HoodieTestDataGenerator.AVRO_SCHEMA; + } + } + + @ParameterizedTest + @EnumSource(value = HoodieTableVersion.class, names = {"SIX", "EIGHT"}) + public void testBackwardsCompatibility(HoodieTableVersion version) throws Exception { + TestMercifulJsonToRowConverter.timestampNTZCompatibility(() -> { + String dirName = "colstats-upgrade-test-v" + version.versionCode(); + String dataPath = basePath + "/" + dirName; + java.nio.file.Path zipOutput = Paths.get(new URI(dataPath)); + HoodieTestUtils.extractZipToDirectory("col-stats/" + dirName + ".zip", zipOutput, getClass()); + String tableBasePath = zipOutput.resolve("trips_logical_types_json").toString(); + + TableSchemaResolver tableSchemaResolver = new TableSchemaResolver( + HoodieTestUtils.createMetaClient(storage, tableBasePath)); + Schema tableSchema = tableSchemaResolver.getTableAvroSchema(false); + Map hudiOpts = new HashMap<>(); + hudiOpts.put("hoodie.datasource.write.recordkey.field", "id"); + logicalAssertions(tableSchema, tableBasePath, hudiOpts, version.versionCode()); + + HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT, Collections.emptyList(), + "placeholder", false, true, false, null, HoodieTableType.MERGE_ON_READ.name()); + cfg.propsFilePath = zipOutput + "/hudi.properties"; + cfg.schemaProviderClassName = "org.apache.hudi.utilities.schema.FilebasedSchemaProvider"; + cfg.sourceOrderingFields = "timestamp"; + cfg.sourceClassName = "org.apache.hudi.utilities.sources.JsonDFSSource"; + cfg.targetTableName = "trips_logical_types_json"; + cfg.configs.add("hoodie.streamer.source.dfs.root=" + zipOutput + "/data/data_6/"); + cfg.configs.add(String.format(("%s=%s"), HoodieWriteConfig.WRITE_TABLE_VERSION.key(), version.versionCode())); + cfg.configs.add(String.format(("%s=%s"), HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key(), "100")); + String schemaPath = zipOutput + "/schema.avsc"; + cfg.configs.add(String.format(("%s=%s"), "hoodie.streamer.schemaprovider.source.schema.file", schemaPath)); + cfg.configs.add(String.format(("%s=%s"), "hoodie.streamer.schemaprovider.target.schema.file", schemaPath)); + cfg.forceDisableCompaction = true; + cfg.sourceLimit = 100_000; + cfg.ignoreCheckpoint = "12345"; + new HoodieDeltaStreamer(cfg, jsc).sync(); + logicalAssertions(tableSchema, tableBasePath, hudiOpts, version.versionCode()); + }); + } + + private void logicalAssertions(Schema tableSchema, String tableBasePath, Map hudiOpts, int tableVersion) { + if (tableVersion > 8) { + assertEquals("timestamp-millis", tableSchema.getField("ts_millis").schema().getLogicalType().getName()); + } + assertEquals("timestamp-micros", tableSchema.getField("ts_micros").schema().getLogicalType().getName()); + if (tableVersion > 8 && !HoodieSparkUtils.isSpark3_3()) { + assertEquals("local-timestamp-millis", tableSchema.getField("local_ts_millis").schema().getLogicalType().getName()); + assertEquals("local-timestamp-micros", tableSchema.getField("local_ts_micros").schema().getLogicalType().getName()); + } + + assertEquals("date", tableSchema.getField("event_date").schema().getLogicalType().getName()); + + if (tableVersion > 8) { + assertEquals("bytes", tableSchema.getField("dec_plain_large").schema().getType().getName()); + assertEquals("decimal", tableSchema.getField("dec_plain_large").schema().getLogicalType().getName()); + assertEquals(20, ((LogicalTypes.Decimal) tableSchema.getField("dec_plain_large").schema().getLogicalType()).getPrecision()); + assertEquals(10, ((LogicalTypes.Decimal) tableSchema.getField("dec_plain_large").schema().getLogicalType()).getScale()); + } + assertEquals("fixed", tableSchema.getField("dec_fixed_small").schema().getType().getName()); + assertEquals(3, tableSchema.getField("dec_fixed_small").schema().getFixedSize()); + assertEquals("decimal", tableSchema.getField("dec_fixed_small").schema().getLogicalType().getName()); + assertEquals(5, ((LogicalTypes.Decimal) tableSchema.getField("dec_fixed_small").schema().getLogicalType()).getPrecision()); + assertEquals(2, ((LogicalTypes.Decimal) tableSchema.getField("dec_fixed_small").schema().getLogicalType()).getScale()); + assertEquals("fixed", tableSchema.getField("dec_fixed_large").schema().getType().getName()); + assertEquals(8, tableSchema.getField("dec_fixed_large").schema().getFixedSize()); + assertEquals("decimal", tableSchema.getField("dec_fixed_large").schema().getLogicalType().getName()); + assertEquals(18, ((LogicalTypes.Decimal) tableSchema.getField("dec_fixed_large").schema().getLogicalType()).getPrecision()); + assertEquals(9, ((LogicalTypes.Decimal) tableSchema.getField("dec_fixed_large").schema().getLogicalType()).getScale()); + + sqlContext.clearCache(); + Dataset df = sqlContext.read() + .options(hudiOpts) + .format("org.apache.hudi") + .load(tableBasePath); + + long totalCount = df.count(); + long expectedHalf = totalCount / 2; + long tolerance = totalCount / 20; + if (totalCount < 100) { + tolerance = totalCount / 4; + } + + if (tableVersion > 8) { + assertHalfSplit(df, "ts_millis > timestamp('2020-01-01 00:00:00Z')", expectedHalf, tolerance, "ts_millis > threshold"); + assertHalfSplit(df, "ts_millis < timestamp('2020-01-01 00:00:00Z')", expectedHalf, tolerance, "ts_millis < threshold"); + assertBoundaryCounts(df, "ts_millis > timestamp('2020-01-01 00:00:00.001Z')", "ts_millis <= timestamp('2020-01-01 00:00:00.001Z')", totalCount); + assertBoundaryCounts(df, "ts_millis < timestamp('2019-12-31 23:59:59.999Z')", "ts_millis >= timestamp('2019-12-31 23:59:59.999Z')", totalCount); + } + + assertHalfSplit(df, "ts_micros > timestamp('2020-06-01 12:00:00Z')", expectedHalf, tolerance, "ts_micros > threshold"); + assertHalfSplit(df, "ts_micros < timestamp('2020-06-01 12:00:00Z')", expectedHalf, tolerance, "ts_micros < threshold"); + assertBoundaryCounts(df, "ts_micros > timestamp('2020-06-01 12:00:00.000001Z')", "ts_micros <= timestamp('2020-06-01 12:00:00.000001Z')", totalCount); + assertBoundaryCounts(df, "ts_micros < timestamp('2020-06-01 11:59:59.999999Z')", "ts_micros >= timestamp('2020-06-01 11:59:59.999999Z')", totalCount); + + if (tableVersion > 8 && !HoodieSparkUtils.isSpark3_3()) { + assertHalfSplit(df, "local_ts_millis > CAST('2015-05-20 12:34:56' AS TIMESTAMP_NTZ)", expectedHalf, tolerance, "local_ts_millis > threshold"); + assertHalfSplit(df, "local_ts_millis < CAST('2015-05-20 12:34:56' AS TIMESTAMP_NTZ)", expectedHalf, tolerance, "local_ts_millis < threshold"); + assertBoundaryCounts(df, "local_ts_millis > CAST('2015-05-20 12:34:56.001' AS TIMESTAMP_NTZ)", "local_ts_millis <= CAST('2015-05-20 12:34:56.001' AS TIMESTAMP_NTZ)", totalCount); + assertBoundaryCounts(df, "local_ts_millis < CAST('2015-05-20 12:34:55.999' AS TIMESTAMP_NTZ)", "local_ts_millis >= CAST('2015-05-20 12:34:55.999' AS TIMESTAMP_NTZ)", totalCount); + + assertHalfSplit(df, "local_ts_micros > CAST('2017-07-07 07:07:07' AS TIMESTAMP_NTZ)", expectedHalf, tolerance, "local_ts_micros > threshold"); + assertHalfSplit(df, "local_ts_micros < CAST('2017-07-07 07:07:07' AS TIMESTAMP_NTZ)", expectedHalf, tolerance, "local_ts_micros < threshold"); + assertBoundaryCounts(df, "local_ts_micros > CAST('2017-07-07 07:07:07.000001' AS TIMESTAMP_NTZ)", "local_ts_micros <= CAST('2017-07-07 07:07:07.000001' AS TIMESTAMP_NTZ)", totalCount); + assertBoundaryCounts(df, "local_ts_micros < CAST('2017-07-07 07:07:06.999999' AS TIMESTAMP_NTZ)", "local_ts_micros >= CAST('2017-07-07 07:07:06.999999' AS TIMESTAMP_NTZ)", totalCount); + + } + + assertHalfSplit(df, "event_date > date('2000-01-01')", expectedHalf, tolerance, "event_date > threshold"); + assertHalfSplit(df, "event_date < date('2000-01-01')", expectedHalf, tolerance, "event_date < threshold"); + assertBoundaryCounts(df, "event_date > date('2000-01-02')", "event_date <= date('2000-01-02')", totalCount); + assertBoundaryCounts(df, "event_date < date('1999-12-31')", "event_date >= date('1999-12-31')", totalCount); + + if (tableVersion > 8) { + assertHalfSplit(df, "dec_plain_large < 1234567890.0987654321", expectedHalf, tolerance, "dec_plain_large < threshold"); + assertHalfSplit(df, "dec_plain_large > 1234567890.0987654321", expectedHalf, tolerance, "dec_plain_large > threshold"); + assertBoundaryCounts(df, "dec_plain_large < 1234567890.0987654320", "dec_plain_large >= 1234567890.0987654320", totalCount); + assertBoundaryCounts(df, "dec_plain_large > 1234567890.0987654322", "dec_plain_large <= 1234567890.0987654322", totalCount); + } + + assertHalfSplit(df, "dec_fixed_small < 543.21", expectedHalf, tolerance, "dec_fixed_small < threshold"); + assertHalfSplit(df, "dec_fixed_small > 543.21", expectedHalf, tolerance, "dec_fixed_small > threshold"); + assertBoundaryCounts(df, "dec_fixed_small < 543.20", "dec_fixed_small >= 543.20", totalCount); + assertBoundaryCounts(df, "dec_fixed_small > 543.22", "dec_fixed_small <= 543.22", totalCount); + + assertHalfSplit(df, "dec_fixed_large < 987654321.123456789", expectedHalf, tolerance, "dec_fixed_large < threshold"); + assertHalfSplit(df, "dec_fixed_large > 987654321.123456789", expectedHalf, tolerance, "dec_fixed_large > threshold"); + assertBoundaryCounts(df, "dec_fixed_large < 987654321.123456788", "dec_fixed_large >= 987654321.123456788", totalCount); + assertBoundaryCounts(df, "dec_fixed_large > 987654321.123456790", "dec_fixed_large <= 987654321.123456790", totalCount); + } + + private void assertHalfSplit(Dataset df, String filterExpr, long expectedHalf, long tolerance, String msg) { + long count = df.filter(filterExpr).count(); + assertTrue(Math.abs(count - expectedHalf) <= tolerance, msg + " (got=" + count + ", expected=" + expectedHalf + ")"); + } + + private void assertBoundaryCounts(Dataset df, String exprZero, String exprTotal, long totalCount) { + assertEquals(0, df.filter(exprZero).count(), exprZero); + assertEquals(totalCount, df.filter(exprTotal).count(), exprTotal); + } + private static Stream continuousModeArgs() { return Stream.of( Arguments.of("AVRO", "CURRENT"), diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestMercifulJsonToRowConverter.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestMercifulJsonToRowConverter.java index de8076cab8a59..595132eaacf4b 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestMercifulJsonToRowConverter.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestMercifulJsonToRowConverter.java @@ -19,8 +19,10 @@ package org.apache.hudi.utilities.sources.helpers; import org.apache.hudi.AvroConversionUtils; +import org.apache.hudi.HoodieSparkUtils; import org.apache.hudi.avro.MercifulJsonConverterTestBase; import org.apache.hudi.common.testutils.SchemaTestUtil; +import org.apache.hudi.stats.ValueType; import org.apache.hudi.utilities.exception.HoodieJsonToRowConversionException; import com.fasterxml.jackson.core.JsonProcessingException; @@ -60,7 +62,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; -class TestMercifulJsonToRowConverter extends MercifulJsonConverterTestBase { +public class TestMercifulJsonToRowConverter extends MercifulJsonConverterTestBase { private static final ObjectMapper MAPPER = new ObjectMapper(); private static final MercifulJsonToRowConverter CONVERTER = new MercifulJsonToRowConverter(true, "__"); @@ -318,10 +320,10 @@ void dateLogicalTypeTest(String groundTruthRow, Object dateInput) throws IOExcep if (groundTruthRow == null) { return; } - Row rec = RowFactory.create(java.sql.Date.valueOf(groundTruthRow)); + Row rec = RowFactory.create(java.sql.Date.valueOf(groundTruthRow).toLocalDate()); Row realRow = CONVERTER.convertToRow(json, schema); validateSchemaCompatibility(Collections.singletonList(realRow), schema); - assertEquals(rec.getDate(0).toString(), realRow.getDate(0).toString()); + assertEquals(rec.getLocalDate(0).toString(), realRow.getLocalDate(0).toString()); } /** @@ -344,6 +346,33 @@ void dateLogicalTypeTest() throws IOException { private static final String LOCAL_TIME_AVRO_FILE_PATH = "/local-timestamp-logical-type.avsc"; + @FunctionalInterface + public interface ThrowingRunnable { + void run() throws Exception; + } + + public static void timestampNTZCompatibility(ThrowingRunnable r) throws Exception { + // TODO: Remove this when we get rid of spark3.3. TimestampNTZ needs this config + // to be set to true to work. + boolean isSpark33 = HoodieSparkUtils.isSpark3_3(); + String propertyValue = null; + if (isSpark33) { + propertyValue = System.getProperty("spark.testing"); + System.setProperty("spark.testing", "true"); + } + try { + r.run(); + } finally { + if (isSpark33) { + if (propertyValue == null) { + System.clearProperty("spark.testing"); + } else { + System.setProperty("spark.testing", propertyValue); + } + } + } + } + /** * Covered case: * Avro Logical Type: localTimestampMillisField & localTimestampMillisField @@ -354,23 +383,25 @@ void dateLogicalTypeTest() throws IOException { @ParameterizedTest @MethodSource("localTimestampGoodCaseProvider") void localTimestampLogicalTypeGoodCaseTest( - Long expectedMicroSecOfDay, Object timeMilli, Object timeMicro) throws IOException { - // Example inputs - long microSecOfDay = expectedMicroSecOfDay; - long milliSecOfDay = expectedMicroSecOfDay / 1000; // Represents 12h 30 min since the start of the day + Long expectedMicroSecOfDay, Object timeMilli, Object timeMicro) throws Exception { + timestampNTZCompatibility(() -> { + // Example inputs + long microSecOfDay = expectedMicroSecOfDay; + long milliSecOfDay = expectedMicroSecOfDay / 1000; // Represents 12h 30 min since the start of the day - // Define the schema for the date logical type - Schema schema = SchemaTestUtil.getSchema(LOCAL_TIME_AVRO_FILE_PATH); + // Define the schema for the date logical type + Schema schema = SchemaTestUtil.getSchema(LOCAL_TIME_AVRO_FILE_PATH); - Map data = new HashMap<>(); - data.put("localTimestampMillisField", timeMilli); - data.put("localTimestampMicrosField", timeMicro); - String json = MAPPER.writeValueAsString(data); + Map data = new HashMap<>(); + data.put("localTimestampMillisField", timeMilli); + data.put("localTimestampMicrosField", timeMicro); + String json = MAPPER.writeValueAsString(data); - Row rec = RowFactory.create(milliSecOfDay, microSecOfDay); - Row actualRow = CONVERTER.convertToRow(json, schema); - validateSchemaCompatibility(Collections.singletonList(actualRow), schema); - assertEquals(rec, actualRow); + Row rec = RowFactory.create(ValueType.castToLocalTimestampMillis(milliSecOfDay, null), ValueType.castToLocalTimestampMicros(microSecOfDay, null)); + Row actualRow = CONVERTER.convertToRow(json, schema); + validateSchemaCompatibility(Collections.singletonList(actualRow), schema); + assertEquals(rec, actualRow); + }); } @ParameterizedTest diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/ColStatsUpgradeTesting.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/ColStatsUpgradeTesting.java new file mode 100644 index 0000000000000..5ba3f4ea82b11 --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/ColStatsUpgradeTesting.java @@ -0,0 +1,160 @@ +/* + * 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.utilities.testutils; + +import org.apache.hudi.common.table.HoodieTableVersion; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.utilities.deltastreamer.TestHoodieDeltaStreamer; + +import org.apache.avro.Schema; +import org.junit.jupiter.api.Disabled; + +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.io.Writer; +import java.util.List; + +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.recordsToStrings; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Used by {@link TestHoodieDeltaStreamer#testBackwardsCompatibility(HoodieTableVersion)}. + * Only run this manually + */ +public class ColStatsUpgradeTesting { + + @Disabled + public void generate() throws IOException { + generateTestAssets("/tmp/", 6); + generateTestAssets("/tmp/", 8); + } + + public void generateDsScript(StoragePath assetDirectory, StoragePath runScript, StoragePath tablePath, StoragePath propsFile, StoragePath dataDirectory, int version) throws IOException { + HoodieStorage storage = HoodieTestUtils.getDefaultStorage(); + String bundleURL; + String bundleName; + String instruction; + if (version == 6) { + bundleURL = "https://repo1.maven.org/maven2/org/apache/hudi/hudi-utilities-bundle_2.12/0.14.1/hudi-utilities-bundle_2.12-0.14.1.jar"; + bundleName = "hudi-utilities-bundle_2.12-0.14.1.jar"; + instruction = "run with spark 3.1.X"; + } else if (version == 8) { + bundleURL = "https://repo1.maven.org/maven2/org/apache/hudi/hudi-utilities-bundle_2.12/1.0.2/hudi-utilities-bundle_2.12-1.0.2.jar"; + bundleName = "hudi-utilities-bundle_2.12-1.0.2.jar"; + instruction = "run with spark 3.5.X"; + } else { + throw new IllegalArgumentException("Unsupported version: " + version); + } + + String runscript = "# " + instruction + "\n" + + "wget " + bundleURL + ";\n" + + "for i in {0..4}; do\n" + + " spark-submit \\\n" + + " --class org.apache.hudi.utilities.streamer.HoodieStreamer \\\n" + + " " + bundleName + " \\\n" + + " --table-type MERGE_ON_READ \\\n" + + " --source-class org.apache.hudi.utilities.sources.JsonDFSSource \\\n" + + " --source-ordering-field timestamp \\\n" + + " --target-base-path " + tablePath + " \\\n" + + " --target-table trips_logical_types_json \\\n" + + " --props " + propsFile.makeQualified(storage.getUri()).toUri().toString() + "\\\n" + + " --schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider \\\n" + + " --disable-compaction \\\n" + + " --op UPSERT \\\n" + + " --hoodie-conf hoodie.streamer.source.dfs.root=" + dataDirectory.makeQualified(storage.getUri()).toUri().toString() + "/data_$i\n" + + "done;\n" + + "cd " + assetDirectory + ";\n" + + "zip -r $HUDI_HOME/hudi-utilities/src/test/resources/col-stats/" + assetDirectory.getName() + ".zip .;\n"; + + try (Writer writer = new OutputStreamWriter(storage.create(runScript))) { + writer.write(runscript); + writer.write("\n"); + } + } + + public void generateTestAssets(String assetDirectory, int version) throws IOException { + HoodieStorage storage = HoodieTestUtils.getDefaultStorage(); + StoragePath directory = new StoragePath(assetDirectory, "colstats-upgrade-test-v" + version); + if (storage.exists(directory)) { + storage.deleteDirectory(directory); + } + assertTrue(storage.createDirectory(directory)); + Schema schema; + String schemaStr; + //TODO: once we add the fixes to v8 to allow more types + if (version == 6 || version == 8) { + schema = HoodieTestDataGenerator.AVRO_TRIP_LOGICAL_TYPES_SCHEMA_V6; + schemaStr = HoodieTestDataGenerator.TRIP_LOGICAL_TYPES_SCHEMA_V6; + } else { + schema = HoodieTestDataGenerator.AVRO_TRIP_LOGICAL_TYPES_SCHEMA; + schemaStr = HoodieTestDataGenerator.TRIP_LOGICAL_TYPES_SCHEMA; + } + + StoragePath schemaFile = new StoragePath(directory, "schema.avsc"); + + try (Writer writer = new OutputStreamWriter(storage.create(schemaFile))) { + writer.write(schema.toString(true)); + writer.write("\n"); + } + + HoodieTestDataGenerator datagen = new HoodieTestDataGenerator(); + StoragePath dataDirectory = new StoragePath(directory, "data"); + assertTrue(storage.createDirectory(dataDirectory)); + + StoragePath propsFile = new StoragePath(directory, "hudi.properties"); + try (Writer writer = new OutputStreamWriter(storage.create(propsFile))) { + writer.write("hoodie.table.name=trips_logical_types_json\n"); + writer.write("hoodie.datasource.write.table.type=MERGE_ON_READ\n"); + writer.write("hoodie.datasource.write.recordkey.field=_row_key\n"); + writer.write("hoodie.datasource.write.partitionpath.field=partition_path\n"); + writer.write("hoodie.datasource.write.keygenerator.class=org.apache.hudi.keygen.SimpleKeyGenerator\n"); + writer.write("hoodie.datasource.write.precombine.field=timestamp\n"); + writer.write("hoodie.cleaner.policy=KEEP_LATEST_COMMITS\n"); + writer.write("hoodie.cleaner.commits.retained=2\n"); + writer.write("hoodie.upsert.shuffle.parallelism=2\n"); + writer.write("hoodie.insert.shuffle.parallelism=2\n"); + writer.write("hoodie.compact.inline=false\n"); + writer.write("hoodie.streamer.schemaprovider.source.schema.file=" + schemaFile.makeQualified(storage.getUri()).toUri().toString() + "\n"); + writer.write("hoodie.streamer.schemaprovider.target.schema.file=" + schemaFile.makeQualified(storage.getUri()).toUri().toString() + "\n"); + writer.write("hoodie.metadata.index.column.stats.enable=true\n"); + } + + // generate extra data that we can use to ingest with latest hudi in legacy mode + for (int i = 0; i < 10; i++) { + StoragePath dataCheckpointDir = new StoragePath(dataDirectory, "data_" + i); + StoragePath dataFile = new StoragePath(dataCheckpointDir, "data.json"); + List records = recordsToStrings(i == 0 + ? datagen.generateInsertsAsPerSchema("00" + i, 20, schemaStr) + : datagen.generateUniqueUpdatesAsPerSchema("00" + i, 10, schemaStr)); + try (Writer writer = new OutputStreamWriter(storage.create(dataFile))) { + for (String record : records) { + writer.write(record); + writer.write("\n"); + } + } + } + StoragePath scriptFile = new StoragePath(directory, "runscript.sh"); + StoragePath tablePath = new StoragePath(directory, "trips_logical_types_json"); + generateDsScript(directory, scriptFile, tablePath, propsFile, dataDirectory, version); + } +} \ No newline at end of file diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java index cbe823dd07f0a..276dbd8c26f9d 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java @@ -29,6 +29,7 @@ import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.sources.AvroSource; +import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Row; @@ -49,6 +50,9 @@ public abstract class AbstractBaseTestSource extends AvroSource { + public static String schemaStr = HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; + public static Schema avroSchema = HoodieTestDataGenerator.AVRO_SCHEMA; + private static final Logger LOG = LoggerFactory.getLogger(AbstractBaseTestSource.class); public static final int DEFAULT_PARTITION_NUM = 0; @@ -110,7 +114,7 @@ protected static Stream fetchNextBatch(TypedProperties props, int HoodieTestDataGenerator dataGenerator = dataGeneratorMap.get(partition); // generate `sourceLimit` number of upserts each time. - int numExistingKeys = dataGenerator.getNumExistingKeys(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA); + int numExistingKeys = dataGenerator.getNumExistingKeys(schemaStr); LOG.info("NumExistingKeys={}", numExistingKeys); int numUpdates = Math.min(numExistingKeys, sourceLimit / 2); @@ -137,15 +141,15 @@ protected static Stream fetchNextBatch(TypedProperties props, int if (!reachedMax && numUpdates >= 50) { LOG.info("After adjustments => NumInserts={}, NumUpdates={}, NumDeletes=50, maxUniqueRecords={}", numInserts, (numUpdates - 50), maxUniqueKeys); // if we generate update followed by deletes -> some keys in update batch might be picked up for deletes. Hence generating delete batch followed by updates - deleteStream = dataGenerator.generateUniqueDeleteRecords(instantTime, 50, 0L).stream().map(AbstractBaseTestSource::toGenericRecord); - updateStream = dataGenerator.generateUniqueUpdatesStream(instantTime, numUpdates - 50, HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, 0L) + deleteStream = dataGenerator.generateUniqueDeleteRecordStream(instantTime, 50, false, schemaStr, 0L).map(AbstractBaseTestSource::toGenericRecord); + updateStream = dataGenerator.generateUniqueUpdatesStream(instantTime, numUpdates - 50, schemaStr, 0L) .map(AbstractBaseTestSource::toGenericRecord); } else { LOG.info("After adjustments => NumInserts={}, NumUpdates={}, maxUniqueRecords={}", numInserts, numUpdates, maxUniqueKeys); - updateStream = dataGenerator.generateUniqueUpdatesStream(instantTime, numUpdates, HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, 0L) + updateStream = dataGenerator.generateUniqueUpdatesStream(instantTime, numUpdates, schemaStr, 0L) .map(AbstractBaseTestSource::toGenericRecord); } - Stream insertStream = dataGenerator.generateInsertsStream(instantTime, numInserts, false, HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, 0L) + Stream insertStream = dataGenerator.generateInsertsStream(instantTime, numInserts, false, schemaStr, 0L) .map(AbstractBaseTestSource::toGenericRecord); if (Boolean.valueOf(props.getOrDefault("hoodie.test.source.generate.inserts", "false").toString())) { return insertStream; diff --git a/hudi-utilities/src/test/resources/col-stats/README.md b/hudi-utilities/src/test/resources/col-stats/README.md new file mode 100644 index 0000000000000..afb601bbba23a --- /dev/null +++ b/hudi-utilities/src/test/resources/col-stats/README.md @@ -0,0 +1,40 @@ + + +## How To Generate Test +use org.apache.hudi.utilities.testutils.ColStatsUpgradeTesting +to generate the files. It should only need to be updated if we +change col stats again and need to add another table version to test +against. Also if we fix logical types for deltastreamer in a 0.15.x +we should regen the table version 8 with those extra types. + +There will be a runscript `runscript.sh` that you can use to download the +bundles and the spark submit deltastreamer job in a loop. +After running the script you will need to zip the directory. The hoodie table +is in the directory as well as schemas, the runscript and .properties files. +For the zip you should cd into the directory like + +``` +cd /tmp/colstats-upgrade-test-v6 +zip -r $HUDI_HOME/hudi-utilities/src/test/resources/col-stats/colstats-upgrade-test-v6.zip . +``` +you have to cd into the directory, otherwise when it is unzipped, it will +maintain the structure of the directory it was zipped from. + + + + diff --git a/hudi-utilities/src/test/resources/col-stats/colstats-upgrade-test-v6.zip b/hudi-utilities/src/test/resources/col-stats/colstats-upgrade-test-v6.zip new file mode 100644 index 0000000000000..2ea3779333bbe Binary files /dev/null and b/hudi-utilities/src/test/resources/col-stats/colstats-upgrade-test-v6.zip differ diff --git a/hudi-utilities/src/test/resources/col-stats/colstats-upgrade-test-v8.zip b/hudi-utilities/src/test/resources/col-stats/colstats-upgrade-test-v8.zip new file mode 100644 index 0000000000000..529a12dd9e839 Binary files /dev/null and b/hudi-utilities/src/test/resources/col-stats/colstats-upgrade-test-v8.zip differ diff --git a/hudi-utilities/src/test/resources/streamer-config/source-timestamp-millis.avsc b/hudi-utilities/src/test/resources/streamer-config/source-timestamp-millis.avsc new file mode 100644 index 0000000000000..5d47b96aa9379 --- /dev/null +++ b/hudi-utilities/src/test/resources/streamer-config/source-timestamp-millis.avsc @@ -0,0 +1,162 @@ +/* + * 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. + */ +{ + "type" : "record", + "name" : "triprec", + "fields" : [ + { + "name": "timestamp", + "type": "long" + }, + { + "name": "_row_key", + "type": "string" + }, + { + "name": "partition_path", + "type": "string" + }, + { + "name": "trip_type", + "type": { + "type": "enum", + "name": "TripType", + "symbols": [ + "UNKNOWN", + "UBERX", + "BLACK" + ], + "default": "UNKNOWN" + } + }, + { + "name": "rider", + "type": "string" + }, + { + "name": "driver", + "type": "string" + }, + { + "name": "begin_lat", + "type": "double" + }, + { + "name": "begin_lon", + "type": "double" + }, + { + "name": "end_lat", + "type": "double" + }, + { + "name": "end_lon", + "type": "double" + }, + { + "name": "distance_in_meters", + "type": "int" + }, + { + "name": "seconds_since_epoch", + "type": "long" + }, + { + "name": "weight", + "type": "float" + }, + { + "name": "nation", + "type": "bytes" + }, + { + "name": "current_date", + "type": { + "type": "int", + "logicalType": "date" + } + }, + { + "name": "current_ts", + "type": { + "type": "long", + "logicalType": "timestamp-millis" + } + }, + { + "name": "height", + "type": { + "type": "fixed", + "name": "abc", + "size": 5, + "logicalType": "decimal", + "precision": 10, + "scale": 6 + } + }, + { + "name": "city_to_state", + "type": { + "type": "map", + "values": "string" + } + }, + { + "name": "fare", + "type": { + "type": "record", + "name": "fare", + "fields": [ + { + "name": "amount", + "type": "double" + }, + { + "name": "currency", + "type": "string" + } + ] + } + }, + { + "name": "tip_history", + "type": { + "type": "array", + "items": { + "type": "record", + "name": "tip_history", + "fields": [ + { + "name": "amount", + "type": "double" + }, + { + "name": "currency", + "type": "string" + } + ] + } + } + }, + { + "name": "_hoodie_is_deleted", + "type": "boolean", + "default": false + } + ] +} diff --git a/packaging/bundle-validation/validate.sh b/packaging/bundle-validation/validate.sh index 3500f656b8ca8..3fd84786f62e7 100755 --- a/packaging/bundle-validation/validate.sh +++ b/packaging/bundle-validation/validate.sh @@ -90,11 +90,11 @@ test_spark_hadoop_mr_bundles () { local HIVE_PID=$! change_java_runtime_version echo "::warning::validate.sh Writing sample data via Spark DataSource and run Hive Sync..." - $SPARK_HOME/bin/spark-shell --jars $JARS_DIR/spark.jar < $WORKDIR/spark_hadoop_mr/write.scala + $SPARK_HOME/bin/spark-shell --jars $JARS_DIR/spark.jar --conf 'spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension' --conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer' --conf 'spark.kryo.registrator=org.apache.spark.HoodieSparkKryoRegistrar' --conf 'spark.sql.catalog.spark_catalog=org.apache.spark.sql.hudi.catalog.HoodieCatalog' < $WORKDIR/spark_hadoop_mr/write.scala echo "::warning::validate.sh Query and validate the results using Spark SQL" # save Spark SQL query results - $SPARK_HOME/bin/spark-shell --jars $JARS_DIR/spark.jar < $WORKDIR/spark_hadoop_mr/validate.scala + $SPARK_HOME/bin/spark-shell --jars $JARS_DIR/spark.jar --conf 'spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension' --conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer' --conf 'spark.kryo.registrator=org.apache.spark.HoodieSparkKryoRegistrar' --conf 'spark.sql.catalog.spark_catalog=org.apache.spark.sql.hudi.catalog.HoodieCatalog' < $WORKDIR/spark_hadoop_mr/validate.scala numRecords=$(cat /tmp/spark-bundle/sparksql/trips/results/*.csv | wc -l) if [ "$numRecords" -ne 10 ]; then echo "::error::validate.sh Spark SQL validation failed." diff --git a/scripts/release/validate_source_copyright.sh b/scripts/release/validate_source_copyright.sh index 33cecd1752978..1d1c6bf4506bb 100755 --- a/scripts/release/validate_source_copyright.sh +++ b/scripts/release/validate_source_copyright.sh @@ -50,7 +50,7 @@ echo "Performing custom Licensing Check " # Exclude the 'hudi-trino-plugin' directory. Its license checks are handled by airlift: # https://github.com/airlift/airbase/blob/823101482dbc60600d7862f0f5c93aded6190996/airbase/pom.xml#L1239 # --- -numfilesWithNoLicense=$(find . -path './hudi-trino-plugin' -prune -o -type f -iname '*' | grep -v './hudi-trino-plugin' | grep -v NOTICE | grep -v LICENSE | grep -v '.jpg' | grep -v '.json' | grep -v '.hfile' | grep -v '.data' | grep -v '.commit' | grep -v emptyFile | grep -v DISCLAIMER | grep -v '.sqltemplate' | grep -v KEYS | grep -v '.mailmap' | grep -v 'banner.txt' | grep -v '.txt' | grep -v "fixtures" | xargs grep -L "Licensed to the Apache Software Foundation (ASF)") +numfilesWithNoLicense=$(find . -path './hudi-trino-plugin' -prune -o -type f -iname '*' | grep -v './hudi-trino-plugin' | grep -v NOTICE | grep -v LICENSE | grep -v '.jpg' | grep -v '.json' | grep -v '.zip' | grep -v '.hfile' | grep -v '.data' | grep -v '.commit' | grep -v emptyFile | grep -v DISCLAIMER | grep -v '.sqltemplate' | grep -v KEYS | grep -v '.mailmap' | grep -v 'banner.txt' | grep -v '.txt' | grep -v "fixtures" | xargs grep -L "Licensed to the Apache Software Foundation (ASF)") # Check if the variable holding the list of files is non-empty if [ -n "$numfilesWithNoLicense" ]; then # If the list isn't empty, count the files and report the error