diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index bf4fca36fc450..c54d12f8624ac 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -117,7 +117,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema; +import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; import static org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy.EAGER; import static org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy.LAZY; import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_METADATA_PARTITIONS; @@ -1019,7 +1019,7 @@ static void validateSecondaryIndexSchemaEvolution( if (writerField != null && !tableField.schema().equals(writerField.schema())) { // Check if this is just making the field nullable/non-nullable, which is safe from SI perspective - if (resolveNullableSchema(tableField.schema()).equals(resolveNullableSchema(writerField.schema()))) { + if (getNonNullTypeFromUnion(tableField.schema()).equals(getNonNullTypeFromUnion(writerField.schema()))) { continue; } diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderOnHive.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderOnHive.java index 1da7ca54be4da..ec14e7dc6707a 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderOnHive.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/hadoop/TestHoodieFileGroupReaderOnHive.java @@ -94,7 +94,7 @@ public StorageConfiguration getStorageConf() { @Override public HoodieReaderContext getHoodieReaderContext(String tablePath, Schema avroSchema, StorageConfiguration storageConf, HoodieTableMetaClient metaClient) { - HoodieFileGroupReaderBasedRecordReader.HiveReaderCreator readerCreator = (inputSplit, jobConf) -> new MapredParquetInputFormat().getRecordReader(inputSplit, jobConf, null); + HoodieFileGroupReaderBasedRecordReader.HiveReaderCreator readerCreator = (inputSplit, jobConf, dataSchema) -> new MapredParquetInputFormat().getRecordReader(inputSplit, jobConf, null); JobConf jobConf = new JobConf(storageConf.unwrapAs(Configuration.class)); setupJobconf(jobConf, avroSchema); return new HiveHoodieReaderContext(readerCreator, diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/ArrayWritableTestUtil.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/ArrayWritableTestUtil.java index 70f021610f999..e4e482be2610e 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/ArrayWritableTestUtil.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/ArrayWritableTestUtil.java @@ -23,6 +23,7 @@ import org.apache.avro.Schema; import org.apache.hadoop.hive.serde2.io.DateWritable; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritable; import org.apache.hadoop.io.ArrayWritable; import org.apache.hadoop.io.BooleanWritable; import org.apache.hadoop.io.BytesWritable; @@ -268,7 +269,11 @@ private static void assertWritablePrimaryTypeMatchesSchema(Schema schema, Writab break; case LONG: - assertInstanceOf(LongWritable.class, writable); + if (schema.getLogicalType() instanceof LogicalTypes.TimestampMillis) { + assertInstanceOf(TimestampWritable.class, writable); + } else { + assertInstanceOf(LongWritable.class, writable); + } break; case FLOAT: diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java index 5e5e6b9b4216c..1f3cf74037e7e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java @@ -19,6 +19,7 @@ package org.apache.hudi.io.storage; import org.apache.hudi.SparkAdapterSupport$; +import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.model.HoodieFileFormat; @@ -40,6 +41,7 @@ import org.apache.hadoop.fs.Path; import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.SchemaRepair; import org.apache.spark.sql.HoodieInternalRowUtils; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.UnsafeProjection; @@ -60,6 +62,7 @@ import static org.apache.hudi.common.util.TypeUtils.unsafeCast; import static org.apache.parquet.avro.AvroSchemaConverter.ADD_LIST_ELEMENT_RECORDS; +import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; public class HoodieSparkParquetReader implements HoodieSparkFileReader { @@ -67,6 +70,7 @@ public class HoodieSparkParquetReader implements HoodieSparkFileReader { private final HoodieStorage storage; private final FileFormatUtils parquetUtils; private final List readerIterators = new ArrayList<>(); + private Option fileSchemaOption = Option.empty(); private Option structTypeOption = Option.empty(); private Option schemaOption = Option.empty(); @@ -116,19 +120,20 @@ public ClosableIterator getRecordKeyIterator() throws IOException { } public ClosableIterator getUnsafeRowIterator(Schema requestedSchema) throws IOException { - return getUnsafeRowIterator(HoodieInternalRowUtils.getCachedSchema(requestedSchema)); - } - - public ClosableIterator getUnsafeRowIterator(StructType requestedSchema) throws IOException { - SparkBasicSchemaEvolution evolution = new SparkBasicSchemaEvolution(getStructSchema(), requestedSchema, SQLConf.get().sessionLocalTimeZone()); + Schema nonNullSchema = AvroSchemaUtils.getNonNullTypeFromUnion(requestedSchema); + StructType structSchema = HoodieInternalRowUtils.getCachedSchema(nonNullSchema); + Option messageSchema = Option.of(getAvroSchemaConverter(storage.getConf().unwrapAs(Configuration.class)).convert(nonNullSchema)); + boolean enableTimestampFieldRepair = storage.getConf().getBoolean("logicalTimestampField.repair.enable", true); + StructType dataStructType = convertToStruct(enableTimestampFieldRepair ? SchemaRepair.repairLogicalTypes(getFileSchema(), messageSchema) : getFileSchema()); + SparkBasicSchemaEvolution evolution = new SparkBasicSchemaEvolution(dataStructType, structSchema, SQLConf.get().sessionLocalTimeZone()); String readSchemaJson = evolution.getRequestSchema().json(); storage.getConf().set(ParquetReadSupport.PARQUET_READ_SCHEMA, readSchemaJson); storage.getConf().set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA(), readSchemaJson); storage.getConf().set(SQLConf.PARQUET_BINARY_AS_STRING().key(), SQLConf.get().getConf(SQLConf.PARQUET_BINARY_AS_STRING()).toString()); storage.getConf().set(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), SQLConf.get().getConf(SQLConf.PARQUET_INT96_AS_TIMESTAMP()).toString()); - ParquetReader reader = ParquetReader.builder(new HoodieParquetReadSupport(Option$.MODULE$.empty(), true, + ParquetReader reader = ParquetReader.builder(new HoodieParquetReadSupport(Option$.MODULE$.empty(), true, true, SparkAdapterSupport$.MODULE$.sparkAdapter().getRebaseSpec("CORRECTED"), - SparkAdapterSupport$.MODULE$.sparkAdapter().getRebaseSpec("LEGACY")), + SparkAdapterSupport$.MODULE$.sparkAdapter().getRebaseSpec("LEGACY"), messageSchema), new Path(path.toUri())) .withConf(storage.getConf().unwrapAs(Configuration.class)) .build(); @@ -139,15 +144,22 @@ public ClosableIterator getUnsafeRowIterator(StructType requestedSche return projectedIterator; } + private MessageType getFileSchema() { + if (fileSchemaOption.isEmpty()) { + MessageType messageType = ((ParquetUtils) parquetUtils).readSchema(storage, path); + fileSchemaOption = Option.of(messageType); + } + return fileSchemaOption.get(); + } + @Override public Schema getSchema() { if (schemaOption.isEmpty()) { // Some types in avro are not compatible with parquet. // Avro only supports representing Decimals as fixed byte array // and therefore if we convert to Avro directly we'll lose logical type-info. - MessageType messageType = ((ParquetUtils) parquetUtils).readSchema(storage, path); - StructType structType = new ParquetToSparkSchemaConverter(storage.getConf().unwrapAs(Configuration.class)).convert(messageType); - structTypeOption = Option.of(structType); + MessageType messageType = getFileSchema(); + StructType structType = getStructSchema(); schemaOption = Option.of(SparkAdapterSupport$.MODULE$.sparkAdapter() .getAvroSchemaConverters() .toAvroType(structType, true, messageType.getName(), StringUtils.EMPTY_STRING)); @@ -157,11 +169,16 @@ public Schema getSchema() { protected StructType getStructSchema() { if (structTypeOption.isEmpty()) { - getSchema(); + MessageType messageType = getFileSchema(); + structTypeOption = Option.of(convertToStruct(messageType)); } return structTypeOption.get(); } + private StructType convertToStruct(MessageType messageType) { + return new ParquetToSparkSchemaConverter(storage.getConf().unwrapAs(Configuration.class)).convert(messageType); + } + @Override public void close() { readerIterators.forEach(ClosableIterator::close); 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 594587c616168..edbdab45243d0 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 @@ -76,7 +76,7 @@ import scala.Enumeration; import scala.Function1; -import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema; +import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_FIELD_ID_WRITE_ENABLED; import static org.apache.hudi.config.HoodieWriteConfig.ALLOW_OPERATION_METADATA_FIELD; import static org.apache.hudi.config.HoodieWriteConfig.AVRO_SCHEMA_STRING; @@ -226,7 +226,7 @@ private void writeFields(InternalRow row, StructType schema, ValueWriter[] field } private ValueWriter makeWriter(Schema avroSchema, DataType dataType) { - Schema resolvedSchema = avroSchema == null ? null : resolveNullableSchema(avroSchema); + Schema resolvedSchema = avroSchema == null ? null : getNonNullTypeFromUnion(avroSchema); LogicalType logicalType = resolvedSchema != null ? resolvedSchema.getLogicalType() : null; if (dataType == DataTypes.BooleanType) { @@ -429,7 +429,7 @@ private Type convertField(Schema avroFieldSchema, StructField structField) { } private Type convertField(Schema avroFieldSchema, StructField structField, Type.Repetition repetition) { - Schema resolvedSchema = avroFieldSchema == null ? null : resolveNullableSchema(avroFieldSchema); + Schema resolvedSchema = avroFieldSchema == null ? null : getNonNullTypeFromUnion(avroFieldSchema); LogicalType logicalType = resolvedSchema != null ? resolvedSchema.getLogicalType() : null; DataType dataType = structField.dataType(); diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala index 03e5c7f3a4262..15300447a6310 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala @@ -101,7 +101,7 @@ object AvroConversionUtils { recordNamespace: String): Row => GenericRecord = { val serde = getCatalystRowSerDe(sourceSqlType) val avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(sourceSqlType, structName, recordNamespace) - val nullable = AvroSchemaUtils.resolveNullableSchema(avroSchema) != avroSchema + val nullable = AvroSchemaUtils.getNonNullTypeFromUnion(avroSchema) != avroSchema val converter = AvroConversionUtils.createInternalRowToAvroConverter(sourceSqlType, avroSchema, nullable) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala index 35e6c683d139b..97dfb1b3d5c17 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala @@ -91,7 +91,7 @@ object HoodieSparkUtils extends SparkAdapterSupport with SparkVersionsSupport wi // making Spark deserialize its internal representation [[InternalRow]] into [[Row]] for subsequent conversion // (and back) val sameSchema = writerAvroSchema.equals(readerAvroSchema) - val nullable = AvroSchemaUtils.resolveNullableSchema(writerAvroSchema) != writerAvroSchema + val nullable = AvroSchemaUtils.getNonNullTypeFromUnion(writerAvroSchema) != writerAvroSchema // NOTE: We have to serialize Avro schema, and then subsequently parse it on the executor node, since Spark // serializer is not able to digest it @@ -160,7 +160,7 @@ object HoodieSparkUtils extends SparkAdapterSupport with SparkVersionsSupport wi // making Spark deserialize its internal representation [[InternalRow]] into [[Row]] for subsequent conversion // (and back) val sameSchema = writerAvroSchema.equals(readerAvroSchema) - val nullable = AvroSchemaUtils.resolveNullableSchema(writerAvroSchema) != writerAvroSchema + val nullable = AvroSchemaUtils.getNonNullTypeFromUnion(writerAvroSchema) != writerAvroSchema // NOTE: We have to serialize Avro schema, and then subsequently parse it on the executor node, since Spark // serializer is not able to digest it diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index 950f5d1daf94f..271f9c9aafc4e 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -33,6 +33,9 @@ import org.apache.hudi.common.util.collection.{CachingIterator, ClosableIterator import org.apache.hudi.io.storage.{HoodieSparkFileReaderFactory, HoodieSparkParquetReader} import org.apache.hudi.storage.{HoodieStorage, StorageConfiguration, StoragePath} import org.apache.hudi.util.CloseableInternalRowIterator + +import org.apache.parquet.avro.AvroSchemaConverter +import org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter import org.apache.spark.sql.HoodieInternalRowUtils import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.JoinedRow @@ -70,27 +73,36 @@ class SparkFileFormatInternalRowReaderContext(baseFileReader: SparkColumnarFileR override def getFileRecordIterator(filePath: StoragePath, start: Long, length: Long, - dataSchema: Schema, + dataSchema: Schema, // dataSchema refers to table schema in most cases(non log file reads). requiredSchema: Schema, storage: HoodieStorage): ClosableIterator[InternalRow] = { val hasRowIndexField = AvroSchemaUtils.containsFieldInSchema(requiredSchema, ROW_INDEX_TEMPORARY_COLUMN_NAME) if (hasRowIndexField) { assert(getRecordContext.supportsParquetRowIndex()) } - val structType = HoodieInternalRowUtils.getCachedSchema(requiredSchema) if (FSUtils.isLogFile(filePath)) { // TODO: introduce pk filter in log file reader new HoodieSparkFileReaderFactory(storage).newParquetFileReader(filePath) - .asInstanceOf[HoodieSparkParquetReader].getUnsafeRowIterator(structType).asInstanceOf[ClosableIterator[InternalRow]] + .asInstanceOf[HoodieSparkParquetReader].getUnsafeRowIterator(requiredSchema).asInstanceOf[ClosableIterator[InternalRow]] } else { + val structType = HoodieInternalRowUtils.getCachedSchema(requiredSchema) // partition value is empty because the spark parquet reader will append the partition columns to // each row if they are given. That is the only usage of the partition values in the reader. val fileInfo = sparkAdapter.getSparkPartitionedFileUtils .createPartitionedFile(InternalRow.empty, filePath, start, length) val (readSchema, readFilters) = getSchemaAndFiltersForRead(structType, hasRowIndexField) + + // Convert Avro dataSchema to Parquet MessageType for timestamp precision conversion + val tableSchemaOpt = if (dataSchema != null) { + val hadoopConf = storage.getConf.unwrapAs(classOf[Configuration]) + val parquetSchema = getAvroSchemaConverter(hadoopConf).convert(dataSchema) + org.apache.hudi.common.util.Option.of(parquetSchema) + } else { + org.apache.hudi.common.util.Option.empty[org.apache.parquet.schema.MessageType]() + } new CloseableInternalRowIterator(baseFileReader.read(fileInfo, readSchema, StructType(Seq.empty), getSchemaHandler.getInternalSchemaOpt, - readFilters, storage.getConf.asInstanceOf[StorageConfiguration[Configuration]])) + readFilters, storage.getConf.asInstanceOf[StorageConfiguration[Configuration]], tableSchemaOpt)) } } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/SparkColumnarFileReader.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/SparkColumnarFileReader.scala index 7472b22ea1c6a..e9d2778e0e351 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/SparkColumnarFileReader.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/SparkColumnarFileReader.scala @@ -23,6 +23,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hudi.common.util import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.storage.StorageConfiguration +import org.apache.parquet.schema.MessageType import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType @@ -37,6 +38,7 @@ trait SparkColumnarFileReader extends Serializable { * @param internalSchemaOpt option of internal schema for schema.on.read * @param filters filters for data skipping. Not guaranteed to be used; the spark plan will also apply the filters. * @param storageConf the hadoop conf + * @param tableSchemaOpt option of table schema for timestamp precision conversion * @return iterator of rows read from the file output type says [[InternalRow]] but could be [[ColumnarBatch]] */ def read(file: PartitionedFile, @@ -44,5 +46,6 @@ trait SparkColumnarFileReader extends Serializable { partitionSchema: StructType, internalSchemaOpt: util.Option[InternalSchema], filters: Seq[Filter], - storageConf: StorageConfiguration[Configuration]): Iterator[InternalRow] + storageConf: StorageConfiguration[Configuration], + tableSchemaOpt: util.Option[MessageType] = util.Option.empty()): Iterator[InternalRow] } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala index e3523b75ce679..abf805a430f33 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala @@ -24,7 +24,7 @@ import org.apache.parquet.hadoop.metadata.FileMetaData import org.apache.spark.sql.HoodieSchemaUtils import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.expressions.{ArrayTransform, Attribute, Cast, CreateNamedStruct, CreateStruct, Expression, GetStructField, LambdaFunction, Literal, MapEntries, MapFromEntries, NamedLambdaVariable, UnsafeProjection} -import org.apache.spark.sql.types.{ArrayType, DataType, DateType, DecimalType, DoubleType, FloatType, IntegerType, LongType, MapType, StringType, StructField, StructType} +import org.apache.spark.sql.types.{ArrayType, DataType, DateType, DecimalType, DoubleType, FloatType, IntegerType, LongType, MapType, StringType, StructField, StructType, TimestampNTZType} object HoodieParquetFileFormatHelper { @@ -58,6 +58,9 @@ object HoodieParquetFileFormatHelper { def isDataTypeEqual(requiredType: DataType, fileType: DataType): Boolean = (requiredType, fileType) match { case (requiredType, fileType) if requiredType == fileType => true + // prevent illegal cast + case (TimestampNTZType, LongType) => true + case (ArrayType(rt, _), ArrayType(ft, _)) => // Do not care about nullability as schema evolution require fields to be nullable isDataTypeEqual(rt, ft) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetReadSupport.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetReadSupport.scala index d55b8c8d25d54..40dea22c9fa22 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetReadSupport.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetReadSupport.scala @@ -24,7 +24,7 @@ import org.apache.hudi.common.util.ValidationUtils import org.apache.parquet.hadoop.api.InitContext import org.apache.parquet.hadoop.api.ReadSupport.ReadContext -import org.apache.parquet.schema.{GroupType, MessageType, Type, Types} +import org.apache.parquet.schema.{GroupType, MessageType, SchemaRepair, Type, Types} import org.apache.spark.sql.catalyst.util.RebaseDateTime.RebaseSpec import java.time.ZoneId @@ -34,17 +34,24 @@ import scala.collection.JavaConverters._ class HoodieParquetReadSupport( convertTz: Option[ZoneId], enableVectorizedReader: Boolean, + val enableTimestampFieldRepair: Boolean, datetimeRebaseSpec: RebaseSpec, - int96RebaseSpec: RebaseSpec) + int96RebaseSpec: RebaseSpec, + tableSchemaOpt: org.apache.hudi.common.util.Option[org.apache.parquet.schema.MessageType] = org.apache.hudi.common.util.Option.empty()) extends ParquetReadSupport(convertTz, enableVectorizedReader, datetimeRebaseSpec, int96RebaseSpec) with SparkAdapterSupport { override def init(context: InitContext): ReadContext = { val readContext = super.init(context) - val requestedParquetSchema = readContext.getRequestedSchema + // repair is needed here because this is the schema that is used by the reader to decide what + // conversions are necessary + val requestedParquetSchema = if (enableTimestampFieldRepair) { + SchemaRepair.repairLogicalTypes(readContext.getRequestedSchema, tableSchemaOpt) + } else { + readContext.getRequestedSchema + } val trimmedParquetSchema = HoodieParquetReadSupport.trimParquetSchema(requestedParquetSchema, context.getFileSchema) new ReadContext(trimmedParquetSchema, readContext.getReadSupportMetadata) } - } object HoodieParquetReadSupport { diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/MultipleColumnarFileFormatReader.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/MultipleColumnarFileFormatReader.scala index 5d4415a9e0c56..afe0c36ad1fac 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/MultipleColumnarFileFormatReader.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/MultipleColumnarFileFormatReader.scala @@ -41,16 +41,18 @@ class MultipleColumnarFileFormatReader(parquetReader: SparkColumnarFileReader, o * @param internalSchemaOpt option of internal schema for schema.on.read * @param filters filters for data skipping. Not guaranteed to be used; the spark plan will also apply the filters. * @param storageConf the hadoop conf + * @param tableSchemaOpt option of table schema for timestamp precision conversion fix. * @return iterator of rows read from the file output type says [[InternalRow]] but could be [[ColumnarBatch]] */ - override def read(file: PartitionedFile, requiredSchema: StructType, partitionSchema: StructType, internalSchemaOpt: util.Option[InternalSchema], filters: Seq[Filter], storageConf: StorageConfiguration[Configuration]): Iterator[InternalRow] = { + override def read(file: PartitionedFile, requiredSchema: StructType, partitionSchema: StructType, internalSchemaOpt: util.Option[InternalSchema], filters: Seq[Filter], + storageConf: StorageConfiguration[Configuration], tableSchemaOpt: util.Option[org.apache.parquet.schema.MessageType]): Iterator[InternalRow] = { val filePath = sparkAdapter.getSparkPartitionedFileUtils.getPathFromPartitionedFile(file) val fileFormat = HoodieFileFormat.fromFileExtension(filePath.getFileExtension) fileFormat match { case HoodieFileFormat.PARQUET => - parquetReader.read(file, requiredSchema, partitionSchema, internalSchemaOpt, filters, storageConf) + parquetReader.read(file, requiredSchema, partitionSchema, internalSchemaOpt, filters, storageConf, tableSchemaOpt) case HoodieFileFormat.ORC => - orcReader.read(file, requiredSchema, partitionSchema, internalSchemaOpt, filters, storageConf) + orcReader.read(file, requiredSchema, partitionSchema, internalSchemaOpt, filters, storageConf, tableSchemaOpt) case _ => throw new IllegalArgumentException(s"Unsupported file format for file: $filePath") } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java index b17bae88a18b7..e0068e1f06994 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java @@ -153,7 +153,7 @@ void testTimestampBasedKeyGenerator() throws IOException { // timezone is GMT+8:00, createTime is BigDecimal BigDecimal decimal = new BigDecimal("1578283932000.0001"); Conversions.DecimalConversion conversion = new Conversions.DecimalConversion(); - Schema resolvedNullableSchema = AvroSchemaUtils.resolveNullableSchema(schema.getField("createTimeDecimal").schema()); + Schema resolvedNullableSchema = AvroSchemaUtils.getNonNullTypeFromUnion(schema.getField("createTimeDecimal").schema()); GenericFixed avroDecimal = conversion.toFixed(decimal, resolvedNullableSchema, LogicalTypes.decimal(20, 4)); baseRecord.put("createTimeDecimal", avroDecimal); properties = getBaseKeyConfig("createTimeDecimal", "EPOCHMILLISECONDS", "yyyy-MM-dd hh", "GMT+8:00", null); @@ -223,7 +223,7 @@ void testTimestampBasedKeyGenerator() throws IOException { // Timestamp field is in decimal type, with `EPOCHMICROSECONDS` timestamp type in the key generator decimal = new BigDecimal("1578283932123456.0001"); - resolvedNullableSchema = AvroSchemaUtils.resolveNullableSchema( + resolvedNullableSchema = AvroSchemaUtils.getNonNullTypeFromUnion( schema.getField("createTimeDecimal").schema()); avroDecimal = conversion.toFixed(decimal, resolvedNullableSchema, LogicalTypes.decimal(20, 4)); baseRecord.put("createTimeDecimal", avroDecimal); diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/AvroRecordContext.java b/hudi-common/src/main/java/org/apache/hudi/avro/AvroRecordContext.java index 6d3de9ad1ed64..cbb29034dc626 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/AvroRecordContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/AvroRecordContext.java @@ -74,7 +74,7 @@ public static Object getFieldValueFromIndexedRecord( String[] path = fieldName.split("\\."); for (int i = 0; i < path.length; i++) { if (currentSchema.isUnion()) { - currentSchema = AvroSchemaUtils.resolveNullableSchema(currentSchema); + currentSchema = AvroSchemaUtils.getNonNullTypeFromUnion(currentSchema); } Schema.Field field = currentSchema.getField(path[i]); if (field == null) { diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaComparatorForRecordProjection.java b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaComparatorForRecordProjection.java index 73ea743c050a8..2a75936e93a43 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaComparatorForRecordProjection.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaComparatorForRecordProjection.java @@ -23,7 +23,7 @@ import java.util.List; -import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema; +import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; public class AvroSchemaComparatorForRecordProjection extends AvroSchemaComparatorForSchemaEvolution { @@ -41,7 +41,7 @@ protected boolean schemaEqualsInternal(Schema s1, Schema s2) { if (s1 == null || s2 == null) { return false; } - return super.schemaEqualsInternal(resolveNullableSchema(s1), resolveNullableSchema(s2)); + return super.schemaEqualsInternal(getNonNullTypeFromUnion(s1), getNonNullTypeFromUnion(s2)); } @Override 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 4e51e43ebde1e..a17b0472e7c0f 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 @@ -180,8 +180,8 @@ public static boolean isStrictProjectionOf(Schema sourceSchema, Schema targetSch private static boolean isAtomicTypeProjectable(Schema source, Schema target) { // ignore nullability for projectable checking - source = resolveNullableSchema(source); - target = resolveNullableSchema(target); + source = getNonNullTypeFromUnion(source); + target = getNonNullTypeFromUnion(target); if (source.getType() == Schema.Type.ENUM && target.getType() == Schema.Type.STRING || source.getType() == Schema.Type.STRING && target.getType() == Schema.Type.ENUM) { return true; @@ -238,7 +238,7 @@ public static Option findNestedFieldSchema(Schema schema, String fieldNa String[] parts = fieldName.split("\\."); for (String part : parts) { - Schema.Field foundField = resolveNullableSchema(schema).getField(part); + Schema.Field foundField = getNonNullTypeFromUnion(schema).getField(part); if (foundField == null) { if (allowsMissingField) { return Option.empty(); @@ -247,7 +247,7 @@ public static Option findNestedFieldSchema(Schema schema, String fieldNa } schema = foundField.schema(); } - return Option.of(resolveNullableSchema(schema)); + return Option.of(getNonNullTypeFromUnion(schema)); } public static Option findNestedFieldType(Schema schema, String fieldName) { @@ -274,7 +274,7 @@ public static Option findNestedField(Schema schema, String fieldNa private static Option findNestedField(Schema schema, String[] fieldParts, int index) { if (schema.getType().equals(Schema.Type.UNION)) { - Option notUnion = findNestedField(resolveNullableSchema(schema), fieldParts, index); + Option notUnion = findNestedField(getNonNullTypeFromUnion(schema), fieldParts, index); if (!notUnion.isPresent()) { return Option.empty(); } @@ -302,7 +302,7 @@ private static Option findNestedField(Schema schema, String[] fiel boolean isUnion = false; if (foundSchema.getType().equals(Schema.Type.UNION)) { isUnion = true; - foundSchema = resolveNullableSchema(foundSchema); + foundSchema = getNonNullTypeFromUnion(foundSchema); } Schema newSchema = createNewSchemaFromFieldsWithReference(foundSchema, Collections.singletonList(nestedPart.get())); return Option.of(createNewSchemaField(foundField.name(), isUnion ? createNullableSchema(newSchema) : newSchema, foundField.doc(), foundField.defaultVal())); @@ -424,7 +424,7 @@ public static boolean areSchemasProjectionEquivalent(Schema schema1, Schema sche * data schema metadata where possible */ public static Schema pruneDataSchema(Schema dataSchema, Schema requiredSchema, Set mandatoryFields) { - Schema prunedDataSchema = pruneDataSchemaInternal(resolveNullableSchema(dataSchema), resolveNullableSchema(requiredSchema), mandatoryFields); + Schema prunedDataSchema = pruneDataSchemaInternal(getNonNullTypeFromUnion(dataSchema), getNonNullTypeFromUnion(requiredSchema), mandatoryFields); if (dataSchema.isNullable() && !prunedDataSchema.isNullable()) { return createNullableSchema(prunedDataSchema); } @@ -494,7 +494,7 @@ public static Schema resolveUnionSchema(Schema schema, String fieldSchemaFullNam List innerTypes = schema.getTypes(); if (innerTypes.size() == 2 && isNullable(schema)) { // this is a basic nullable field so handle it more efficiently - return resolveNullableSchema(schema); + return getNonNullTypeFromUnion(schema); } Schema nonNullType = @@ -528,7 +528,7 @@ public static boolean isNullable(Schema schema) { * Resolves typical Avro's nullable schema definition: {@code Union(Schema.Type.NULL, )}, * decomposing union and returning the target non-null type */ - public static Schema resolveNullableSchema(Schema schema) { + public static Schema getNonNullTypeFromUnion(Schema schema) { if (schema.getType() != Schema.Type.UNION) { return schema; } 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 2dd40387c1d0e..66c2a19e55c01 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 @@ -100,7 +100,7 @@ import static org.apache.hudi.avro.AvroSchemaUtils.createNewSchemaFromFieldsWithReference; import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema; import static org.apache.hudi.avro.AvroSchemaUtils.isNullable; -import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema; +import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; import static org.apache.hudi.common.util.ValidationUtils.checkState; @@ -412,7 +412,7 @@ public static Schema makeFieldNonNull(Schema schema, String fieldName, Object fi .stream() .map(field -> { if (Objects.equals(field.name(), fieldName)) { - return createNewSchemaField(field.name(), AvroSchemaUtils.resolveNullableSchema(field.schema()), field.doc(), fieldDefaultValue); + return createNewSchemaField(field.name(), AvroSchemaUtils.getNonNullTypeFromUnion(field.schema()), field.doc(), fieldDefaultValue); } else { return createNewSchemaField(field); } @@ -810,7 +810,7 @@ public static Schema getNestedFieldSchemaFromRecord(GenericRecord record, String Object val = valueNode.get(part); if (i == parts.length - 1) { - return resolveNullableSchema(valueNode.getSchema().getField(part).schema()); + return getNonNullTypeFromUnion(valueNode.getSchema().getField(part).schema()); } else { if (!(val instanceof GenericRecord)) { throw new HoodieException("Cannot find a record at part value :" + part); @@ -836,11 +836,11 @@ public static Schema getNestedFieldSchemaFromWriteSchema(Schema writeSchema, Str String part = parts[i]; try { // Resolve nullable/union schema to the actual schema - currentSchema = resolveNullableSchema(currentSchema.getField(part).schema()); + currentSchema = getNonNullTypeFromUnion(currentSchema.getField(part).schema()); if (i == parts.length - 1) { // Return the schema for the final part - return resolveNullableSchema(currentSchema); + return getNonNullTypeFromUnion(currentSchema); } } catch (Exception e) { throw new HoodieException("Failed to get schema. Not a valid field name: " + fieldName); @@ -881,7 +881,7 @@ public static Object convertValueForSpecificDataTypes(Schema fieldSchema, return null; } - return convertValueForAvroLogicalTypes(resolveNullableSchema(fieldSchema), fieldValue, consistentLogicalTimestampEnabled); + return convertValueForAvroLogicalTypes(getNonNullTypeFromUnion(fieldSchema), fieldValue, consistentLogicalTimestampEnabled); } /** @@ -1208,7 +1208,9 @@ public static Object rewritePrimaryType(Object oldValue, Schema oldSchema, Schem return oldValue; case LONG: if (oldSchema.getLogicalType() != newSchema.getLogicalType()) { - if (oldSchema.getLogicalType() instanceof LogicalTypes.TimestampMillis) { + if (oldSchema.getLogicalType() == null || newSchema.getLogicalType() == null) { + return oldValue; + } else if (oldSchema.getLogicalType() instanceof LogicalTypes.TimestampMillis) { if (newSchema.getLogicalType() instanceof LogicalTypes.TimestampMicros) { return DateTimeUtils.millisToMicros((Long) oldValue); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java index 25786ad6cf60c..a53c072fc141d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java @@ -96,6 +96,7 @@ public abstract class HoodieReaderContext { // the default iterator mode is engine-specific record mode private IteratorMode iteratorMode = IteratorMode.ENGINE_RECORD; protected final HoodieConfig hoodieReaderConfig; + private boolean enableLogicalTimestampFieldRepair = true; protected HoodieReaderContext(StorageConfiguration storageConfiguration, HoodieTableConfig tableConfig, @@ -145,6 +146,10 @@ public String getTablePath() { return tablePath; } + public void setEnableLogicalTimestampFieldRepair(boolean enableLogicalTimestampFieldRepair) { + this.enableLogicalTimestampFieldRepair = enableLogicalTimestampFieldRepair; + } + public void setTablePath(String tablePath) { this.tablePath = tablePath; } @@ -188,6 +193,10 @@ public boolean getNeedsBootstrapMerge() { return needsBootstrapMerge; } + public boolean enableLogicalTimestampFieldRepair() { + return enableLogicalTimestampFieldRepair; + } + public void setNeedsBootstrapMerge(boolean needsBootstrapMerge) { this.needsBootstrapMerge = needsBootstrapMerge; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/PartitionPathParser.java b/hudi-common/src/main/java/org/apache/hudi/common/table/PartitionPathParser.java index 3b846330e63bf..88ddcea7d39bc 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/PartitionPathParser.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/PartitionPathParser.java @@ -33,7 +33,7 @@ import java.time.LocalDateTime; import java.time.ZoneOffset; -import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema; +import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; public class PartitionPathParser { public static final String DEPRECATED_DEFAULT_PARTITION_PATH = "default"; @@ -62,7 +62,7 @@ private static Object[] getPartitionValues(String[] partitionFields, String partitionField = partitionFields[i]; Schema.Field field = schema.getField(partitionField); // if the field is not present in the schema, we assume it is a string - Schema fieldSchema = field == null ? Schema.create(Schema.Type.STRING) : resolveNullableSchema(field.schema()); + Schema fieldSchema = field == null ? Schema.create(Schema.Type.STRING) : getNonNullTypeFromUnion(field.schema()); LogicalType logicalType = fieldSchema.getLogicalType(); if (isTimeBasedLogicalType(logicalType)) { if (hasDateField) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java index d8e44771dcc0f..442f2d2e5703f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java @@ -45,6 +45,7 @@ import org.apache.avro.io.DecoderFactory; import org.apache.avro.io.Encoder; import org.apache.avro.io.EncoderFactory; +import org.apache.parquet.schema.AvroSchemaRepair; import javax.annotation.Nonnull; @@ -136,7 +137,7 @@ protected ClosableIterator> deserializeRecords(byte[] conten checkState(this.readerSchema != null, "Reader's schema has to be non-null"); checkArgument(type != HoodieRecordType.SPARK, "Not support read avro to spark record"); // TODO AvroSparkReader need - RecordIterator iterator = RecordIterator.getInstance(this, content); + RecordIterator iterator = RecordIterator.getInstance(this, content, true); return new CloseableMappingIterator<>(iterator, data -> (HoodieRecord) new HoodieAvroIndexedRecord(data)); } @@ -163,7 +164,7 @@ protected ClosableIterator> deserializeRecords( @Override protected ClosableIterator deserializeRecords(HoodieReaderContext readerContext, byte[] content) throws IOException { checkState(this.readerSchema != null, "Reader's schema has to be non-null"); - RecordIterator iterator = RecordIterator.getInstance(this, content); + RecordIterator iterator = RecordIterator.getInstance(this, content, readerContext.enableLogicalTimestampFieldRepair()); return new CloseableMappingIterator<>(iterator, data -> readerContext.getRecordContext().convertAvroRecord(data)); } @@ -176,7 +177,7 @@ private static class RecordIterator implements ClosableIterator { private int totalRecords = 0; private int readRecords = 0; - private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content) throws IOException { + private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content, boolean enableLogicalTimestampFieldRepair) throws IOException { this.content = content; this.dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(this.content))); @@ -187,16 +188,21 @@ private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content) this.totalRecords = this.dis.readInt(); } - if (recordNeedsRewriteForExtendedAvroTypePromotion(writerSchema, readerSchema)) { - this.reader = new GenericDatumReader<>(writerSchema, writerSchema); + // writer schema could refer to table schema. + // avoid this for MDT for sure. + // and for tables having no logical ts column. + Schema repairedWriterSchema = enableLogicalTimestampFieldRepair ? + AvroSchemaRepair.repairLogicalTypes(writerSchema, readerSchema) : writerSchema; + if (recordNeedsRewriteForExtendedAvroTypePromotion(repairedWriterSchema, readerSchema)) { + this.reader = new GenericDatumReader<>(repairedWriterSchema, repairedWriterSchema); this.promotedSchema = Option.of(readerSchema); } else { - this.reader = new GenericDatumReader<>(writerSchema, readerSchema); + this.reader = new GenericDatumReader<>(repairedWriterSchema, readerSchema); } } - public static RecordIterator getInstance(HoodieAvroDataBlock dataBlock, byte[] content) throws IOException { - return new RecordIterator(dataBlock.readerSchema, dataBlock.getSchemaFromHeader(), content); + public static RecordIterator getInstance(HoodieAvroDataBlock dataBlock, byte[] content, boolean enableLogicalTimestampFieldRepair) throws IOException { + return new RecordIterator(dataBlock.readerSchema, dataBlock.getSchemaFromHeader(), content, enableLogicalTimestampFieldRepair); } @Override @@ -272,11 +278,12 @@ private StreamingRecordIterator(Schema readerSchema, Schema writerSchema, Seekab this.totalRecords = this.inputStream.readInt(); } - if (recordNeedsRewriteForExtendedAvroTypePromotion(writerSchema, readerSchema)) { - this.reader = new GenericDatumReader<>(writerSchema, writerSchema); + Schema repairedWriterSchema = AvroSchemaRepair.repairLogicalTypes(writerSchema, readerSchema); + if (recordNeedsRewriteForExtendedAvroTypePromotion(repairedWriterSchema, readerSchema)) { + this.reader = new GenericDatumReader<>(repairedWriterSchema, repairedWriterSchema); this.promotedSchema = Option.of(readerSchema); } else { - this.reader = new GenericDatumReader<>(writerSchema, readerSchema); + this.reader = new GenericDatumReader<>(repairedWriterSchema, readerSchema); } this.buffer = ByteBuffer.allocate(Math.min(bufferSize, Math.toIntExact(contentLocation.getBlockSize()))); 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 acd229f3ced89..f4dd2d8af167c 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 @@ -109,7 +109,7 @@ private static void collectColNamesFromAvroSchema(Schema schema, Deque v return; case UNION: - collectColNamesFromAvroSchema(AvroSchemaUtils.resolveNullableSchema(schema), visited, resultSet); + collectColNamesFromAvroSchema(AvroSchemaUtils.getNonNullTypeFromUnion(schema), visited, resultSet); return; case ARRAY: @@ -132,7 +132,7 @@ private static void collectColNamesFromAvroSchema(Schema schema, Deque v } private static void addFullNameIfLeafNode(Schema schema, String name, Deque visited, List resultSet) { - addFullNameIfLeafNode(AvroSchemaUtils.resolveNullableSchema(schema).getType(), name, visited, resultSet); + addFullNameIfLeafNode(AvroSchemaUtils.getNonNullTypeFromUnion(schema).getType(), name, visited, resultSet); } private static void addFullNameIfLeafNode(Schema.Type type, String name, Deque visited, List resultSet) { 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 cfa4c921f35da..98bdf563a05fa 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 @@ -158,7 +158,7 @@ import java.util.stream.Stream; import static java.util.stream.Collectors.toList; -import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema; +import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; import static org.apache.hudi.avro.HoodieAvroUtils.addMetadataFields; import static org.apache.hudi.avro.HoodieAvroUtils.projectSchema; import static org.apache.hudi.common.config.HoodieCommonConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES; @@ -285,7 +285,7 @@ public static Map> collectColumnRa // with the values from this record targetFields.forEach(fieldNameFieldPair -> { String fieldName = fieldNameFieldPair.getKey(); - Schema fieldSchema = resolveNullableSchema(fieldNameFieldPair.getValue().schema()); + Schema fieldSchema = getNonNullTypeFromUnion(fieldNameFieldPair.getValue().schema()); ColumnStats colStats = allColumnStats.computeIfAbsent(fieldName, ignored -> new ColumnStats(getValueMetadata(fieldSchema, indexVersion))); Object fieldValue = collectColumnRangeFieldValue(record, colStats.valueMetadata, fieldName, fieldSchema, recordSchema, properties); @@ -1856,7 +1856,7 @@ public static Comparable coerceToComparable(Schema schema, Object val) { switch (schema.getType()) { case UNION: // TODO we need to handle unions in general case as well - return coerceToComparable(resolveNullableSchema(schema), val); + return coerceToComparable(getNonNullTypeFromUnion(schema), val); case FIXED: case BYTES: @@ -1989,7 +1989,7 @@ private static Double castToDouble(Object val) { } public static boolean isColumnTypeSupported(Schema schema, Option recordType, HoodieIndexVersion indexVersion) { - Schema schemaToCheck = resolveNullableSchema(schema); + Schema schemaToCheck = getNonNullTypeFromUnion(schema); if (indexVersion.lowerThan(HoodieIndexVersion.V2)) { return isColumnTypeSupportedV1(schemaToCheck, recordType); } 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 index cebcb2aac3bab..043af52db6643 100644 --- a/hudi-common/src/main/java/org/apache/hudi/stats/ValueMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/stats/ValueMetadata.java @@ -31,7 +31,7 @@ import java.io.Serializable; -import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema; +import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; 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; @@ -241,7 +241,7 @@ public static ValueMetadata getValueMetadata(Schema fieldSchema, HoodieIndexVers if (fieldSchema == null) { throw new IllegalArgumentException("Field schema cannot be null"); } - Schema valueSchema = resolveNullableSchema(fieldSchema); + Schema valueSchema = getNonNullTypeFromUnion(fieldSchema); ValueType valueType = ValueType.fromSchema(valueSchema); if (valueType == ValueType.V1) { throw new IllegalArgumentException("Unsupported logical type for: " + valueSchema.getLogicalType()); 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 index 9b6ec143f39ae..c58de6d5dc285 100644 --- a/hudi-common/src/main/java/org/apache/hudi/stats/ValueType.java +++ b/hudi-common/src/main/java/org/apache/hudi/stats/ValueType.java @@ -312,7 +312,7 @@ public static ValueType fromSchema(Schema schema) { } throw new IllegalArgumentException("Unsupported logical type for Fixed: " + schema.getLogicalType()); case UNION: - return fromSchema(AvroSchemaUtils.resolveNullableSchema(schema)); + return fromSchema(AvroSchemaUtils.getNonNullTypeFromUnion(schema)); default: throw new IllegalArgumentException("Unsupported type: " + schema.getType()); } diff --git a/hudi-common/src/main/java/org/apache/parquet/schema/AvroSchemaRepair.java b/hudi-common/src/main/java/org/apache/parquet/schema/AvroSchemaRepair.java new file mode 100644 index 0000000000000..4d7cb8e938d48 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/parquet/schema/AvroSchemaRepair.java @@ -0,0 +1,242 @@ +/* + * 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.avro.AvroSchemaCache; +import org.apache.hudi.avro.AvroSchemaUtils; + +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; + +import java.util.ArrayList; +import java.util.List; + +public class AvroSchemaRepair { + public static Schema repairLogicalTypes(Schema fileSchema, Schema tableSchema) { + Schema repairedSchema = repairAvroSchema(fileSchema, tableSchema); + if (repairedSchema != fileSchema) { + return AvroSchemaCache.intern(repairedSchema); + } + return fileSchema; + } + + /** + * Performs schema repair on a schema, handling nullable unions. + */ + private static Schema repairAvroSchema(Schema fileSchema, Schema tableSchema) { + // Always resolve nullable schemas first (returns unchanged if not a union) + Schema nonNullFileSchema = AvroSchemaUtils.getNonNullTypeFromUnion(fileSchema); + Schema nonNullTableSchema = AvroSchemaUtils.getNonNullTypeFromUnion(tableSchema); + + // Perform repair on the non-null types + Schema nonNullRepairedSchema = repairAvroSchemaNonNull(nonNullFileSchema, nonNullTableSchema); + + // If nothing changed, return the original schema + if (nonNullRepairedSchema == nonNullFileSchema) { + return fileSchema; + } + + // If the original was a union, wrap the repaired schema back in a nullable union + if (fileSchema.getType() == Schema.Type.UNION) { + return AvroSchemaUtils.createNullableSchema(nonNullRepairedSchema); + } + + return nonNullRepairedSchema; + } + + /** + * Repairs non-nullable schemas (after unions have been resolved). + */ + private static Schema repairAvroSchemaNonNull(Schema fileSchema, Schema tableSchema) { + // If schemas are already equal, nothing to repair + if (fileSchema.equals(tableSchema)) { + return fileSchema; + } + + // If types are different, no repair can be done + if (fileSchema.getType() != tableSchema.getType()) { + return fileSchema; + } + + // Handle record types (nested structs) + if (fileSchema.getType() == Schema.Type.RECORD) { + return repairRecord(fileSchema, tableSchema); + } + + // Handle array types + if (fileSchema.getType() == Schema.Type.ARRAY) { + Schema repairedElementSchema = repairAvroSchema(fileSchema.getElementType(), tableSchema.getElementType()); + // If element didn't change, return original array schema + if (repairedElementSchema == fileSchema.getElementType()) { + return fileSchema; + } + return Schema.createArray(repairedElementSchema); + } + + // Handle map types + if (fileSchema.getType() == Schema.Type.MAP) { + Schema repairedValueSchema = repairAvroSchema(fileSchema.getValueType(), tableSchema.getValueType()); + // If value didn't change, return original map schema + if (repairedValueSchema == fileSchema.getValueType()) { + return fileSchema; + } + return Schema.createMap(repairedValueSchema); + } + + // Check primitive if we need to repair + if (needsLogicalTypeRepair(fileSchema, tableSchema)) { + // If we need to repair, return the table schema + return tableSchema; + } + + // Default: return file schema + return fileSchema; + } + + /** + * Quick check if a logical type repair is needed (no allocations). + */ + private static boolean needsLogicalTypeRepair(Schema fileSchema, Schema tableSchema) { + if (fileSchema.getType() != Schema.Type.LONG || tableSchema.getType() != Schema.Type.LONG) { + return false; + } + + LogicalType fileSchemaLogicalType = fileSchema.getLogicalType(); + LogicalType tableSchemaLogicalType = tableSchema.getLogicalType(); + + // if file scheam has no logical type, and the table has a local timestamp, then we need to repair + if (fileSchemaLogicalType == null) { + return tableSchemaLogicalType instanceof LogicalTypes.LocalTimestampMillis + || tableSchemaLogicalType instanceof LogicalTypes.LocalTimestampMicros; + } + + // if file schema is timestamp-micros, and the table is timestamp-millis, then we need to repair + return fileSchemaLogicalType instanceof LogicalTypes.TimestampMicros + && tableSchemaLogicalType instanceof LogicalTypes.TimestampMillis; + } + + /** + * Performs record repair, returning the original schema if nothing changed. + */ + private static Schema repairRecord(Schema fileSchema, Schema tableSchema) { + List fields = fileSchema.getFields(); + + // First pass: find the first field that changes + int firstChangedIndex = -1; + Schema firstRepairedSchema = null; + + for (int i = 0; i < fields.size(); i++) { + Schema.Field requestedField = fields.get(i); + Schema.Field tableField = tableSchema.getField(requestedField.name()); + if (tableField != null) { + Schema repairedSchema = repairAvroSchema(requestedField.schema(), tableField.schema()); + if (repairedSchema != requestedField.schema()) { + firstChangedIndex = i; + firstRepairedSchema = repairedSchema; + break; + } + } + } + + // If nothing changed, return the original schema + if (firstChangedIndex == -1) { + return fileSchema; + } + + // Second pass: build the new schema with repaired fields + List repairedFields = new ArrayList<>(fields.size()); + + // Copy all fields before the first changed field + for (int i = 0; i < firstChangedIndex; i++) { + Schema.Field field = fields.get(i); + // Must create new Field since they cannot be reused + repairedFields.add(new Schema.Field( + field.name(), + field.schema(), + field.doc(), + field.defaultVal() + )); + } + + // Add the first changed field (using cached repaired schema) + Schema.Field firstChangedField = fields.get(firstChangedIndex); + repairedFields.add(new Schema.Field( + firstChangedField.name(), + firstRepairedSchema, + firstChangedField.doc(), + firstChangedField.defaultVal() + )); + + // Process remaining fields + for (int i = firstChangedIndex + 1; i < fields.size(); i++) { + Schema.Field requestedField = fields.get(i); + Schema.Field tableField = tableSchema.getField(requestedField.name()); + Schema repairedSchema; + + if (tableField != null) { + repairedSchema = repairAvroSchema(requestedField.schema(), tableField.schema()); + } else { + repairedSchema = requestedField.schema(); + } + + // Must create new Field since they cannot be reused + repairedFields.add(new Schema.Field( + requestedField.name(), + repairedSchema, + requestedField.doc(), + requestedField.defaultVal() + )); + } + + return Schema.createRecord( + fileSchema.getName(), + fileSchema.getDoc(), + fileSchema.getNamespace(), + fileSchema.isError(), + repairedFields + ); + } + + public static boolean hasTimestampMillisField(Schema tableSchema) { + switch (tableSchema.getType()) { + case RECORD: + for (Schema.Field field : tableSchema.getFields()) { + if (hasTimestampMillisField(field.schema())) { + return true; + } + } + return false; + + case ARRAY: + return hasTimestampMillisField(tableSchema.getElementType()); + + case MAP: + return hasTimestampMillisField(tableSchema.getValueType()); + + case UNION: + return hasTimestampMillisField(AvroSchemaUtils.getNonNullTypeFromUnion(tableSchema)); + + default: + return tableSchema.getType() == Schema.Type.LONG + && (tableSchema.getLogicalType() instanceof LogicalTypes.TimestampMillis || tableSchema.getLogicalType() instanceof LogicalTypes.LocalTimestampMillis); + } + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java b/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java index b655f9bfdc8ee..050d2924de7c7 100644 --- a/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java @@ -113,7 +113,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema; +import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; import static org.apache.hudi.avro.HoodieAvroUtils.getNestedFieldSchemaFromWriteSchema; import static org.apache.hudi.avro.HoodieAvroUtils.sanitizeName; import static org.apache.hudi.avro.HoodieAvroWrapperUtils.unwrapAvroValueWrapper; @@ -1076,7 +1076,7 @@ public static Stream getSchemaForFieldParamsNested() { public void testGetSchemaForFieldNested(String colName, Schema.Type schemaType) { Pair actualColNameAndSchemaFile = HoodieAvroUtils.getSchemaForField(SCHEMA_WITH_NESTED_FIELD_LARGE, colName); assertEquals(colName, actualColNameAndSchemaFile.getKey()); - assertEquals(schemaType, resolveNullableSchema(actualColNameAndSchemaFile.getValue().schema()).getType()); + assertEquals(schemaType, getNonNullTypeFromUnion(actualColNameAndSchemaFile.getValue().schema()).getType()); } public static Stream getExpectedSchemaForFields() { @@ -1315,7 +1315,7 @@ private static Object generateRandomValue(Schema schema, Object defaultValue) { // Handle Union type. Schema actualSchema = schema; try { - actualSchema = resolveNullableSchema(schema); + actualSchema = getNonNullTypeFromUnion(schema); } catch (Exception e) { // If we can't resolve the schema, just use the original // Op. 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 e9db289b81643..f50359fe1f6a8 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 @@ -1735,7 +1735,7 @@ private void generateCustomValues(GenericRecord rec, String customPrefix) { rec.put(field.name(), ByteBuffer.wrap(getUTF8Bytes(genPseudoRandomUUID(rand).toString()))); break; case UNION: - if (!AvroSchemaUtils.resolveNullableSchema(field.schema()).getType().equals(Schema.Type.BOOLEAN)) { + if (!AvroSchemaUtils.getNonNullTypeFromUnion(field.schema()).getType().equals(Schema.Type.BOOLEAN)) { throw new IllegalStateException("Union should only be boolean"); } rec.put(field.name(), rand.nextBoolean()); diff --git a/hudi-common/src/test/java/org/apache/parquet/schema/TestAvroSchemaRepair.java b/hudi-common/src/test/java/org/apache/parquet/schema/TestAvroSchemaRepair.java new file mode 100644 index 0000000000000..e612cb6d8b606 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/parquet/schema/TestAvroSchemaRepair.java @@ -0,0 +1,983 @@ +/* + * 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.avro.AvroSchemaUtils; + +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests {@link AvroSchemaRepair}. + */ +public class TestAvroSchemaRepair { + + @Test + public void testNoRepairNeededIdenticalSchemas() { + Schema requestedSchema = Schema.create(Schema.Type.LONG); + Schema tableSchema = Schema.create(Schema.Type.LONG); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "When schemas are identical, should return same instance"); + } + + @Test + public void testNoRepairNeededDifferentPrimitiveTypes() { + Schema requestedSchema = Schema.create(Schema.Type.STRING); + Schema tableSchema = Schema.create(Schema.Type.INT); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "When types differ, should return original schema"); + } + + @Test + public void testRepairLongWithoutLogicalTypeToLocalTimestampMillis() { + Schema requestedSchema = Schema.create(Schema.Type.LONG); + Schema tableSchema = LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create a new schema with logical type"); + assertEquals(Schema.Type.LONG, result.getType()); + assertEquals(LogicalTypes.localTimestampMillis(), result.getLogicalType()); + } + + @Test + public void testRepairLongWithoutLogicalTypeToLocalTimestampMicros() { + Schema requestedSchema = Schema.create(Schema.Type.LONG); + Schema tableSchema = LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create a new schema with logical type"); + assertEquals(Schema.Type.LONG, result.getType()); + assertEquals(LogicalTypes.localTimestampMicros(), result.getLogicalType()); + } + + @Test + public void testRepairTimestampMicrosToTimestampMillis() { + Schema requestedSchema = LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + Schema tableSchema = LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create a new schema with timestamp-millis"); + assertEquals(Schema.Type.LONG, result.getType()); + assertEquals(LogicalTypes.timestampMillis(), result.getLogicalType()); + } + + @Test + public void testNoRepairNeededTimestampMillisToTimestampMicros() { + // This direction should NOT trigger repair + Schema requestedSchema = LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)); + Schema tableSchema = LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "Should not repair timestamp-millis to timestamp-micros"); + } + + @Test + public void testNoRepairNeededNonLongTypes() { + Schema requestedSchema = Schema.create(Schema.Type.INT); + Schema tableSchema = LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "Should not repair non-LONG types"); + } + + @Test + public void testRepairNullableSchemaLongToLocalTimestampMillis() { + Schema requestedSchema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + Schema.create(Schema.Type.LONG) + ); + Schema tableSchema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new nullable schema with repaired type"); + assertEquals(Schema.Type.UNION, result.getType()); + assertEquals(2, result.getTypes().size()); + + Schema nonNullType = AvroSchemaUtils.getNonNullTypeFromUnion(result); + assertEquals(LogicalTypes.localTimestampMillis(), nonNullType.getLogicalType()); + } + + @Test + public void testRepairNullableSchemaTimestampMicrosToMillis() { + Schema requestedSchema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)) + ); + Schema tableSchema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new nullable schema"); + assertEquals(Schema.Type.UNION, result.getType()); + + Schema nonNullType = AvroSchemaUtils.getNonNullTypeFromUnion(result); + assertEquals(LogicalTypes.timestampMillis(), nonNullType.getLogicalType()); + } + + @Test + public void testRepairRecordSingleField() { + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new record schema"); + assertEquals(Schema.Type.RECORD, result.getType()); + assertEquals("TestRecord", result.getName()); + assertEquals(1, result.getFields().size()); + + Schema.Field field = result.getField("timestamp"); + assertEquals(LogicalTypes.localTimestampMillis(), field.schema().getLogicalType()); + } + + @Test + public void testRepairRecordMultipleFieldsOnlyOneNeedsRepair() { + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type().longType().noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new record schema"); + assertEquals(3, result.getFields().size()); + + // Verify id field unchanged - should be same schema instance + assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema()); + + // Verify timestamp field repaired + assertEquals(LogicalTypes.localTimestampMicros(), result.getField("timestamp").schema().getLogicalType()); + + // Verify name field unchanged - should be same schema instance + assertSame(requestedSchema.getField("name").schema(), result.getField("name").schema()); + } + + @Test + public void testRepairRecordNestedRecord() { + Schema nestedRequestedSchema = SchemaBuilder.record("NestedRecord") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema nestedTableSchema = SchemaBuilder.record("NestedRecord") + .fields() + .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema requestedSchema = SchemaBuilder.record("OuterRecord") + .fields() + .name("id").type().intType().noDefault() + .name("nested").type(nestedRequestedSchema).noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("OuterRecord") + .fields() + .name("id").type().intType().noDefault() + .name("nested").type(nestedTableSchema).noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new schema for nested record"); + + // Verify id field unchanged - should be same schema instance + assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema()); + + // Verify nested record was repaired + Schema nestedResult = result.getField("nested").schema(); + assertEquals(Schema.Type.RECORD, nestedResult.getType()); + assertEquals(LogicalTypes.localTimestampMillis(), + nestedResult.getField("timestamp").schema().getLogicalType()); + } + + @Test + public void testRepairRecordNullableNestedField() { + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("timestamp").type().optional().longType() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("timestamp").type().optional().type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new schema"); + + Schema fieldSchema = result.getField("timestamp").schema(); + assertEquals(Schema.Type.UNION, fieldSchema.getType()); + + Schema nonNullType = AvroSchemaUtils.getNonNullTypeFromUnion(fieldSchema); + assertEquals(LogicalTypes.localTimestampMillis(), nonNullType.getLogicalType()); + } + + @Test + public void testRepairArrayElementNeedsRepair() { + Schema requestedSchema = Schema.createArray(Schema.create(Schema.Type.LONG)); + Schema tableSchema = Schema.createArray( + LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new array schema"); + assertEquals(Schema.Type.ARRAY, result.getType()); + assertEquals(LogicalTypes.localTimestampMillis(), result.getElementType().getLogicalType()); + } + + @Test + public void testRepairArrayNoRepairNeeded() { + Schema elementSchema = Schema.create(Schema.Type.STRING); + Schema requestedSchema = Schema.createArray(elementSchema); + Schema tableSchema = Schema.createArray(elementSchema); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "Should return same array when no repair needed"); + } + + @Test + public void testRepairArrayNullableElements() { + Schema requestedSchema = Schema.createArray( + Schema.createUnion(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.LONG)) + ); + Schema tableSchema = Schema.createArray( + Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)) + ) + ); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new array schema"); + Schema elementSchema = result.getElementType(); + assertEquals(Schema.Type.UNION, elementSchema.getType()); + + Schema nonNullType = AvroSchemaUtils.getNonNullTypeFromUnion(elementSchema); + assertEquals(LogicalTypes.localTimestampMicros(), nonNullType.getLogicalType()); + } + + @Test + public void testRepairMapValueNeedsRepair() { + Schema requestedSchema = Schema.createMap(Schema.create(Schema.Type.LONG)); + Schema tableSchema = Schema.createMap( + LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new map schema"); + assertEquals(Schema.Type.MAP, result.getType()); + assertEquals(LogicalTypes.localTimestampMillis(), result.getValueType().getLogicalType()); + } + + @Test + public void testRepairMapNoRepairNeeded() { + Schema valueSchema = Schema.create(Schema.Type.STRING); + Schema requestedSchema = Schema.createMap(valueSchema); + Schema tableSchema = Schema.createMap(valueSchema); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "Should return same map when no repair needed"); + } + + @Test + public void testRepairMapNullableValues() { + Schema requestedSchema = Schema.createMap( + Schema.createUnion(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.LONG)) + ); + Schema tableSchema = Schema.createMap( + Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ) + ); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new map schema"); + Schema valueSchema = result.getValueType(); + assertEquals(Schema.Type.UNION, valueSchema.getType()); + + Schema nonNullType = AvroSchemaUtils.getNonNullTypeFromUnion(valueSchema); + assertEquals(LogicalTypes.localTimestampMillis(), nonNullType.getLogicalType()); + } + + @Test + public void testComplexSchemaMultiLevelNesting() { + // Create a complex schema with nested records, arrays, and maps + Schema innerRecordRequested = SchemaBuilder.record("Inner") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema innerRecordTable = SchemaBuilder.record("Inner") + .fields() + .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema requestedSchema = SchemaBuilder.record("Outer") + .fields() + .name("id").type().intType().noDefault() + .name("records").type().array().items(innerRecordRequested).noDefault() + .name("mapping").type().map().values(Schema.create(Schema.Type.LONG)).noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("Outer") + .fields() + .name("id").type().intType().noDefault() + .name("records").type().array().items(innerRecordTable).noDefault() + .name("mapping").type().map().values( + LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)) + ).noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new complex schema"); + + // Verify id field unchanged - should be same schema instance + assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema()); + + // Verify array of records was repaired + Schema arrayElementSchema = result.getField("records").schema().getElementType(); + assertEquals(LogicalTypes.localTimestampMillis(), + arrayElementSchema.getField("timestamp").schema().getLogicalType()); + + // Verify map values were repaired + Schema mapValueSchema = result.getField("mapping").schema().getValueType(); + assertEquals(LogicalTypes.localTimestampMicros(), mapValueSchema.getLogicalType()); + } + + @Test + public void testRepairRecordMissingFieldInTableSchema() { + // Requested schema has a field not present in table schema + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("newField").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + // Should return original schema unchanged since newField doesn't exist in table schema + assertSame(requestedSchema, result, "Should return original when field missing in table schema"); + } + + @Test + public void testRepairRecordMultipleFieldsMissingInTableSchema() { + // Requested schema has multiple fields not present in table schema + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("newField1").type().longType().noDefault() + .name("name").type().stringType().noDefault() + .name("newField2").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + // Should return original schema unchanged since new fields don't exist in table schema + assertSame(requestedSchema, result, "Should return original when multiple fields missing in table schema"); + } + + @Test + public void testRepairRecordMixedMissingAndRepairableFields() { + // Requested schema has some fields missing in table, some needing repair, some unchanged + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type().longType().noDefault() + .name("newField").type().longType().noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + // Should create new schema with timestamp repaired, but newField preserved from requested + assertNotSame(requestedSchema, result, "Should create new schema"); + assertEquals(4, result.getFields().size()); + + // Verify id field unchanged + assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema()); + + // Verify timestamp field repaired + assertEquals(LogicalTypes.localTimestampMillis(), result.getField("timestamp").schema().getLogicalType()); + + // Verify newField preserved from requested schema (not in table) + assertSame(requestedSchema.getField("newField").schema(), result.getField("newField").schema()); + + // Verify name field unchanged + assertSame(requestedSchema.getField("name").schema(), result.getField("name").schema()); + } + + @Test + public void testRepairNestedRecordFieldMissingInTableSchema() { + // Requested nested record has a field not present in table's nested record + Schema nestedRequestedSchema = SchemaBuilder.record("NestedRecord") + .fields() + .name("timestamp").type().longType().noDefault() + .name("extraField").type().stringType().noDefault() + .endRecord(); + + Schema nestedTableSchema = SchemaBuilder.record("NestedRecord") + .fields() + .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema requestedSchema = SchemaBuilder.record("OuterRecord") + .fields() + .name("id").type().intType().noDefault() + .name("nested").type(nestedRequestedSchema).noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("OuterRecord") + .fields() + .name("id").type().intType().noDefault() + .name("nested").type(nestedTableSchema).noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new schema"); + + // Verify id field unchanged + assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema()); + + // Verify nested record was repaired but still has extraField + Schema nestedResult = result.getField("nested").schema(); + assertEquals(Schema.Type.RECORD, nestedResult.getType()); + assertEquals(2, nestedResult.getFields().size()); + + // Timestamp should be repaired + assertEquals(LogicalTypes.localTimestampMillis(), + nestedResult.getField("timestamp").schema().getLogicalType()); + + // extraField should be preserved from requested schema + assertSame(nestedRequestedSchema.getField("extraField").schema(), + nestedResult.getField("extraField").schema()); + } + + @Test + public void testRepairRecordWholeNestedRecordMissingInTableSchema() { + // Requested schema has a nested record field that doesn't exist in table schema + Schema nestedRequestedSchema = SchemaBuilder.record("NestedRecord") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema requestedSchema = SchemaBuilder.record("OuterRecord") + .fields() + .name("id").type().intType().noDefault() + .name("newNested").type(nestedRequestedSchema).noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("OuterRecord") + .fields() + .name("id").type().intType().noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + // Should return original schema unchanged since newNested field doesn't exist in table + assertSame(requestedSchema, result, "Should return original when nested field missing in table schema"); + } + + @Test + public void testRepairRecordPreservesFieldMetadata() { + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .doc("Test documentation") + .fields() + .name("timestamp").doc("Timestamp field").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result); + assertEquals("TestRecord", result.getName()); + assertEquals("Test documentation", result.getDoc()); + assertEquals("Timestamp field", result.getField("timestamp").doc()); + } + + @Test + public void testEdgeCaseEmptyRecord() { + Schema requestedSchema = SchemaBuilder.record("EmptyRecord").fields().endRecord(); + Schema tableSchema = SchemaBuilder.record("EmptyRecord").fields().endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "Empty records should return same instance"); + } + + @Test + public void testRepairRecordFirstFieldChanged() { + // Test the optimization path where the first field needs repair + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("timestamp1").type().longType().noDefault() + .name("timestamp2").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("timestamp1").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .name("timestamp2").type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result); + assertEquals(LogicalTypes.localTimestampMillis(), result.getField("timestamp1").schema().getLogicalType()); + assertEquals(LogicalTypes.localTimestampMicros(), result.getField("timestamp2").schema().getLogicalType()); + } + + @Test + public void testRepairRecordLastFieldChanged() { + // Test the optimization path where only the last field needs repair + Schema requestedSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("name").type().stringType().noDefault() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("name").type().stringType().noDefault() + .name("timestamp").type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema result = AvroSchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result); + // Verify id and name fields unchanged - should be same schema instances + assertSame(requestedSchema.getField("id").schema(), result.getField("id").schema()); + assertSame(requestedSchema.getField("name").schema(), result.getField("name").schema()); + // Verify timestamp field repaired + assertEquals(LogicalTypes.localTimestampMillis(), result.getField("timestamp").schema().getLogicalType()); + } + + @Test + public void testHasTimestampMillisFieldPrimitiveLongWithTimestampMillis() { + Schema schema = LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)); + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for LONG with timestamp-millis logical type"); + } + + @Test + public void testHasTimestampMillisFieldPrimitiveLongWithoutLogicalType() { + Schema schema = Schema.create(Schema.Type.LONG); + assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return false for LONG without logical type"); + } + + @Test + public void testHasTimestampMillisFieldPrimitiveLongWithTimestampMicros() { + Schema schema = LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return false for LONG with timestamp-micros logical type"); + } + + @Test + public void testHasTimestampMillisFieldPrimitiveLongWithLocalTimestampMillis() { + Schema schema = LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)); + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for LONG with local-timestamp-millis logical type"); + } + + @Test + public void testHasTimestampMillisFieldPrimitiveLongWithLocalTimestampMicros() { + Schema schema = LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return false for LONG with local-timestamp-micros logical type"); + } + + @Test + public void testHasTimestampMillisFieldOtherPrimitiveTypes() { + assertFalse(AvroSchemaRepair.hasTimestampMillisField(Schema.create(Schema.Type.STRING)), + "Should return false for STRING type"); + assertFalse(AvroSchemaRepair.hasTimestampMillisField(Schema.create(Schema.Type.INT)), + "Should return false for INT type"); + assertFalse(AvroSchemaRepair.hasTimestampMillisField(Schema.create(Schema.Type.FLOAT)), + "Should return false for FLOAT type"); + assertFalse(AvroSchemaRepair.hasTimestampMillisField(Schema.create(Schema.Type.DOUBLE)), + "Should return false for DOUBLE type"); + assertFalse(AvroSchemaRepair.hasTimestampMillisField(Schema.create(Schema.Type.BOOLEAN)), + "Should return false for BOOLEAN type"); + } + + @Test + public void testHasTimestampMillisFieldRecordWithTimestampMillis() { + Schema schema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for record containing timestamp-millis field"); + } + + @Test + public void testHasTimestampMillisFieldRecordWithoutTimestampMillis() { + Schema schema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return false for record without timestamp-millis field"); + } + + @Test + public void testHasTimestampMillisFieldRecordEmpty() { + Schema schema = SchemaBuilder.record("EmptyRecord").fields().endRecord(); + + assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return false for empty record"); + } + + @Test + public void testHasTimestampMillisFieldNestedRecord() { + Schema innerSchema = SchemaBuilder.record("InnerRecord") + .fields() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema outerSchema = SchemaBuilder.record("OuterRecord") + .fields() + .name("id").type().intType().noDefault() + .name("inner").type(innerSchema).noDefault() + .endRecord(); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(outerSchema), + "Should return true for nested record containing timestamp-millis field"); + } + + @Test + public void testHasTimestampMillisFieldDeeplyNestedRecord() { + Schema level3 = SchemaBuilder.record("Level3") + .fields() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema level2 = SchemaBuilder.record("Level2") + .fields() + .name("data").type(level3).noDefault() + .endRecord(); + + Schema level1 = SchemaBuilder.record("Level1") + .fields() + .name("nested").type(level2).noDefault() + .endRecord(); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(level1), + "Should return true for deeply nested record containing timestamp-millis field"); + } + + @Test + public void testHasTimestampMillisFieldArrayWithTimestampMillis() { + Schema schema = Schema.createArray( + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for array with timestamp-millis elements"); + } + + @Test + public void testHasTimestampMillisFieldArrayWithoutTimestampMillis() { + Schema schema = Schema.createArray(Schema.create(Schema.Type.STRING)); + + assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return false for array without timestamp-millis elements"); + } + + @Test + public void testHasTimestampMillisFieldArrayOfRecordsWithTimestampMillis() { + Schema elementSchema = SchemaBuilder.record("Element") + .fields() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema schema = Schema.createArray(elementSchema); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for array of records containing timestamp-millis field"); + } + + @Test + public void testHasTimestampMillisFieldMapWithTimestampMillis() { + Schema schema = Schema.createMap( + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for map with timestamp-millis values"); + } + + @Test + public void testHasTimestampMillisFieldMapWithoutTimestampMillis() { + Schema schema = Schema.createMap(Schema.create(Schema.Type.STRING)); + + assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return false for map without timestamp-millis values"); + } + + @Test + public void testHasTimestampMillisFieldMapOfRecordsWithTimestampMillis() { + Schema valueSchema = SchemaBuilder.record("Value") + .fields() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema schema = Schema.createMap(valueSchema); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for map of records containing timestamp-millis field"); + } + + @Test + public void testHasTimestampMillisFieldUnionWithTimestampMillis() { + Schema schema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for nullable union with timestamp-millis"); + } + + @Test + public void testHasTimestampMillisFieldUnionWithoutTimestampMillis() { + Schema schema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + Schema.create(Schema.Type.LONG) + ); + + assertFalse(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return false for nullable union without timestamp-millis"); + } + + @Test + public void testHasTimestampMillisFieldUnionWithRecordContainingTimestampMillis() { + Schema recordSchema = SchemaBuilder.record("Record") + .fields() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema schema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + recordSchema + ); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for nullable union with record containing timestamp-millis"); + } + + @Test + public void testHasTimestampMillisFieldComplexNestedStructure() { + // Create a complex schema with arrays, maps, and nested records + Schema innerRecordSchema = SchemaBuilder.record("InnerRecord") + .fields() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + Schema complexSchema = SchemaBuilder.record("ComplexRecord") + .fields() + .name("id").type().intType().noDefault() + .name("arrayOfRecords").type().array().items(innerRecordSchema).noDefault() + .name("mapOfStrings").type().map().values().stringType().noDefault() + .endRecord(); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(complexSchema), + "Should return true for complex nested structure containing timestamp-millis field"); + } + + @Test + public void testHasTimestampMillisFieldComplexStructureWithoutTimestampMillis() { + Schema innerRecordSchema = SchemaBuilder.record("InnerRecord") + .fields() + .name("value").type().longType().noDefault() + .endRecord(); + + Schema complexSchema = SchemaBuilder.record("ComplexRecord") + .fields() + .name("id").type().intType().noDefault() + .name("arrayOfRecords").type().array().items(innerRecordSchema).noDefault() + .name("mapOfLongs").type().map().values( + LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)) + ).noDefault() + .endRecord(); + + assertFalse(AvroSchemaRepair.hasTimestampMillisField(complexSchema), + "Should return false for complex structure without timestamp-millis field"); + } + + @Test + public void testHasTimestampMillisFieldFirstFieldMatches() { + Schema schema = SchemaBuilder.record("TestRecord") + .fields() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .name("id").type().intType().noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true when first field is timestamp-millis"); + } + + @Test + public void testHasTimestampMillisFieldLastFieldMatches() { + Schema schema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("name").type().stringType().noDefault() + .name("timestamp").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true when last field is timestamp-millis"); + } + + @Test + public void testHasTimestampMillisFieldMultipleTimestampMillisFields() { + Schema schema = SchemaBuilder.record("TestRecord") + .fields() + .name("createdAt").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .name("id").type().intType().noDefault() + .name("updatedAt").type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .endRecord(); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true when multiple timestamp-millis fields exist"); + } + + @Test + public void testHasTimestampMillisFieldNullableFieldWithTimestampMillis() { + Schema schema = SchemaBuilder.record("TestRecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type().optional().type( + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ) + .endRecord(); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for nullable field with timestamp-millis"); + } + + @Test + public void testHasTimestampMillisFieldArrayOfNullableTimestampMillis() { + Schema elementSchema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + Schema schema = Schema.createArray(elementSchema); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for array of nullable timestamp-millis elements"); + } + + @Test + public void testHasTimestampMillisFieldMapOfNullableTimestampMillis() { + Schema valueSchema = Schema.createUnion( + Schema.create(Schema.Type.NULL), + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)) + ); + + Schema schema = Schema.createMap(valueSchema); + + assertTrue(AvroSchemaRepair.hasTimestampMillisField(schema), + "Should return true for map of nullable timestamp-millis values"); + } +} diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroParquetReader.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroParquetReader.java index 5881003373973..71a0854c2aa3b 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroParquetReader.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroParquetReader.java @@ -46,6 +46,7 @@ import org.apache.parquet.avro.HoodieAvroParquetReaderBuilder; import org.apache.parquet.hadoop.ParquetInputFormat; import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.schema.AvroSchemaRepair; import java.io.IOException; import java.util.ArrayList; @@ -55,6 +56,7 @@ import java.util.Set; import static org.apache.hudi.common.util.TypeUtils.unsafeCast; +import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; /** * {@link HoodieFileReader} implementation for parquet format. @@ -178,17 +180,20 @@ private ClosableIterator getIndexedRecordIteratorInternal(Schema // sure that in case the file-schema is not equal to read-schema we'd still // be able to read that file (in case projection is a proper one) Configuration hadoopConf = storage.getConf().unwrapCopyAs(Configuration.class); + Schema repairedFileSchema = AvroSchemaRepair.repairLogicalTypes(getSchema(), schema); Option promotedSchema = Option.empty(); - if (!renamedColumns.isEmpty() || HoodieAvroUtils.recordNeedsRewriteForExtendedAvroTypePromotion(getSchema(), schema)) { - AvroReadSupport.setAvroReadSchema(hadoopConf, getSchema()); - AvroReadSupport.setRequestedProjection(hadoopConf, getSchema()); + if (!renamedColumns.isEmpty() || HoodieAvroUtils.recordNeedsRewriteForExtendedAvroTypePromotion(repairedFileSchema, schema)) { + AvroReadSupport.setAvroReadSchema(hadoopConf, repairedFileSchema); + AvroReadSupport.setRequestedProjection(hadoopConf, repairedFileSchema); promotedSchema = Option.of(schema); } else { AvroReadSupport.setAvroReadSchema(hadoopConf, schema); AvroReadSupport.setRequestedProjection(hadoopConf, schema); } ParquetReader reader = - new HoodieAvroParquetReaderBuilder(path).withConf(hadoopConf) + new HoodieAvroParquetReaderBuilder(path) + .withTableSchema(getAvroSchemaConverter(hadoopConf).convert(schema)) + .withConf(hadoopConf) .set(AvroSchemaConverter.ADD_LIST_ELEMENT_RECORDS, hadoopConf.get(AvroSchemaConverter.ADD_LIST_ELEMENT_RECORDS)) .set(ParquetInputFormat.STRICT_TYPE_CHECKING, hadoopConf.get(ParquetInputFormat.STRICT_TYPE_CHECKING)) .build(); diff --git a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetReaderBuilder.java b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetReaderBuilder.java index ef58c52902373..9fe32b456ef3b 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetReaderBuilder.java +++ b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetReaderBuilder.java @@ -18,6 +18,7 @@ package org.apache.parquet.avro; +import org.apache.hudi.common.util.Option; import org.apache.hudi.storage.StoragePath; import org.apache.avro.generic.GenericData; @@ -26,6 +27,7 @@ import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.hadoop.api.ReadSupport; import org.apache.parquet.io.InputFile; +import org.apache.parquet.schema.MessageType; /** * Copy from org.apache.parquet.avro.AvroParquetReader.Builder. @@ -37,6 +39,7 @@ public class HoodieAvroParquetReaderBuilder extends ParquetReader.Builder private GenericData model = null; private boolean enableCompatibility = true; private boolean isReflect = true; + private Option tableSchema = Option.empty(); @Deprecated public HoodieAvroParquetReaderBuilder(StoragePath path) { @@ -69,6 +72,11 @@ public HoodieAvroParquetReaderBuilder withCompatibility(boolean enableCompati return this; } + public HoodieAvroParquetReaderBuilder withTableSchema(MessageType tableSchema) { + this.tableSchema = Option.of(tableSchema); + return this; + } + @Override protected ReadSupport getReadSupport() { if (isReflect) { @@ -76,6 +84,6 @@ protected ReadSupport getReadSupport() { } else { conf.setBoolean(AvroReadSupport.AVRO_COMPATIBILITY, enableCompatibility); } - return new HoodieAvroReadSupport<>(model); + return new HoodieAvroReadSupport<>(model, tableSchema); } } diff --git a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java index 246052bccd5c3..d06cfae1ede37 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java +++ b/hudi-hadoop-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java @@ -18,6 +18,8 @@ package org.apache.parquet.avro; +import org.apache.hudi.common.util.Option; + import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.hadoop.conf.Configuration; @@ -25,6 +27,7 @@ import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.SchemaRepair; import org.apache.parquet.schema.Type; import java.util.ArrayList; @@ -39,11 +42,15 @@ */ public class HoodieAvroReadSupport extends AvroReadSupport { - public HoodieAvroReadSupport(GenericData model) { + private Option tableSchema; + + public HoodieAvroReadSupport(GenericData model, Option tableSchema) { super(model); + this.tableSchema = tableSchema; } public HoodieAvroReadSupport() { + tableSchema = Option.empty(); } @Override @@ -51,7 +58,7 @@ public ReadContext init(Configuration configuration, Map keyValu boolean legacyMode = checkLegacyMode(fileSchema.getFields()); adjustConfToReadWithFileProduceMode(legacyMode, configuration); ReadContext readContext = super.init(configuration, keyValueMetaData, fileSchema); - MessageType requestedSchema = readContext.getRequestedSchema(); + MessageType requestedSchema = SchemaRepair.repairLogicalTypes(readContext.getRequestedSchema(), tableSchema); // support non-legacy map. Convert non-legacy map to legacy map // Because there is no AvroWriteSupport.WRITE_OLD_MAP_STRUCTURE // according to AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE @@ -74,7 +81,7 @@ public ReadContext init(Configuration configuration, Map keyValu public ReadContext init(ParquetConfiguration configuration, Map keyValueMetaData, MessageType fileSchema) { boolean legacyMode = checkLegacyMode(fileSchema.getFields()); configuration.set(AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE, String.valueOf(legacyMode)); - MessageType projection = fileSchema; + MessageType projection = SchemaRepair.repairLogicalTypes(fileSchema, tableSchema); Map metadata = new LinkedHashMap(); String requestedProjectionString = configuration.get(AVRO_REQUESTED_PROJECTION); diff --git a/hudi-hadoop-common/src/main/java/org/apache/parquet/schema/SchemaRepair.java b/hudi-hadoop-common/src/main/java/org/apache/parquet/schema/SchemaRepair.java new file mode 100644 index 0000000000000..b85b3b98fac38 --- /dev/null +++ b/hudi-hadoop-common/src/main/java/org/apache/parquet/schema/SchemaRepair.java @@ -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. + */ + +package org.apache.parquet.schema; + +import org.apache.hudi.common.util.Option; + +import java.util.ArrayList; +import java.util.List; + +public class SchemaRepair { + + public static MessageType repairLogicalTypes(MessageType fileSchema, Option tableSchema) { + if (tableSchema.isEmpty()) { + return fileSchema; + } + return repairLogicalTypes(fileSchema, tableSchema.get()); + } + + static MessageType repairLogicalTypes(MessageType fileSchema, MessageType tableSchema) { + List repairedFields = repairFields(fileSchema.getFields(), tableSchema); + + // If nothing changed, return the original schema + if (repairedFields == null) { + return fileSchema; + } + + return new MessageType(fileSchema.getName(), repairedFields); + } + + /** + * Repairs a list of fields against a table schema (MessageType or GroupType). + * Returns null if no changes were made, otherwise returns the repaired field list. + */ + private static List repairFields(List fileSchemaFields, GroupType tableSchema) { + // First pass: find the first field that changes + int firstChangedIndex = -1; + Type firstRepairedField = null; + + for (int i = 0; i < fileSchemaFields.size(); i++) { + Type requestedField = fileSchemaFields.get(i); + if (tableSchema.containsField(requestedField.getName())) { + Type tableField = tableSchema.getType(requestedField.getName()); + Type repaired = repairField(requestedField, tableField); + if (repaired != requestedField) { + firstChangedIndex = i; + firstRepairedField = repaired; + break; + } + } + } + + // If nothing changed, return null + if (firstChangedIndex == -1) { + return null; + } + + // Second pass: build the new field list with repaired fields + List repairedFields = new ArrayList<>(fileSchemaFields.size()); + + // Copy all fields before the first changed field + for (int i = 0; i < firstChangedIndex; i++) { + repairedFields.add(fileSchemaFields.get(i)); + } + + // Add the first changed field (using cached repaired field) + repairedFields.add(firstRepairedField); + + // Process remaining fields + for (int i = firstChangedIndex + 1; i < fileSchemaFields.size(); i++) { + Type fileSchemaField = fileSchemaFields.get(i); + Type repaired = fileSchemaField; + if (tableSchema.containsField(fileSchemaField.getName())) { + Type tableSchemaField = tableSchema.getType(fileSchemaField.getName()); + repaired = repairField(fileSchemaField, tableSchemaField); + } + repairedFields.add(repaired); + } + + return repairedFields; + } + + private static Type repairField(Type fileSchemaFieldType, Type tableSchemaFieldType) { + if (fileSchemaFieldType.isPrimitive() && tableSchemaFieldType.isPrimitive()) { + return repairPrimitiveType(fileSchemaFieldType.asPrimitiveType(), tableSchemaFieldType.asPrimitiveType()); + } else if (!fileSchemaFieldType.isPrimitive() && !tableSchemaFieldType.isPrimitive()) { + // recurse into nested structs + GroupType reqGroup = fileSchemaFieldType.asGroupType(); + GroupType tblGroup = tableSchemaFieldType.asGroupType(); + + // Repair fields directly without creating MessageType intermediaries + List repairedFields = repairFields(reqGroup.getFields(), tblGroup); + + // If nothing changed, return the original field + if (repairedFields == null) { + return fileSchemaFieldType; + } + + return new GroupType( + reqGroup.getRepetition(), + reqGroup.getName(), + reqGroup.getLogicalTypeAnnotation(), + repairedFields + ); + } else { + // fallback: keep requested + return fileSchemaFieldType; + } + } + + private static PrimitiveType repairPrimitiveType(PrimitiveType fileSchemaPrimitiveType, PrimitiveType tableSchemaPrimitiveType) { + // Quick check if repair is needed (no allocations) + if (needsLogicalTypeRepair(fileSchemaPrimitiveType, tableSchemaPrimitiveType)) { + return Types.primitive(tableSchemaPrimitiveType.getPrimitiveTypeName(), fileSchemaPrimitiveType.getRepetition()) + .as(tableSchemaPrimitiveType.getLogicalTypeAnnotation()) + .named(fileSchemaPrimitiveType.getName()); + } + return fileSchemaPrimitiveType; + } + + /** + * Quick check if a logical type repair is needed (no allocations). + */ + private static boolean needsLogicalTypeRepair(PrimitiveType fileSchemaPrimitiveType, PrimitiveType tableSchemaPrimitiveType) { + if (fileSchemaPrimitiveType.getPrimitiveTypeName() != PrimitiveType.PrimitiveTypeName.INT64 + || tableSchemaPrimitiveType.getPrimitiveTypeName() != PrimitiveType.PrimitiveTypeName.INT64) { + return false; + } + LogicalTypeAnnotation fileLogicalTypeAnnotation = fileSchemaPrimitiveType.getLogicalTypeAnnotation(); + LogicalTypeAnnotation tableLogicalTypeAnnotation = tableSchemaPrimitiveType.getLogicalTypeAnnotation(); + + // if requested has no logical type, and the table has a local timestamp, then we need to repair + if (fileLogicalTypeAnnotation == null) { + return tableLogicalTypeAnnotation instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation + && !((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) tableLogicalTypeAnnotation).isAdjustedToUTC(); + } + + // if requested is timestamp-micros and table is timestamp-millis then we need to repair + return fileLogicalTypeAnnotation instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation + && tableLogicalTypeAnnotation instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation + && ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) fileLogicalTypeAnnotation).getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS + && ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) tableLogicalTypeAnnotation).getUnit() == LogicalTypeAnnotation.TimeUnit.MILLIS + && ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) fileLogicalTypeAnnotation).isAdjustedToUTC() + && ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) tableLogicalTypeAnnotation).isAdjustedToUTC(); + } +} diff --git a/hudi-hadoop-common/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java b/hudi-hadoop-common/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java new file mode 100644 index 0000000000000..17569f58f0f71 --- /dev/null +++ b/hudi-hadoop-common/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java @@ -0,0 +1,956 @@ +/* + * 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 java.util.Arrays; +import java.util.Collections; +import org.apache.avro.JsonProperties; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.MessageTypeParser; +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 org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import static org.apache.avro.Schema.Type.INT; +import static org.apache.avro.Schema.Type.LONG; +import static org.apache.avro.Schema.Type.STRING; +import static org.apache.avro.SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE; +import static org.apache.avro.SchemaCompatibility.checkReaderWriterCompatibility; +import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource; +import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; +import static org.apache.parquet.schema.OriginalType.DATE; +import static org.apache.parquet.schema.OriginalType.TIMESTAMP_MICROS; +import static org.apache.parquet.schema.OriginalType.TIMESTAMP_MILLIS; +import static org.apache.parquet.schema.OriginalType.TIME_MICROS; +import static org.apache.parquet.schema.OriginalType.TIME_MILLIS; +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.PrimitiveType.PrimitiveTypeName.INT96; +import static org.apache.parquet.schema.Type.Repetition.REQUIRED; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; + +public class TestAvroSchemaConverter { + + private static final Configuration NEW_BEHAVIOR = new Configuration(false); + + @BeforeAll + public static void setupConf() { + NEW_BEHAVIOR.setBoolean("parquet.avro.add-list-element-records", false); + NEW_BEHAVIOR.setBoolean("parquet.avro.write-old-list-structure", false); + } + + public static final String ALL_PARQUET_SCHEMA = "message org.apache.parquet.avro.myrecord {\n" + + " required boolean myboolean;\n" + + " required int32 myint;\n" + + " required int64 mylong;\n" + + " required float myfloat;\n" + + " required double mydouble;\n" + + " required binary mybytes;\n" + + " required binary mystring (UTF8);\n" + + " required group mynestedrecord {\n" + + " required int32 mynestedint;\n" + + " }\n" + + " required binary myenum (ENUM);\n" + + " required group myarray (LIST) {\n" + + " repeated int32 array;\n" + + " }\n" + + " optional group myoptionalarray (LIST) {\n" + + " repeated int32 array;\n" + + " }\n" + + " required group myarrayofoptional (LIST) {\n" + + " repeated group list {\n" + + " optional int32 element;\n" + + " }\n" + + " }\n" + + " required group myrecordarray (LIST) {\n" + + " repeated group array {\n" + + " required int32 a;\n" + + " required int32 b;\n" + + " }\n" + + " }\n" + + " required group mymap (MAP) {\n" + + " repeated group map (MAP_KEY_VALUE) {\n" + + " required binary key (UTF8);\n" + + " required int32 value;\n" + + " }\n" + + " }\n" + + " required fixed_len_byte_array(1) myfixed;\n" + + "}\n"; + + private void testAvroToParquetConversion(Schema avroSchema, String schemaString) throws Exception { + testAvroToParquetConversion(new Configuration(false), avroSchema, schemaString); + } + + private void testAvroToParquetConversion(Configuration conf, Schema avroSchema, String schemaString) + throws Exception { + HoodieAvroParquetSchemaConverter avroSchemaConverter = getAvroSchemaConverter(conf); + MessageType schema = avroSchemaConverter.convert(avroSchema); + MessageType expectedMT = MessageTypeParser.parseMessageType(schemaString); + assertEquals("converting " + schema + " to " + schemaString, expectedMT.toString(), schema.toString()); + } + + private void testParquetToAvroConversion(Schema avroSchema, String schemaString) throws Exception { + testParquetToAvroConversion(new Configuration(false), avroSchema, schemaString); + } + + private void testParquetToAvroConversion(Configuration conf, Schema avroSchema, String schemaString) + throws Exception { + HoodieAvroParquetSchemaConverter avroSchemaConverter = getAvroSchemaConverter(conf); + Schema schema = avroSchemaConverter.convert(MessageTypeParser.parseMessageType(schemaString)); + assertEquals("converting " + schemaString + " to " + avroSchema, avroSchema.toString(), schema.toString()); + } + + private void testRoundTripConversion(Schema avroSchema, String schemaString) throws Exception { + testRoundTripConversion(new Configuration(), avroSchema, schemaString); + } + + private void testRoundTripConversion(Configuration conf, Schema avroSchema, String schemaString) throws Exception { + HoodieAvroParquetSchemaConverter avroSchemaConverter = getAvroSchemaConverter(conf); + MessageType schema = avroSchemaConverter.convert(avroSchema); + MessageType expectedMT = MessageTypeParser.parseMessageType(schemaString); + assertEquals("converting " + schema + " to " + schemaString, expectedMT.toString(), schema.toString()); + Schema convertedAvroSchema = avroSchemaConverter.convert(expectedMT); + assertEquals( + "converting " + expectedMT + " to " + avroSchema.toString(true), + avroSchema.toString(), + convertedAvroSchema.toString()); + } + + @Test() + public void testTopLevelMustBeARecord() { + assertThrows("expected to throw", IllegalArgumentException.class, () -> getAvroSchemaConverter(new Configuration()).convert(Schema.create(INT))); + } + + @Test + public void testAllTypes() throws Exception { + Schema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/all.avsc"); + testAvroToParquetConversion( + NEW_BEHAVIOR, + schema, + "message org.apache.parquet.avro.myrecord {\n" + // Avro nulls are not encoded, unless they are null unions + + " required boolean myboolean;\n" + + " required int32 myint;\n" + + " required int64 mylong;\n" + + " required float myfloat;\n" + + " required double mydouble;\n" + + " required binary mybytes;\n" + + " required binary mystring (UTF8);\n" + + " required group mynestedrecord {\n" + + " required int32 mynestedint;\n" + + " }\n" + + " required binary myenum (ENUM);\n" + + " required group myarray (LIST) {\n" + + " repeated group list {\n" + + " required int32 element;\n" + + " }\n" + + " }\n" + + " required group myemptyarray (LIST) {\n" + + " repeated group list {\n" + + " required int32 element;\n" + + " }\n" + + " }\n" + + " optional group myoptionalarray (LIST) {\n" + + " repeated group list {\n" + + " required int32 element;\n" + + " }\n" + + " }\n" + + " required group myarrayofoptional (LIST) {\n" + + " repeated group list {\n" + + " optional int32 element;\n" + + " }\n" + + " }\n" + + " required group mymap (MAP) {\n" + + " repeated group key_value (MAP_KEY_VALUE) {\n" + + " required binary key (UTF8);\n" + + " required int32 value;\n" + + " }\n" + + " }\n" + + " required group myemptymap (MAP) {\n" + + " repeated group key_value (MAP_KEY_VALUE) {\n" + + " required binary key (UTF8);\n" + + " required int32 value;\n" + + " }\n" + + " }\n" + + " required fixed_len_byte_array(1) myfixed;\n" + + "}\n"); + } + + @Test + public void testAllTypesOldListBehavior() throws Exception { + Schema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/all.avsc"); + testAvroToParquetConversion( + schema, + "message org.apache.parquet.avro.myrecord {\n" + // Avro nulls are not encoded, unless they are null unions + + " required boolean myboolean;\n" + + " required int32 myint;\n" + + " required int64 mylong;\n" + + " required float myfloat;\n" + + " required double mydouble;\n" + + " required binary mybytes;\n" + + " required binary mystring (UTF8);\n" + + " required group mynestedrecord {\n" + + " required int32 mynestedint;\n" + + " }\n" + + " required binary myenum (ENUM);\n" + + " required group myarray (LIST) {\n" + + " repeated int32 array;\n" + + " }\n" + + " required group myemptyarray (LIST) {\n" + + " repeated int32 array;\n" + + " }\n" + + " optional group myoptionalarray (LIST) {\n" + + " repeated int32 array;\n" + + " }\n" + + " required group myarrayofoptional (LIST) {\n" + + " repeated int32 array;\n" + + " }\n" + + " required group mymap (MAP) {\n" + + " repeated group key_value (MAP_KEY_VALUE) {\n" + + " required binary key (UTF8);\n" + + " required int32 value;\n" + + " }\n" + + " }\n" + + " required group myemptymap (MAP) {\n" + + " repeated group key_value (MAP_KEY_VALUE) {\n" + + " required binary key (UTF8);\n" + + " required int32 value;\n" + + " }\n" + + " }\n" + + " required fixed_len_byte_array(1) myfixed;\n" + + "}\n"); + } + + @Test + public void testAllTypesParquetToAvro() throws Exception { + Schema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/allFromParquetNewBehavior.avsc"); + // Cannot use round-trip assertion because enum is lost + testParquetToAvroConversion(NEW_BEHAVIOR, schema, ALL_PARQUET_SCHEMA); + } + + @Test + public void testAllTypesParquetToAvroOldBehavior() throws Exception { + Schema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/allFromParquetOldBehavior.avsc"); + // Cannot use round-trip assertion because enum is lost + testParquetToAvroConversion(schema, ALL_PARQUET_SCHEMA); + } + + @Test + public void testParquetMapWithNonStringKeyFails() throws Exception { + MessageType parquetSchema = + MessageTypeParser.parseMessageType("message myrecord {\n" + " required group mymap (MAP) {\n" + + " repeated group map (MAP_KEY_VALUE) {\n" + + " required int32 key;\n" + + " required int32 value;\n" + + " }\n" + + " }\n" + + "}\n"); + assertThrows("expected to throw", IllegalArgumentException.class, () -> getAvroSchemaConverter(new Configuration()).convert(parquetSchema)); + } + + @Test + public void testOptionalFields() throws Exception { + Schema schema = Schema.createRecord("record1", null, null, false); + Schema optionalInt = optional(Schema.create(INT)); + schema.setFields( + Collections.singletonList(new Schema.Field("myint", optionalInt, null, JsonProperties.NULL_VALUE))); + testRoundTripConversion(schema, "message record1 {\n" + " optional int32 myint;\n" + "}\n"); + } + + @Test + public void testOptionalMapValue() throws Exception { + Schema schema = Schema.createRecord("record1", null, null, false); + Schema optionalIntMap = Schema.createMap(optional(Schema.create(INT))); + schema.setFields(Arrays.asList(new Schema.Field("myintmap", optionalIntMap, null, null))); + testRoundTripConversion( + schema, + "message record1 {\n" + " required group myintmap (MAP) {\n" + + " repeated group key_value (MAP_KEY_VALUE) {\n" + + " required binary key (UTF8);\n" + + " optional int32 value;\n" + + " }\n" + + " }\n" + + "}\n"); + } + + @Test + public void testOptionalArrayElement() throws Exception { + Schema schema = Schema.createRecord("record1", null, null, false); + Schema optionalIntArray = Schema.createArray(optional(Schema.create(INT))); + schema.setFields(Arrays.asList(new Schema.Field("myintarray", optionalIntArray, null, null))); + testRoundTripConversion( + NEW_BEHAVIOR, + schema, + "message record1 {\n" + " required group myintarray (LIST) {\n" + + " repeated group list {\n" + + " optional int32 element;\n" + + " }\n" + + " }\n" + + "}\n"); + } + + @Test + public void testUnionOfTwoTypes() throws Exception { + Schema schema = Schema.createRecord("record2", null, null, false); + Schema multipleTypes = Schema.createUnion( + Arrays.asList(Schema.create(Schema.Type.NULL), Schema.create(INT), Schema.create(Schema.Type.FLOAT))); + schema.setFields(Arrays.asList(new Schema.Field("myunion", multipleTypes, null, JsonProperties.NULL_VALUE))); + + // Avro union is modelled using optional data members of the different + // types. This does not translate back into an Avro union + testAvroToParquetConversion( + schema, + "message record2 {\n" + " optional group myunion {\n" + + " optional int32 member0;\n" + + " optional float member1;\n" + + " }\n" + + "}\n"); + } + + @Test + public void testArrayOfOptionalRecords() throws Exception { + Schema innerRecord = Schema.createRecord("element", null, null, false); + Schema optionalString = optional(Schema.create(Schema.Type.STRING)); + innerRecord.setFields(Arrays.asList( + new Schema.Field("s1", optionalString, null, JsonProperties.NULL_VALUE), + new Schema.Field("s2", optionalString, null, JsonProperties.NULL_VALUE))); + Schema schema = Schema.createRecord("HasArray", null, null, false); + schema.setFields( + Arrays.asList(new Schema.Field("myarray", Schema.createArray(optional(innerRecord)), null, null))); + System.err.println("Avro schema: " + schema.toString(true)); + + testRoundTripConversion( + NEW_BEHAVIOR, + schema, + "message HasArray {\n" + " required group myarray (LIST) {\n" + + " repeated group list {\n" + + " optional group element {\n" + + " optional binary s1 (UTF8);\n" + + " optional binary s2 (UTF8);\n" + + " }\n" + + " }\n" + + " }\n" + + "}\n"); + } + + @Test + public void testArrayOfOptionalRecordsOldBehavior() throws Exception { + Schema innerRecord = Schema.createRecord("InnerRecord", null, null, false); + Schema optionalString = optional(Schema.create(Schema.Type.STRING)); + innerRecord.setFields(Arrays.asList( + new Schema.Field("s1", optionalString, null, JsonProperties.NULL_VALUE), + new Schema.Field("s2", optionalString, null, JsonProperties.NULL_VALUE))); + Schema schema = Schema.createRecord("HasArray", null, null, false); + schema.setFields( + Arrays.asList(new Schema.Field("myarray", Schema.createArray(optional(innerRecord)), null, null))); + System.err.println("Avro schema: " + schema.toString(true)); + + // Cannot use round-trip assertion because InnerRecord optional is removed + testAvroToParquetConversion( + schema, + "message HasArray {\n" + " required group myarray (LIST) {\n" + + " repeated group array {\n" + + " optional binary s1 (UTF8);\n" + + " optional binary s2 (UTF8);\n" + + " }\n" + + " }\n" + + "}\n"); + } + + @Test + public void testOldAvroListOfLists() throws Exception { + Schema listOfLists = optional(Schema.createArray(Schema.createArray(Schema.create(INT)))); + Schema schema = Schema.createRecord("AvroCompatListInList", null, null, false); + schema.setFields( + Arrays.asList(new Schema.Field("listOfLists", listOfLists, null, JsonProperties.NULL_VALUE))); + System.err.println("Avro schema: " + schema.toString(true)); + + testRoundTripConversion( + schema, + "message AvroCompatListInList {\n" + " optional group listOfLists (LIST) {\n" + + " repeated group array (LIST) {\n" + + " repeated int32 array;\n" + + " }\n" + + " }\n" + + "}"); + // Cannot use round-trip assertion because 3-level representation is used + testParquetToAvroConversion( + NEW_BEHAVIOR, + schema, + "message AvroCompatListInList {\n" + " optional group listOfLists (LIST) {\n" + + " repeated group array (LIST) {\n" + + " repeated int32 array;\n" + + " }\n" + + " }\n" + + "}"); + } + + @Test + public void testOldThriftListOfLists() throws Exception { + Schema listOfLists = optional(Schema.createArray(Schema.createArray(Schema.create(INT)))); + Schema schema = Schema.createRecord("ThriftCompatListInList", null, null, false); + schema.setFields( + Arrays.asList(new Schema.Field("listOfLists", listOfLists, null, JsonProperties.NULL_VALUE))); + System.err.println("Avro schema: " + schema.toString(true)); + + // Cannot use round-trip assertion because repeated group names differ + testParquetToAvroConversion( + schema, + "message ThriftCompatListInList {\n" + " optional group listOfLists (LIST) {\n" + + " repeated group listOfLists_tuple (LIST) {\n" + + " repeated int32 listOfLists_tuple_tuple;\n" + + " }\n" + + " }\n" + + "}"); + // Cannot use round-trip assertion because 3-level representation is used + testParquetToAvroConversion( + NEW_BEHAVIOR, + schema, + "message ThriftCompatListInList {\n" + " optional group listOfLists (LIST) {\n" + + " repeated group listOfLists_tuple (LIST) {\n" + + " repeated int32 listOfLists_tuple_tuple;\n" + + " }\n" + + " }\n" + + "}"); + } + + @Test + public void testUnknownTwoLevelListOfLists() throws Exception { + // This tests the case where we don't detect a 2-level list by the repeated + // group's name, but it must be 2-level because the repeated group doesn't + // contain an optional or repeated element as required for 3-level lists + Schema listOfLists = optional(Schema.createArray(Schema.createArray(Schema.create(INT)))); + Schema schema = Schema.createRecord("UnknownTwoLevelListInList", null, null, false); + schema.setFields( + Arrays.asList(new Schema.Field("listOfLists", listOfLists, null, JsonProperties.NULL_VALUE))); + System.err.println("Avro schema: " + schema.toString(true)); + + // Cannot use round-trip assertion because repeated group names differ + testParquetToAvroConversion( + schema, + "message UnknownTwoLevelListInList {\n" + " optional group listOfLists (LIST) {\n" + + " repeated group mylist (LIST) {\n" + + " repeated int32 innerlist;\n" + + " }\n" + + " }\n" + + "}"); + // Cannot use round-trip assertion because 3-level representation is used + testParquetToAvroConversion( + NEW_BEHAVIOR, + schema, + "message UnknownTwoLevelListInList {\n" + " optional group listOfLists (LIST) {\n" + + " repeated group mylist (LIST) {\n" + + " repeated int32 innerlist;\n" + + " }\n" + + " }\n" + + "}"); + } + + @Test + public void testParquetMapWithoutMapKeyValueAnnotation() throws Exception { + Schema schema = Schema.createRecord("myrecord", null, null, false); + Schema map = Schema.createMap(Schema.create(INT)); + schema.setFields(Collections.singletonList(new Schema.Field("mymap", map, null, null))); + String parquetSchema = "message myrecord {\n" + " required group mymap (MAP) {\n" + + " repeated group map {\n" + + " required binary key (UTF8);\n" + + " required int32 value;\n" + + " }\n" + + " }\n" + + "}\n"; + + testParquetToAvroConversion(schema, parquetSchema); + testParquetToAvroConversion(NEW_BEHAVIOR, schema, parquetSchema); + } + + @Test + public void testDecimalBytesType() throws Exception { + Schema schema = Schema.createRecord("myrecord", null, null, false); + Schema decimal = LogicalTypes.decimal(9, 2).addToSchema(Schema.create(Schema.Type.BYTES)); + schema.setFields(Collections.singletonList(new Schema.Field("dec", decimal, null, null))); + + testRoundTripConversion(schema, "message myrecord {\n" + " required binary dec (DECIMAL(9,2));\n" + "}\n"); + } + + @Test + public void testDecimalFixedType() throws Exception { + Schema schema = Schema.createRecord("myrecord", null, null, false); + Schema decimal = LogicalTypes.decimal(9, 2).addToSchema(Schema.createFixed("dec", null, null, 8)); + schema.setFields(Collections.singletonList(new Schema.Field("dec", decimal, null, null))); + + testRoundTripConversion( + schema, "message myrecord {\n" + " required fixed_len_byte_array(8) dec (DECIMAL(9,2));\n" + "}\n"); + } + + @Test + public void testDecimalIntegerType() throws Exception { + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("dec", Schema.create(INT), null, null))); + + // the decimal portion is lost because it isn't valid in Avro + testParquetToAvroConversion( + expected, "message myrecord {\n" + " required int32 dec (DECIMAL(9,2));\n" + "}\n"); + } + + @Test + public void testDecimalLongType() throws Exception { + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("dec", Schema.create(LONG), null, null))); + + // the decimal portion is lost because it isn't valid in Avro + testParquetToAvroConversion( + expected, "message myrecord {\n" + " required int64 dec (DECIMAL(9,2));\n" + "}\n"); + } + + @Test + public void testParquetInt96AsFixed12AvroType() throws Exception { + Configuration enableInt96ReadingConfig = new Configuration(); + enableInt96ReadingConfig.setBoolean(AvroReadSupport.READ_INT96_AS_FIXED, true); + + Schema schema = Schema.createRecord("myrecord", null, null, false); + Schema int96schema = Schema.createFixed("INT96", "INT96 represented as byte[12]", null, 12); + schema.setFields(Collections.singletonList( + new Schema.Field("int96_field", int96schema, null, null))); + + testParquetToAvroConversion(enableInt96ReadingConfig, schema, "message myrecord {\n" + + " required int96 int96_field;\n" + + "}\n"); + } + + @Test + public void testParquetInt96DefaultFail() throws Exception { + Schema schema = Schema.createRecord("myrecord", null, null, false); + + MessageType parquetSchemaWithInt96 = + MessageTypeParser.parseMessageType("message myrecord {\n required int96 int96_field;\n}\n"); + + assertThrows( + "INT96 is deprecated. As interim enable READ_INT96_AS_FIXED flag to read as byte array.", + IllegalArgumentException.class, + () -> getAvroSchemaConverter(new Configuration()).convert(parquetSchemaWithInt96)); + } + + @Test + public void testDateType() throws Exception { + Schema date = LogicalTypes.date().addToSchema(Schema.create(INT)); + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("date", date, null, null))); + + testRoundTripConversion(expected, "message myrecord {\n" + " required int32 date (DATE);\n" + "}\n"); + + for (PrimitiveTypeName primitive : + new PrimitiveTypeName[] {INT64, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) { + final PrimitiveType type; + if (primitive == FIXED_LEN_BYTE_ARRAY) { + type = new PrimitiveType(REQUIRED, primitive, 12, "test", DATE); + } else { + type = new PrimitiveType(REQUIRED, primitive, "test", DATE); + } + + assertThrows( + "Should not allow TIME_MICROS with " + primitive, + IllegalArgumentException.class, + () -> getAvroSchemaConverter(new Configuration()).convert(message(type))); + } + } + + @Test + public void testTimeMillisType() throws Exception { + Schema date = LogicalTypes.timeMillis().addToSchema(Schema.create(INT)); + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("time", date, null, null))); + + testRoundTripConversion( + expected, "message myrecord {\n" + " required int32 time (TIME(MILLIS,true));\n" + "}\n"); + + for (PrimitiveTypeName primitive : + new PrimitiveTypeName[] {INT64, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) { + final PrimitiveType type; + if (primitive == FIXED_LEN_BYTE_ARRAY) { + type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIME_MILLIS); + } else { + type = new PrimitiveType(REQUIRED, primitive, "test", TIME_MILLIS); + } + + assertThrows( + "Should not allow TIME_MICROS with " + primitive, + IllegalArgumentException.class, + () -> getAvroSchemaConverter(new Configuration()).convert(message(type))); + } + } + + @Test + public void testTimeMicrosType() throws Exception { + Schema date = LogicalTypes.timeMicros().addToSchema(Schema.create(LONG)); + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("time", date, null, null))); + + testRoundTripConversion( + expected, "message myrecord {\n" + " required int64 time (TIME(MICROS,true));\n" + "}\n"); + + for (PrimitiveTypeName primitive : + new PrimitiveTypeName[] {INT32, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) { + final PrimitiveType type; + if (primitive == FIXED_LEN_BYTE_ARRAY) { + type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIME_MICROS); + } else { + type = new PrimitiveType(REQUIRED, primitive, "test", TIME_MICROS); + } + + assertThrows( + "Should not allow TIME_MICROS with " + primitive, + IllegalArgumentException.class, + () -> getAvroSchemaConverter(new Configuration()).convert(message(type))); + } + } + + @Test + public void testTimestampMillisType() throws Exception { + Schema date = LogicalTypes.timestampMillis().addToSchema(Schema.create(LONG)); + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("timestamp", date, null, null))); + + testRoundTripConversion( + expected, "message myrecord {\n" + " required int64 timestamp (TIMESTAMP(MILLIS,true));\n" + "}\n"); + + final Schema converted = getAvroSchemaConverter(new Configuration()) + .convert(Types.buildMessage() + .addField(Types.primitive(INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType( + false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .length(1) + .named("timestamp_type")) + .named("TestAvro")); + assertEquals( + "local-timestamp-millis", + converted + .getField("timestamp_type") + .schema() + .getLogicalType() + .getName()); + + for (PrimitiveTypeName primitive : + new PrimitiveTypeName[] {INT32, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) { + final PrimitiveType type; + if (primitive == FIXED_LEN_BYTE_ARRAY) { + type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIMESTAMP_MILLIS); + } else { + type = new PrimitiveType(REQUIRED, primitive, "test", TIMESTAMP_MILLIS); + } + + assertThrows( + "Should not allow TIMESTAMP_MILLIS with " + primitive, + IllegalArgumentException.class, + () -> getAvroSchemaConverter(new Configuration()).convert(message(type))); + } + } + + @Test + public void testLocalTimestampMillisType() throws Exception { + Schema date = LogicalTypes.localTimestampMillis().addToSchema(Schema.create(LONG)); + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("timestamp", date, null, null))); + + testRoundTripConversion( + expected, "message myrecord {\n" + " required int64 timestamp (TIMESTAMP(MILLIS,false));\n" + "}\n"); + + for (PrimitiveTypeName primitive : + new PrimitiveTypeName[] {INT32, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) { + final PrimitiveType type; + if (primitive == FIXED_LEN_BYTE_ARRAY) { + type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIMESTAMP_MILLIS); + } else { + type = new PrimitiveType(REQUIRED, primitive, "test", TIMESTAMP_MILLIS); + } + + assertThrows( + "Should not allow TIMESTAMP_MILLIS with " + primitive, + IllegalArgumentException.class, + () -> getAvroSchemaConverter(new Configuration()).convert(message(type))); + } + } + + @Test + public void testTimestampMicrosType() throws Exception { + Schema date = LogicalTypes.timestampMicros().addToSchema(Schema.create(LONG)); + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("timestamp", date, null, null))); + + testRoundTripConversion( + expected, "message myrecord {\n" + " required int64 timestamp (TIMESTAMP(MICROS,true));\n" + "}\n"); + + for (PrimitiveTypeName primitive : + new PrimitiveTypeName[] {INT32, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) { + final PrimitiveType type; + if (primitive == FIXED_LEN_BYTE_ARRAY) { + type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIMESTAMP_MICROS); + } else { + type = new PrimitiveType(REQUIRED, primitive, "test", TIMESTAMP_MICROS); + } + + assertThrows( + "Should not allow TIMESTAMP_MICROS with " + primitive, + IllegalArgumentException.class, + () -> getAvroSchemaConverter(new Configuration()).convert(message(type))); + } + + final Schema converted = getAvroSchemaConverter(new Configuration()) + .convert(Types.buildMessage() + .addField(Types.primitive(INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType( + false, LogicalTypeAnnotation.TimeUnit.MICROS)) + .length(1) + .named("timestamp_type")) + .named("TestAvro")); + + assertEquals( + "local-timestamp-micros", + converted + .getField("timestamp_type") + .schema() + .getLogicalType() + .getName()); + } + + @Test + public void testLocalTimestampMicrosType() throws Exception { + Schema date = LogicalTypes.localTimestampMicros().addToSchema(Schema.create(LONG)); + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("timestamp", date, null, null))); + + testRoundTripConversion( + expected, "message myrecord {\n" + " required int64 timestamp (TIMESTAMP(MICROS,false));\n" + "}\n"); + + for (PrimitiveTypeName primitive : + new PrimitiveTypeName[] {INT32, INT96, FLOAT, DOUBLE, BOOLEAN, BINARY, FIXED_LEN_BYTE_ARRAY}) { + final PrimitiveType type; + if (primitive == FIXED_LEN_BYTE_ARRAY) { + type = new PrimitiveType(REQUIRED, primitive, 12, "test", TIMESTAMP_MICROS); + } else { + type = new PrimitiveType(REQUIRED, primitive, "test", TIMESTAMP_MICROS); + } + + assertThrows( + "Should not allow TIMESTAMP_MICROS with " + primitive, + IllegalArgumentException.class, + () -> getAvroSchemaConverter(new Configuration()).convert(message(type))); + } + } + + @Test + public void testReuseNameInNestedStructure() throws Exception { + Schema innerA1 = record("a1", "a12", field("a4", primitive(Schema.Type.FLOAT))); + + Schema outerA1 = record("a1", field("a2", primitive(Schema.Type.FLOAT)), optionalField("a1", innerA1)); + Schema schema = record("Message", optionalField("a1", outerA1)); + + String parquetSchema = "message Message {\n" + + " optional group a1 {\n" + + " required float a2;\n" + + " optional group a1 {\n" + + " required float a4;\n" + + " }\n" + + " }\n" + + "}\n"; + + testParquetToAvroConversion(schema, parquetSchema); + testParquetToAvroConversion(NEW_BEHAVIOR, schema, parquetSchema); + } + + @Test + public void testReuseNameInNestedStructureAtSameLevel() throws Exception { + Schema a2 = record("a2", field("a4", primitive(Schema.Type.FLOAT))); + Schema a22 = record( + "a2", "a22", field("a4", primitive(Schema.Type.FLOAT)), field("a5", primitive(Schema.Type.FLOAT))); + + Schema a1 = record("a1", optionalField("a2", a2)); + Schema a3 = record("a3", optionalField("a2", a22)); + + Schema schema = record("Message", optionalField("a1", a1), optionalField("a3", a3)); + + String parquetSchema = "message Message {\n" + + " optional group a1 {\n" + + " optional group a2 {\n" + + " required float a4;\n" + + " }\n" + + " }\n" + + " optional group a3 {\n" + + " optional group a2 {\n" + + " required float a4;\n" + + " required float a5;\n" + + " }\n" + + " }\n" + + "}\n"; + + testParquetToAvroConversion(schema, parquetSchema); + testParquetToAvroConversion(NEW_BEHAVIOR, schema, parquetSchema); + } + + @Test + public void testUUIDType() throws Exception { + Schema fromAvro = Schema.createRecord( + "myrecord", + null, + null, + false, + Arrays.asList( + new Schema.Field("uuid", LogicalTypes.uuid().addToSchema(Schema.create(STRING)), null, null))); + String parquet = "message myrecord {\n" + " required binary uuid (STRING);\n" + "}\n"; + Schema toAvro = Schema.createRecord( + "myrecord", + null, + null, + false, + Arrays.asList(new Schema.Field("uuid", Schema.create(STRING), null, null))); + + testAvroToParquetConversion(fromAvro, parquet); + testParquetToAvroConversion(toAvro, parquet); + + assertEquals( + COMPATIBLE, checkReaderWriterCompatibility(fromAvro, toAvro).getType()); + } + + @Test + public void testUUIDTypeWithParquetUUID() throws Exception { + Schema uuid = LogicalTypes.uuid().addToSchema(Schema.create(STRING)); + Schema expected = Schema.createRecord( + "myrecord", null, null, false, Arrays.asList(new Schema.Field("uuid", uuid, null, null))); + + testRoundTripConversion( + conf(AvroWriteSupport.WRITE_PARQUET_UUID, true), + expected, + "message myrecord {\n" + " required fixed_len_byte_array(16) uuid (UUID);\n" + "}\n"); + } + + @Test + public void testAvroFixed12AsParquetInt96Type() throws Exception { + Schema schema = getSchemaFromResource(TestAvroSchemaConverter.class, "/parquet-java/fixedToInt96.avsc"); + + Configuration conf = new Configuration(); + conf.setStrings( + "parquet.avro.writeFixedAsInt96", + "int96", + "mynestedrecord.int96inrecord", + "mynestedrecord.myarrayofoptional", + "mynestedrecord.mymap"); + testAvroToParquetConversion( + conf, + schema, + "message org.apache.parquet.avro.fixedToInt96 {\n" + + " required int96 int96;\n" + + " required fixed_len_byte_array(12) notanint96;\n" + + " required group mynestedrecord {\n" + + " required int96 int96inrecord;\n" + + " required group myarrayofoptional (LIST) {\n" + + " repeated int96 array;\n" + + " }\n" + + " required group mymap (MAP) {\n" + + " repeated group key_value (MAP_KEY_VALUE) {\n" + + " required binary key (STRING);\n" + + " required int96 value;\n" + + " }\n" + + " }\n" + + " }\n" + + " required fixed_len_byte_array(1) onebytefixed;\n" + + "}"); + + conf.setStrings("parquet.avro.writeFixedAsInt96", "onebytefixed"); + assertThrows( + "Exception should be thrown for fixed types to be converted to INT96 where the size is not 12 bytes", + IllegalArgumentException.class, + () -> getAvroSchemaConverter(conf).convert(schema)); + } + + public static Schema optional(Schema original) { + return Schema.createUnion(Arrays.asList(Schema.create(Schema.Type.NULL), original)); + } + + public static MessageType message(PrimitiveType primitive) { + return Types.buildMessage().addField(primitive).named("myrecord"); + } + + /** + * A convenience method to avoid a large number of @Test(expected=...) tests + * + * @param message A String message to describe this assertion + * @param expected An Exception class that the Runnable should throw + * @param runnable A Runnable that is expected to throw the exception + */ + public static void assertThrows(String message, Class expected, Runnable runnable) { + try { + runnable.run(); + fail("No exception was thrown (" + message + "), expected: " + expected.getName()); + } catch (Exception actual) { + try { + assertEquals(expected, actual.getClass(), message); + } catch (AssertionError e) { + e.addSuppressed(actual); + throw e; + } + } + } + + public static Schema record(String name, String namespace, Schema.Field... fields) { + Schema record = Schema.createRecord(name, null, namespace, false); + record.setFields(Arrays.asList(fields)); + return record; + } + + public static Schema record(String name, Schema.Field... fields) { + return record(name, null, fields); + } + + public static Schema.Field field(String name, Schema schema) { + return new Schema.Field(name, schema, null, null); + } + + public static Schema.Field optionalField(String name, Schema schema) { + return new Schema.Field(name, optional(schema), null, JsonProperties.NULL_VALUE); + } + + public static Schema array(Schema element) { + return Schema.createArray(element); + } + + public static Schema primitive(Schema.Type type) { + return Schema.create(type); + } + + public static Configuration conf(String name, boolean value) { + Configuration conf = new Configuration(false); + conf.setBoolean(name, value); + return conf; + } + +} \ No newline at end of file diff --git a/hudi-hadoop-common/src/test/java/org/apache/parquet/schema/TestSchemaRepair.java b/hudi-hadoop-common/src/test/java/org/apache/parquet/schema/TestSchemaRepair.java new file mode 100644 index 0000000000000..b31d37c835dbd --- /dev/null +++ b/hudi-hadoop-common/src/test/java/org/apache/parquet/schema/TestSchemaRepair.java @@ -0,0 +1,600 @@ +/* + * 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.common.util.Option; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertSame; + +/** + * Tests {@link SchemaRepair}. + */ +public class TestSchemaRepair { + + @Test + public void testNoRepairNeededIdenticalSchemas() { + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "When schemas are identical, should return same instance"); + } + + @Test + public void testNoRepairNeededDifferentPrimitiveTypes() { + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "When field names differ, should return original schema"); + } + + @Test + public void testRepairLongWithoutLogicalTypeToLocalTimestampMillis() { + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create a new schema with logical type"); + PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType(); + assertEquals(PrimitiveType.PrimitiveTypeName.INT64, timestampField.getPrimitiveTypeName()); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestampField.getLogicalTypeAnnotation()); + } + + @Test + public void testRepairLongWithoutLogicalTypeToLocalTimestampMicros() { + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create a new schema with logical type"); + PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType(); + assertEquals(PrimitiveType.PrimitiveTypeName.INT64, timestampField.getPrimitiveTypeName()); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS), + timestampField.getLogicalTypeAnnotation()); + } + + @Test + public void testRepairTimestampMicrosToTimestampMillis() { + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("timestamp") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create a new schema with timestamp-millis"); + PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType(); + assertEquals(PrimitiveType.PrimitiveTypeName.INT64, timestampField.getPrimitiveTypeName()); + assertEquals(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestampField.getLogicalTypeAnnotation()); + } + + @Test + public void testNoRepairNeededTimestampMillisToTimestampMicros() { + // This direction should NOT trigger repair + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "Should not repair timestamp-millis to timestamp-micros"); + } + + @Test + public void testNoRepairNeededNonLongTypes() { + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.dateType()) + .named("id") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "Should not repair non-LONG types"); + } + + @Test + public void testRepairRecordSingleField() { + MessageType requestedSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + + MessageType tableSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new record schema"); + assertEquals(1, result.getFields().size()); + + PrimitiveType field = result.getType("timestamp").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + field.getLogicalTypeAnnotation()); + } + + @Test + public void testRepairRecordMultipleFieldsOnlyOneNeedsRepair() { + MessageType requestedSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name") + ); + + MessageType tableSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("timestamp"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new record schema"); + assertEquals(3, result.getFields().size()); + + // Verify id field unchanged - should be same type instance + assertSame(requestedSchema.getType("id"), result.getType("id")); + + // Verify timestamp field repaired + PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS), + timestampField.getLogicalTypeAnnotation()); + + // Verify name field unchanged - should be same type instance + assertSame(requestedSchema.getType("name"), result.getType("name")); + } + + @Test + public void testRepairRecordNestedRecord() { + GroupType nestedRequestedSchema = new GroupType(Type.Repetition.REQUIRED, "nested", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + + GroupType nestedTableSchema = new GroupType(Type.Repetition.REQUIRED, "nested", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType requestedSchema = new MessageType("OuterRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + nestedRequestedSchema + ); + + MessageType tableSchema = new MessageType("OuterRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + nestedTableSchema + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new schema for nested record"); + + // Verify id field unchanged - should be same type instance + assertSame(requestedSchema.getType("id"), result.getType("id")); + + // Verify nested record was repaired + GroupType nestedResult = result.getType("nested").asGroupType(); + PrimitiveType nestedTimestamp = nestedResult.getType("timestamp").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + nestedTimestamp.getLogicalTypeAnnotation()); + } + + @Test + public void testRepairRecordMissingFieldInTableSchema() { + // Requested schema has a field not present in table schema + MessageType requestedSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("newField") + ); + + MessageType tableSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + // Should return original schema unchanged since newField doesn't exist in table schema + assertSame(requestedSchema, result, "Should return original when field missing in table schema"); + } + + @Test + public void testRepairRecordMultipleFieldsMissingInTableSchema() { + // Requested schema has multiple fields not present in table schema + MessageType requestedSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("newField1"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("newField2") + ); + + MessageType tableSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + // Should return original schema unchanged since new fields don't exist in table schema + assertSame(requestedSchema, result, "Should return original when multiple fields missing in table schema"); + } + + @Test + public void testRepairRecordMixedMissingAndRepairableFields() { + // Requested schema has some fields missing in table, some needing repair, some unchanged + MessageType requestedSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("newField"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name") + ); + + MessageType tableSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + // Should create new schema with timestamp repaired, but newField preserved from requested + assertNotSame(requestedSchema, result, "Should create new schema"); + assertEquals(4, result.getFields().size()); + + // Verify id field unchanged + assertSame(requestedSchema.getType("id"), result.getType("id")); + + // Verify timestamp field repaired + PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestampField.getLogicalTypeAnnotation()); + + // Verify newField preserved from requested schema (not in table) + assertSame(requestedSchema.getType("newField"), result.getType("newField")); + + // Verify name field unchanged + assertSame(requestedSchema.getType("name"), result.getType("name")); + } + + @Test + public void testRepairNestedRecordFieldMissingInTableSchema() { + // Requested nested record has a field not present in table's nested record + GroupType nestedRequestedSchema = new GroupType(Type.Repetition.REQUIRED, "nested", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("extraField") + ); + + GroupType nestedTableSchema = new GroupType(Type.Repetition.REQUIRED, "nested", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType requestedSchema = new MessageType("OuterRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + nestedRequestedSchema + ); + + MessageType tableSchema = new MessageType("OuterRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + nestedTableSchema + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result, "Should create new schema"); + + // Verify id field unchanged + assertSame(requestedSchema.getType("id"), result.getType("id")); + + // Verify nested record was repaired but still has extraField + GroupType nestedResult = result.getType("nested").asGroupType(); + assertEquals(2, nestedResult.getFieldCount()); + + // Timestamp should be repaired + PrimitiveType timestampField = nestedResult.getType("timestamp").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestampField.getLogicalTypeAnnotation()); + + // extraField should be preserved from requested schema + assertSame(nestedRequestedSchema.getType("extraField"), nestedResult.getType("extraField")); + } + + @Test + public void testRepairRecordWholeNestedRecordMissingInTableSchema() { + // Requested schema has a nested record field that doesn't exist in table schema + GroupType nestedRequestedSchema = new GroupType(Type.Repetition.REQUIRED, "newNested", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + + MessageType requestedSchema = new MessageType("OuterRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + nestedRequestedSchema + ); + + MessageType tableSchema = new MessageType("OuterRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + // Should return original schema unchanged since newNested field doesn't exist in table + assertSame(requestedSchema, result, "Should return original when nested field missing in table schema"); + } + + @Test + public void testEdgeCaseEmptyRecord() { + MessageType requestedSchema = new MessageType("EmptyRecord"); + MessageType tableSchema = new MessageType("EmptyRecord"); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertSame(requestedSchema, result, "Empty records should return same instance"); + } + + @Test + public void testRepairRecordFirstFieldChanged() { + // Test the optimization path where the first field needs repair + MessageType requestedSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp1"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp2") + ); + + MessageType tableSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp1"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("timestamp2") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result); + PrimitiveType timestamp1 = result.getType("timestamp1").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestamp1.getLogicalTypeAnnotation()); + PrimitiveType timestamp2 = result.getType("timestamp2").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MICROS), + timestamp2.getLogicalTypeAnnotation()); + } + + @Test + public void testRepairRecordLastFieldChanged() { + // Test the optimization path where only the last field needs repair + MessageType requestedSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + + MessageType tableSchema = new MessageType("TestRecord", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.REQUIRED) + .named("id"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.stringType()) + .named("name"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result); + // Verify id and name fields unchanged - should be same type instances + assertSame(requestedSchema.getType("id"), result.getType("id")); + assertSame(requestedSchema.getType("name"), result.getType("name")); + // Verify timestamp field repaired + PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestampField.getLogicalTypeAnnotation()); + } + + @Test + public void testRepairLogicalTypesWithOptionEmpty() { + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, Option.empty()); + + assertSame(requestedSchema, result, "Should return original when Option is empty"); + } + + @Test + public void testRepairLogicalTypesWithOptionPresent() { + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, Option.of(tableSchema)); + + assertNotSame(requestedSchema, result, "Should repair when Option is present"); + PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestampField.getLogicalTypeAnnotation()); + } + + @Test + public void testRepairOptionalFieldRepetition() { + // Test that repair preserves the requested field's repetition (OPTIONAL vs REQUIRED) + MessageType requestedSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.OPTIONAL) + .named("timestamp") + ); + MessageType tableSchema = new MessageType("TestSchema", + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result); + PrimitiveType timestampField = result.getType("timestamp").asPrimitiveType(); + assertEquals(Type.Repetition.OPTIONAL, timestampField.getRepetition(), + "Should preserve requested field's repetition"); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestampField.getLogicalTypeAnnotation()); + } + + @Test + public void testRepairNestedGroupPreservesLogicalType() { + // Test that repair preserves the group's logical type annotation + GroupType nestedRequestedSchema = new GroupType(Type.Repetition.REQUIRED, "nested", + LogicalTypeAnnotation.listType(), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .named("timestamp") + ); + + GroupType nestedTableSchema = new GroupType(Type.Repetition.REQUIRED, "nested", + LogicalTypeAnnotation.listType(), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .as(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS)) + .named("timestamp") + ); + + MessageType requestedSchema = new MessageType("OuterRecord", nestedRequestedSchema); + MessageType tableSchema = new MessageType("OuterRecord", nestedTableSchema); + + MessageType result = SchemaRepair.repairLogicalTypes(requestedSchema, tableSchema); + + assertNotSame(requestedSchema, result); + GroupType nestedResult = result.getType("nested").asGroupType(); + assertEquals(LogicalTypeAnnotation.listType(), nestedResult.getLogicalTypeAnnotation(), + "Should preserve group's logical type annotation"); + PrimitiveType timestampField = nestedResult.getType("timestamp").asPrimitiveType(); + assertEquals(LogicalTypeAnnotation.timestampType(false, LogicalTypeAnnotation.TimeUnit.MILLIS), + timestampField.getLogicalTypeAnnotation()); + } +} diff --git a/hudi-hadoop-common/src/test/java/org/apache/parquet/schema/TestSchemaRepairEquivalence.java b/hudi-hadoop-common/src/test/java/org/apache/parquet/schema/TestSchemaRepairEquivalence.java new file mode 100644 index 0000000000000..75fe9ffde7d61 --- /dev/null +++ b/hudi-hadoop-common/src/test/java/org/apache/parquet/schema/TestSchemaRepairEquivalence.java @@ -0,0 +1,481 @@ +/* + * 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.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.avro.HoodieAvroParquetSchemaConverter; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * Tests equivalence between {@link SchemaRepair} and {@link AvroSchemaRepair}. + * + * This test class verifies that both repair implementations produce logically + * equivalent results when converting between Avro and Parquet schemas. + */ +public class TestSchemaRepairEquivalence { + + private HoodieAvroParquetSchemaConverter converter; + + @BeforeEach + public void setUp() { + converter = HoodieAvroParquetSchemaConverter.getAvroSchemaConverter(new Configuration()); + } + + /** + * Helper method to verify that AvroSchemaRepair and SchemaRepair produce equivalent results. + */ + private void assertRepairEquivalence(Schema requestedAvro, Schema tableAvro) { + // Apply Avro repair + Schema repairedAvro = AvroSchemaRepair.repairLogicalTypes(requestedAvro, tableAvro); + + // Convert to Parquet schemas + MessageType requestedParquet = converter.convert(requestedAvro); + MessageType tableParquet = converter.convert(tableAvro); + + // Apply Parquet repair + MessageType repairedParquet = SchemaRepair.repairLogicalTypes(requestedParquet, tableParquet); + + // Convert repaired Parquet back to Avro + Schema repairedParquetAsAvro = converter.convert(repairedParquet); + + // Verify equivalence + assertEquals(repairedAvro, repairedParquetAsAvro, + "SchemaRepair and AvroSchemaRepair should produce equivalent results"); + } + + @Test + public void testEquivalenceNoRepairNeeded() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("value").type().longType().noDefault() + .endRecord(); + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("value").type().longType().noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceLongToLocalTimestampMillis() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceLongToLocalTimestampMicros() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceTimestampMicrosToMillis() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceNoRepairTimestampMillisToMicros() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceSimpleRecord() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceRecordMultipleFields() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type().longType().noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp") + .type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceNestedRecord() { + Schema nestedRequestedSchema = SchemaBuilder.record("nestedrecord") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema nestedTableSchema = SchemaBuilder.record("nestedrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + Schema requestedSchema = SchemaBuilder.record("outerrecord") + .fields() + .name("id").type().intType().noDefault() + .name("nestedrecord").type(nestedRequestedSchema).noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("outerrecord") + .fields() + .name("id").type().intType().noDefault() + .name("nestedrecord").type(nestedTableSchema).noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceRecordWithExtraFieldInRequested() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type().longType().noDefault() + .name("newfield").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceRecordMixedFields() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp").type().longType().noDefault() + .name("newfield").type().stringType().noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("id").type().intType().noDefault() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .name("name").type().stringType().noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceNestedRecordWithExtraField() { + Schema nestedRequestedSchema = SchemaBuilder.record("nestedrecord") + .fields() + .name("timestamp").type().longType().noDefault() + .name("extrafield").type().stringType().noDefault() + .endRecord(); + + Schema nestedTableSchema = SchemaBuilder.record("nestedrecord") + .fields() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + Schema requestedSchema = SchemaBuilder.record("outerrecord") + .fields() + .name("id").type().intType().noDefault() + .name("nestedrecord").type(nestedRequestedSchema).noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("outerrecord") + .fields() + .name("id").type().intType().noDefault() + .name("nestedrecord").type(nestedTableSchema).noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceRecordFirstFieldChanged() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp1").type().longType().noDefault() + .name("timestamp2").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("timestamp1") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .name("timestamp2") + .type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceRecordLastFieldChanged() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("id").type().intType().noDefault() + .name("name").type().stringType().noDefault() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("id").type().intType().noDefault() + .name("name").type().stringType().noDefault() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceComplexNestedStructure() { + Schema innerRecordRequested = SchemaBuilder.record("inner") + .fields() + .name("timestamp").type().longType().noDefault() + .name("value").type().intType().noDefault() + .endRecord(); + + Schema innerRecordTable = SchemaBuilder.record("inner") + .fields() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .name("value").type().intType().noDefault() + .endRecord(); + + Schema middleRecordRequested = SchemaBuilder.record("middle") + .fields() + .name("inner").type(innerRecordRequested).noDefault() + .name("middletimestamp").type().longType().noDefault() + .endRecord(); + + Schema middleRecordTable = SchemaBuilder.record("middle") + .fields() + .name("inner").type(innerRecordTable).noDefault() + .name("middletimestamp") + .type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + Schema requestedSchema = SchemaBuilder.record("outer") + .fields() + .name("id").type().intType().noDefault() + .name("middle").type(middleRecordRequested).noDefault() + .name("outertimestamp").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("outer") + .fields() + .name("id").type().intType().noDefault() + .name("middle").type(middleRecordTable).noDefault() + .name("outertimestamp") + .type(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceEmptyRecord() { + Schema requestedSchema = SchemaBuilder.record("emptyrecord").fields().endRecord(); + Schema tableSchema = SchemaBuilder.record("emptyrecord").fields().endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceRecordNoFieldsMatch() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("field1").type().longType().noDefault() + .name("field2").type().stringType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("field3").type().intType().noDefault() + .name("field4") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceMultipleTimestampRepairs() { + Schema requestedSchema = SchemaBuilder.record("testrecord") + .fields() + .name("ts1").type().longType().noDefault() + .name("ts2").type().longType().noDefault() + .name("ts3").type(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))).noDefault() + .name("ts4").type().longType().noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("testrecord") + .fields() + .name("ts1") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .name("ts2") + .type(LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .name("ts3") + .type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .name("ts4").type().longType().noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } + + @Test + public void testEquivalenceDeepNesting() { + Schema level3Requested = SchemaBuilder.record("level3") + .fields() + .name("timestamp").type().longType().noDefault() + .endRecord(); + + Schema level3Table = SchemaBuilder.record("level3") + .fields() + .name("timestamp") + .type(LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG))) + .noDefault() + .endRecord(); + + Schema level2Requested = SchemaBuilder.record("level2") + .fields() + .name("level3").type(level3Requested).noDefault() + .endRecord(); + + Schema level2Table = SchemaBuilder.record("level2") + .fields() + .name("level3").type(level3Table).noDefault() + .endRecord(); + + Schema level1Requested = SchemaBuilder.record("level1") + .fields() + .name("level2").type(level2Requested).noDefault() + .endRecord(); + + Schema level1Table = SchemaBuilder.record("level1") + .fields() + .name("level2").type(level2Table).noDefault() + .endRecord(); + + Schema requestedSchema = SchemaBuilder.record("level0") + .fields() + .name("level1").type(level1Requested).noDefault() + .endRecord(); + + Schema tableSchema = SchemaBuilder.record("level0") + .fields() + .name("level1").type(level1Table).noDefault() + .endRecord(); + + assertRepairEquivalence(requestedSchema, tableSchema); + } +} diff --git a/hudi-hadoop-common/src/test/resources/parquet-java/all.avsc b/hudi-hadoop-common/src/test/resources/parquet-java/all.avsc new file mode 100644 index 0000000000000..116e98c519de4 --- /dev/null +++ b/hudi-hadoop-common/src/test/resources/parquet-java/all.avsc @@ -0,0 +1,110 @@ +/* + * 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. + */ +{ + "name" : "myrecord", + "namespace": "org.apache.parquet.avro", + "type" : "record", + "fields" : [ { + "name" : "mynull", + "type" : "null" + }, { + "name" : "myboolean", + "type" : "boolean" + }, { + "name" : "myint", + "type" : "int" + }, { + "name" : "mylong", + "type" : "long" + }, { + "name" : "myfloat", + "type" : "float" + }, { + "name" : "mydouble", + "type" : "double" + }, { + "name" : "mybytes", + "type" : "bytes" + }, { + "name" : "mystring", + "type" : "string" + }, { + "name" : "mynestedrecord", + "type" : { + "type" : "record", + "name" : "ignored1", + "namespace" : "", + "fields" : [ { + "name" : "mynestedint", + "type" : "int" + } ] + } + }, { + "name" : "myenum", + "type" : { + "type" : "enum", + "name" : "ignored2", + "namespace" : "", + "symbols" : [ "a", "b" ] + } + }, { + "name" : "myarray", + "type" : { + "type" : "array", + "items" : "int" + } + }, { + "name" : "myemptyarray", + "type" : { + "type" : "array", + "items" : "int" + } + }, { + "name" : "myoptionalarray", + "type" : [ "null", { + "type" : "array", + "items" : "int" + }] + }, { + "name" : "myarrayofoptional", + "type" : { + "type" : "array", + "items" : [ "null", "int" ] + } + }, { + "name" : "mymap", + "type" : { + "type" : "map", + "values" : "int" + } + }, { + "name" : "myemptymap", + "type" : { + "type" : "map", + "values" : "int" + } + }, { + "name" : "myfixed", + "type" : { + "type" : "fixed", + "name" : "ignored3", + "namespace" : "", + "size" : 1 + } + } ] +} \ No newline at end of file diff --git a/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetNewBehavior.avsc b/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetNewBehavior.avsc new file mode 100644 index 0000000000000..606213cb16830 --- /dev/null +++ b/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetNewBehavior.avsc @@ -0,0 +1,108 @@ +/* + * 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. + */ +{ + "name" : "myrecord", + "namespace": "org.apache.parquet.avro", + "type" : "record", + "fields" : [ { + "name" : "myboolean", + "type" : "boolean" + }, { + "name" : "myint", + "type" : "int" + }, { + "name" : "mylong", + "type" : "long" + }, { + "name" : "myfloat", + "type" : "float" + }, { + "name" : "mydouble", + "type" : "double" + }, { + "name" : "mybytes", + "type" : "bytes" + }, { + "name" : "mystring", + "type" : "string" + }, { + "name" : "mynestedrecord", + "type" : { + "type" : "record", + "name" : "mynestedrecord", + "namespace" : "", + "fields" : [ { + "name" : "mynestedint", + "type" : "int" + } ] + } + }, { + "name" : "myenum", + "type" : "string" + }, { + "name" : "myarray", + "type" : { + "type" : "array", + "items" : "int" + } + }, { + "name" : "myoptionalarray", + "type" : [ "null", { + "type" : "array", + "items" : "int" + }], + "default" : null + }, { + "name" : "myarrayofoptional", + "type" : { + "type" : "array", + "items" : ["null", "int"] + } + }, { + "name" : "myrecordarray", + "type" : { + "type" : "array", + "items" : { + "type" : "record", + "name" : "array", + "namespace" : "", + "fields" : [ { + "name" : "a", + "type" : "int" + }, { + "name" : "b", + "type" : "int" + } ] + } + } + }, { + "name" : "mymap", + "type" : { + "type" : "map", + "values" : "int" + } + }, { + "name" : "myfixed", + "type" : { + "type" : "fixed", + "name" : "myfixed", + "namespace" : "", + "size" : 1 + } + } ] +} \ No newline at end of file diff --git a/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetOldBehavior.avsc b/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetOldBehavior.avsc new file mode 100644 index 0000000000000..7a98a74633559 --- /dev/null +++ b/hudi-hadoop-common/src/test/resources/parquet-java/allFromParquetOldBehavior.avsc @@ -0,0 +1,117 @@ +/* + * 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. + */ +{ + "name" : "myrecord", + "namespace": "org.apache.parquet.avro", + "type" : "record", + "fields" : [ { + "name" : "myboolean", + "type" : "boolean" + }, { + "name" : "myint", + "type" : "int" + }, { + "name" : "mylong", + "type" : "long" + }, { + "name" : "myfloat", + "type" : "float" + }, { + "name" : "mydouble", + "type" : "double" + }, { + "name" : "mybytes", + "type" : "bytes" + }, { + "name" : "mystring", + "type" : "string" + }, { + "name" : "mynestedrecord", + "type" : { + "type" : "record", + "name" : "mynestedrecord", + "namespace" : "", + "fields" : [ { + "name" : "mynestedint", + "type" : "int" + } ] + } + }, { + "name" : "myenum", + "type" : "string" + }, { + "name" : "myarray", + "type" : { + "type" : "array", + "items" : "int" + } + }, { + "name" : "myoptionalarray", + "type" : [ "null", { + "type" : "array", + "items" : "int" + }], + "default" : null + }, { + "name" : "myarrayofoptional", + "type" : { + "type" : "array", + "items" : { + "type": "record", + "name": "list", + "namespace": "", + "fields": [ { + "name": "element", + "type": ["null", "int"], + "default": null + } ] + } + } + }, { + "name" : "myrecordarray", + "type" : { + "type" : "array", + "items" : { + "type" : "record", + "name" : "array", + "namespace" : "", + "fields" : [ { + "name" : "a", + "type" : "int" + }, { + "name" : "b", + "type" : "int" + } ] + } + } + }, { + "name" : "mymap", + "type" : { + "type" : "map", + "values" : "int" + } + }, { + "name" : "myfixed", + "type" : { + "type" : "fixed", + "name" : "myfixed", + "namespace" : "", + "size" : 1 + } + } ] +} \ No newline at end of file diff --git a/hudi-hadoop-common/src/test/resources/parquet-java/fixedToInt96.avsc b/hudi-hadoop-common/src/test/resources/parquet-java/fixedToInt96.avsc new file mode 100644 index 0000000000000..ca1e505ec3380 --- /dev/null +++ b/hudi-hadoop-common/src/test/resources/parquet-java/fixedToInt96.avsc @@ -0,0 +1,97 @@ +/* + * 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. + */ +{ + "name": "fixedToInt96", + "namespace": "org.apache.parquet.avro", + "type": "record", + "fields": [ + { + "name": "int96", + "type": { + "type": "fixed", + "name": "ignored1", + "namespace": "", + "size": 12 + } + }, + { + "name": "notanint96", + "type": { + "type": "fixed", + "name": "ignored2", + "namespace": "", + "size": 12 + } + }, + { + "name": "mynestedrecord", + "type": { + "type": "record", + "name": "ignored3", + "namespace": "", + "fields": [ + { + "name": "int96inrecord", + "type": { + "type": "fixed", + "name": "ignored4", + "namespace": "", + "size": 12 + } + }, + { + "name": "myarrayofoptional", + "type": { + "type": "array", + "items": [ + "null", + { + "type": "fixed", + "name": "ignored5", + "namespace": "", + "size": 12 + } + ] + } + }, + { + "name": "mymap", + "type": { + "type": "map", + "values": { + "type": "fixed", + "name": "ignored6", + "namespace": "", + "size": 12 + } + } + } + ] + } + }, + { + "name": "onebytefixed", + "type": { + "type": "fixed", + "name": "ignored7", + "namespace": "", + "size": 1 + } + } + ] +} \ No newline at end of file diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hadoop/hive/serde2/avro/HiveTypeUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hadoop/hive/serde2/avro/HiveTypeUtils.java index 0c4199e2ad664..96fba474d7d71 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hadoop/hive/serde2/avro/HiveTypeUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hadoop/hive/serde2/avro/HiveTypeUtils.java @@ -238,7 +238,7 @@ private static TypeInfo generateTypeInfoWorker(Schema schema, // Avro requires NULLable types to be defined as unions of some type T // and NULL. This is annoying and we're going to hide it from the user. if (AvroSchemaUtils.isNullable(schema)) { - return generateTypeInfo(AvroSchemaUtils.resolveNullableSchema(schema), seenSchemas); + return generateTypeInfo(AvroSchemaUtils.getNonNullTypeFromUnion(schema), seenSchemas); } Schema.Type type = schema.getType(); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveHoodieReaderContext.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveHoodieReaderContext.java index 5e282accad8e4..1d36f8bcb26bd 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveHoodieReaderContext.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveHoodieReaderContext.java @@ -57,6 +57,7 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.schema.AvroSchemaRepair; import java.io.IOException; import java.util.Collections; @@ -127,8 +128,8 @@ private ClosableIterator getFileRecordIterator(StoragePath filePa // mdt file schema irregular and does not work with this logic. Also, log file evolution is handled inside the log block boolean isParquetOrOrc = filePath.getFileExtension().equals(HoodieFileFormat.PARQUET.getFileExtension()) || filePath.getFileExtension().equals(HoodieFileFormat.ORC.getFileExtension()); - Schema avroFileSchema = isParquetOrOrc ? HoodieIOFactory.getIOFactory(storage) - .getFileFormatUtils(filePath).readAvroSchema(storage, filePath) : dataSchema; + Schema avroFileSchema = AvroSchemaRepair.repairLogicalTypes(isParquetOrOrc ? HoodieIOFactory.getIOFactory(storage) + .getFileFormatUtils(filePath).readAvroSchema(storage, filePath) : dataSchema, dataSchema); Schema actualRequiredSchema = isParquetOrOrc ? AvroSchemaUtils.pruneDataSchema(avroFileSchema, requiredSchema, Collections.emptySet()) : requiredSchema; JobConf jobConfCopy = new JobConf(storage.getConf().unwrapAs(Configuration.class)); if (getNeedsBootstrapMerge()) { @@ -146,7 +147,7 @@ private ClosableIterator getFileRecordIterator(StoragePath filePa partitionCols.stream().filter(c -> avroFileSchema.getField(c) != null)).collect(Collectors.toList())); setSchemas(jobConfCopy, modifiedDataSchema, actualRequiredSchema); InputSplit inputSplit = new FileSplit(new Path(filePath.toString()), start, length, hosts); - RecordReader recordReader = readerCreator.getRecordReader(inputSplit, jobConfCopy); + RecordReader recordReader = readerCreator.getRecordReader(inputSplit, jobConfCopy, modifiedDataSchema); if (firstRecordReader == null) { firstRecordReader = recordReader; } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieFileGroupReaderBasedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieFileGroupReaderBasedRecordReader.java index 450bb056d6bdd..c74ad8dd2e5a6 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieFileGroupReaderBasedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieFileGroupReaderBasedRecordReader.java @@ -93,7 +93,8 @@ public class HoodieFileGroupReaderBasedRecordReader implements RecordReader getRecordReader( final org.apache.hadoop.mapred.InputSplit split, - final org.apache.hadoop.mapred.JobConf job + final org.apache.hadoop.mapred.JobConf job, + Schema dataSchema ) throws IOException; } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java index 327558276bc5b..f34c14304d9ec 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java @@ -123,15 +123,15 @@ public RecordReader getRecordReader(final InputSpli return super.getRecordReader(split, job, reporter); } if (supportAvroRead && HoodieColumnProjectionUtils.supportTimestamp(job)) { - return new HoodieFileGroupReaderBasedRecordReader((s, j) -> { + return new HoodieFileGroupReaderBasedRecordReader((s, j, d) -> { try { - return new ParquetRecordReaderWrapper(new HoodieTimestampAwareParquetInputFormat(Option.empty()), s, j, reporter); + return new ParquetRecordReaderWrapper(new HoodieTimestampAwareParquetInputFormat(Option.empty(), Option.of(d)), s, j, reporter); } catch (InterruptedException e) { throw new RuntimeException(e); } }, split, job); } else { - return new HoodieFileGroupReaderBasedRecordReader((s, j) -> super.getRecordReader(s, j, reporter), split, job); + return new HoodieFileGroupReaderBasedRecordReader((s, j, d) -> super.getRecordReader(s, j, reporter), split, job); } } catch (final IOException e) { throw new RuntimeException("Cannot create a RecordReaderWrapper", e); @@ -174,7 +174,7 @@ private RecordReader getRecordReaderInternal(InputS Option internalSchemaOption) throws IOException { try { if (supportAvroRead && HoodieColumnProjectionUtils.supportTimestamp(job)) { - return new ParquetRecordReaderWrapper(new HoodieTimestampAwareParquetInputFormat(internalSchemaOption), split, job, reporter); + return new ParquetRecordReaderWrapper(new HoodieTimestampAwareParquetInputFormat(internalSchemaOption, Option.empty()), split, job, reporter); } else { return super.getRecordReader(split, job, reporter); } 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 6bc7c21ccc02e..aa52b108e0056 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 @@ -55,31 +55,34 @@ public class HoodieAvroParquetReader extends RecordReader { private final ParquetRecordReader parquetRecordReader; private Schema baseSchema; - public HoodieAvroParquetReader(InputSplit inputSplit, Configuration conf, Option internalSchemaOption) throws IOException { - // get base schema - ParquetMetadata fileFooter = - ParquetFileReader.readFooter(conf, ((ParquetInputSplit) inputSplit).getPath(), ParquetMetadataConverter.NO_FILTER); - MessageType messageType = fileFooter.getFileMetaData().getSchema(); - baseSchema = getAvroSchemaConverter(conf).convert(messageType); - - if (internalSchemaOption.isPresent()) { - // do schema reconciliation in case there exists read column which is not in the file schema. - InternalSchema mergedInternalSchema = new InternalSchemaMerger( - AvroInternalSchemaConverter.convert(baseSchema), - internalSchemaOption.get(), - true, - true).mergeSchema(); - baseSchema = AvroInternalSchemaConverter.convert(mergedInternalSchema, baseSchema.getFullName()); + public HoodieAvroParquetReader(InputSplit inputSplit, Configuration conf, Option internalSchemaOption, Option dataSchema) throws IOException { + if (dataSchema.isPresent()) { + baseSchema = dataSchema.get(); + } else { + // get base schema + ParquetMetadata fileFooter = + ParquetFileReader.readFooter(conf, ((ParquetInputSplit) inputSplit).getPath(), ParquetMetadataConverter.NO_FILTER); + MessageType messageType = fileFooter.getFileMetaData().getSchema(); + baseSchema = getAvroSchemaConverter(conf).convert(messageType); + + if (internalSchemaOption.isPresent()) { + // do schema reconciliation in case there exists read column which is not in the file schema. + InternalSchema mergedInternalSchema = new InternalSchemaMerger( + AvroInternalSchemaConverter.convert(baseSchema), + internalSchemaOption.get(), + true, + true).mergeSchema(); + baseSchema = AvroInternalSchemaConverter.convert(mergedInternalSchema, baseSchema.getFullName()); + } + + // if exists read columns, we need to filter columns. + List readColNames = Arrays.asList(HoodieColumnProjectionUtils.getReadColumnNames(conf)); + if (!readColNames.isEmpty()) { + Schema filterSchema = HoodieAvroUtils.generateProjectionSchema(baseSchema, readColNames); + AvroReadSupport.setAvroReadSchema(conf, filterSchema); + AvroReadSupport.setRequestedProjection(conf, filterSchema); + } } - - // if exists read columns, we need to filter columns. - List readColNames = Arrays.asList(HoodieColumnProjectionUtils.getReadColumnNames(conf)); - if (!readColNames.isEmpty()) { - Schema filterSchema = HoodieAvroUtils.generateProjectionSchema(baseSchema, readColNames); - AvroReadSupport.setAvroReadSchema(conf, filterSchema); - AvroReadSupport.setRequestedProjection(conf, filterSchema); - } - parquetRecordReader = new ParquetRecordReader<>(new AvroReadSupport<>(), getFilter(conf)); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieTimestampAwareParquetInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieTimestampAwareParquetInputFormat.java index 4f88d1a4dfcf4..4c08b346eed61 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieTimestampAwareParquetInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieTimestampAwareParquetInputFormat.java @@ -18,6 +18,7 @@ package org.apache.hudi.hadoop.avro; +import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.ArrayWritable; import org.apache.hadoop.mapreduce.InputSplit; @@ -36,10 +37,12 @@ */ public class HoodieTimestampAwareParquetInputFormat extends ParquetInputFormat { private final Option internalSchemaOption; + private final Option dataSchema; - public HoodieTimestampAwareParquetInputFormat(Option internalSchemaOption) { + public HoodieTimestampAwareParquetInputFormat(Option internalSchemaOption, Option dataSchema) { super(); this.internalSchemaOption = internalSchemaOption; + this.dataSchema = dataSchema; } @Override @@ -47,6 +50,6 @@ public RecordReader createRecordReader( InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException { Configuration conf = ContextUtil.getConfiguration(taskAttemptContext); - return new HoodieAvroParquetReader(inputSplit, conf, internalSchemaOption); + return new HoodieAvroParquetReader(inputSplit, conf, internalSchemaOption, dataSchema); } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HiveAvroSerializer.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HiveAvroSerializer.java index 4278439fbb14e..89ddf1ff29535 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HiveAvroSerializer.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HiveAvroSerializer.java @@ -66,7 +66,7 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema; +import static org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion; import static org.apache.hudi.avro.AvroSchemaUtils.resolveUnionSchema; import static org.apache.hudi.avro.HoodieAvroUtils.isMetadataField; @@ -83,7 +83,7 @@ public class HiveAvroSerializer { private static final Logger LOG = LoggerFactory.getLogger(HiveAvroSerializer.class); public HiveAvroSerializer(Schema schema) { - schema = AvroSchemaUtils.resolveNullableSchema(schema); + schema = AvroSchemaUtils.getNonNullTypeFromUnion(schema); if (schema.getType() != Schema.Type.RECORD) { throw new IllegalArgumentException("Expected record schema, but got: " + schema); } @@ -195,7 +195,7 @@ private FieldContext extractFieldFromRecord(ArrayWritable record, StructObjectIn int fieldIdx = schemaField.pos(); TypeInfo fieldTypeInfo = fieldTypes.get(fieldIdx); - Schema fieldSchema = resolveNullableSchema(schemaField.schema()); + Schema fieldSchema = getNonNullTypeFromUnion(schemaField.schema()); StructField structField = structObjectInspector.getStructFieldRef(fieldName); if (structField == null) { @@ -289,7 +289,7 @@ private Object serialize(TypeInfo typeInfo, ObjectInspector fieldOI, Object stru return null; } - schema = resolveNullableSchema(schema); + schema = getNonNullTypeFromUnion(schema); /* Because we use Hive's 'string' type when Avro calls for enum, we have to expressly check for enum-ness */ if (Schema.Type.ENUM.equals(schema.getType())) { @@ -430,7 +430,7 @@ private Object serializeList(ListTypeInfo typeInfo, ListObjectInspector fieldOI, ObjectInspector listElementObjectInspector = fieldOI.getListElementObjectInspector(); // NOTE: We have to resolve nullable schema, since Avro permits array elements // to be null - Schema arrayNestedType = resolveNullableSchema(schema.getElementType()); + Schema arrayNestedType = getNonNullTypeFromUnion(schema.getElementType()); Schema elementType; if (listElementObjectInspector.getCategory() == ObjectInspector.Category.PRIMITIVE) { elementType = arrayNestedType; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieArrayWritableAvroUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieArrayWritableAvroUtils.java index 64087331c63bd..31f564dbce5d2 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieArrayWritableAvroUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieArrayWritableAvroUtils.java @@ -71,8 +71,8 @@ private static Writable rewriteRecordWithNewSchema(Writable writable, Schema old if (writable == null) { return null; } - Schema oldSchema = AvroSchemaUtils.resolveNullableSchema(oldAvroSchema); - Schema newSchema = AvroSchemaUtils.resolveNullableSchema(newAvroSchema); + Schema oldSchema = AvroSchemaUtils.getNonNullTypeFromUnion(oldAvroSchema); + Schema newSchema = AvroSchemaUtils.getNonNullTypeFromUnion(newAvroSchema); if (areSchemasProjectionEquivalent(oldSchema, newSchema)) { return writable; } @@ -107,7 +107,7 @@ private static Writable rewriteRecordWithNewSchemaInternal(Writable writable, Sc } else if (!isNullable(newField.schema()) && newField.defaultVal() == null) { throw new SchemaCompatibilityException("Field " + createFullName(fieldNames) + " has no default value and is non-nullable"); } else if (newField.defaultVal() != null) { - switch (AvroSchemaUtils.resolveNullableSchema(newField.schema()).getType()) { + switch (AvroSchemaUtils.getNonNullTypeFromUnion(newField.schema()).getType()) { case BOOLEAN: values[i] = new BooleanWritable((Boolean) newField.defaultVal()); break; diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index dbcd065552f05..8bb7b2f1feac0 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -26,7 +26,7 @@ import org.apache.hudi.HoodieConversionUtils.{toProperties, toScalaOption} import org.apache.hudi.HoodieSparkSqlWriter.StreamingWriteParams import org.apache.hudi.HoodieSparkSqlWriterInternal.{handleInsertDuplicates, shouldDropDuplicatesForInserts, shouldFailWhenDuplicatesFound} import org.apache.hudi.HoodieWriterUtils._ -import org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema +import org.apache.hudi.avro.AvroSchemaUtils.getNonNullTypeFromUnion import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.client.{HoodieWriteResult, SparkRDDWriteClient} import org.apache.hudi.client.common.HoodieSparkEngineContext @@ -791,7 +791,7 @@ class HoodieSparkSqlWriterInternal { def validateSchemaForHoodieIsDeleted(schema: Schema): Unit = { if (schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD) != null && - resolveNullableSchema(schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).schema()).getType != Schema.Type.BOOLEAN) { + getNonNullTypeFromUnion(schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).schema()).getType != Schema.Type.BOOLEAN) { throw new HoodieException(HoodieRecord.HOODIE_IS_DELETED_FIELD + " has to be BOOLEAN type. Passed in dataframe's schema has type " + schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).schema().getType) } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala index 6dd0464f350f5..471f9a3f0a6e2 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala @@ -47,6 +47,7 @@ import org.apache.hudi.storage.{StorageConfiguration, StoragePath} import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord import org.apache.hadoop.conf.Configuration +import org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter import org.apache.spark.sql.HoodieCatalystExpressionUtils.generateUnsafeProjection import org.apache.spark.sql.HoodieInternalRowUtils import org.apache.spark.sql.avro.HoodieAvroDeserializer @@ -138,6 +139,14 @@ class CDCFileGroupIterator(split: HoodieCDCFileGroupSplit, private lazy val sparkPartitionedFileUtils = sparkAdapter.getSparkPartitionedFileUtils + private lazy val tableSchemaOpt = if (avroSchema != null) { + val hadoopConf = storage.getConf.unwrapAs(classOf[Configuration]) + val parquetSchema = getAvroSchemaConverter(hadoopConf).convert(avroSchema) + org.apache.hudi.common.util.Option.of(parquetSchema) + } else { + org.apache.hudi.common.util.Option.empty[org.apache.parquet.schema.MessageType]() + } + /** * The deserializer used to convert the CDC GenericRecord to Spark InternalRow. */ @@ -400,7 +409,7 @@ class CDCFileGroupIterator(split: HoodieCDCFileGroupSplit, val pf = sparkPartitionedFileUtils.createPartitionedFile( InternalRow.empty, absCDCPath, 0, fileStatus.getLength) recordIter = baseFileReader.read(pf, originTableSchema.structTypeSchema, new StructType(), - toJavaOption(originTableSchema.internalSchema), Seq.empty, conf) + toJavaOption(originTableSchema.internalSchema), Seq.empty, conf, tableSchemaOpt) .map(record => BufferedRecords.fromEngineRecord(record, avroSchema, readerContext.getRecordContext, orderingFieldNames, false)) case BASE_FILE_DELETE => assert(currentCDCFileSplit.getBeforeFileSlice.isPresent) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/orc/SparkOrcReaderBase.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/orc/SparkOrcReaderBase.scala index 776dd2c567e3b..ca16e323246db 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/orc/SparkOrcReaderBase.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/orc/SparkOrcReaderBase.scala @@ -58,7 +58,7 @@ abstract class SparkOrcReaderBase(enableVectorizedReader: Boolean, */ override def read(file: PartitionedFile, requiredSchema: StructType, partitionSchema: StructType, internalSchemaOpt: util.Option[InternalSchema], filters: Seq[Filter], - storageConf: StorageConfiguration[Configuration]): Iterator[InternalRow] = { + storageConf: StorageConfiguration[Configuration], tableSchemaOpt: util.Option[org.apache.parquet.schema.MessageType]): Iterator[InternalRow] = { val resultSchema = StructType(requiredSchema.fields ++ partitionSchema.fields) val conf = storageConf.unwrap() 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 6179dc9317081..0cf9a797615d4 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.hudi.{AvroConversionUtils, HoodieFileIndex, HoodiePartitionCDCFileGroupMapping, HoodiePartitionFileSliceMapping, HoodieTableSchema, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} +import org.apache.hudi.{AvroConversionUtils, HoodieFileIndex, HoodiePartitionCDCFileGroupMapping, HoodiePartitionFileSliceMapping, HoodieSparkUtils, HoodieTableSchema, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} import org.apache.hudi.avro.AvroSchemaUtils import org.apache.hudi.cdc.{CDCFileGroupIterator, CDCRelation, HoodieCDCFileGroupSplit} import org.apache.hudi.client.common.HoodieSparkEngineContext @@ -25,8 +25,9 @@ import org.apache.hudi.client.utils.SparkInternalSchemaConverter import org.apache.hudi.common.config.{HoodieMemoryConfig, TypedProperties} import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.HoodieFileFormat -import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, ParquetTableSchemaResolver} import org.apache.hudi.common.table.read.HoodieFileGroupReader +import org.apache.hudi.common.util.{Option => HOption} import org.apache.hudi.common.util.collection.ClosableIterator import org.apache.hudi.data.CloseableIteratorListener import org.apache.hudi.exception.HoodieNotSupportedException @@ -39,6 +40,7 @@ import org.apache.avro.Schema import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce.Job +import org.apache.parquet.schema.{AvroSchemaRepair, MessageType} import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging import org.apache.spark.sql.HoodieCatalystExpressionUtils.generateUnsafeProjection @@ -87,6 +89,14 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, private lazy val avroTableSchema = new Schema.Parser().parse(tableSchema.avroSchemaStr) + private lazy val tableSchemaAsMessageType: HOption[MessageType] = { + HOption.ofNullable( + ParquetTableSchemaResolver.convertAvroSchemaToParquet(avroTableSchema, new Configuration()) + ) + } + + private lazy val supportBatchTimestampRepair = HoodieSparkUtils.gteqSpark3_5 || !AvroSchemaRepair.hasTimestampMillisField(avroTableSchema) + override def shortName(): String = "HudiFileGroup" override def toString: String = "HoodieFileGroupReaderBasedFileFormat" @@ -118,7 +128,7 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, */ override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { val conf = sparkSession.sessionState.conf - val parquetBatchSupported = ParquetUtils.isBatchReadSupportedForSchema(conf, schema) + val parquetBatchSupported = ParquetUtils.isBatchReadSupportedForSchema(conf, schema) && supportBatchTimestampRepair val orcBatchSupported = conf.orcVectorizedReaderEnabled && schema.forall(s => OrcUtils.supportColumnarReads( s.dataType, sparkSession.sessionState.conf.orcVectorizedReaderNestedColumnEnabled)) @@ -165,10 +175,10 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, } } - private lazy val internalSchemaOpt: org.apache.hudi.common.util.Option[InternalSchema] = if (tableSchema.internalSchema.isEmpty) { - org.apache.hudi.common.util.Option.empty() + private lazy val internalSchemaOpt: HOption[InternalSchema] = if (tableSchema.internalSchema.isEmpty) { + HOption.empty() } else { - org.apache.hudi.common.util.Option.of(tableSchema.internalSchema.get) + HOption.of(tableSchema.internalSchema.get) } override def isSplitable(sparkSession: SparkSession, @@ -186,6 +196,7 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, val isCount = requiredSchema.isEmpty && !isMOR && !isIncremental val augmentedStorageConf = new HadoopStorageConfiguration(hadoopConf).getInline setSchemaEvolutionConfigs(augmentedStorageConf) + augmentedStorageConf.set("logicalTimestampField.repair.enable", supportBatchTimestampRepair.toString) val (remainingPartitionSchemaArr, fixedPartitionIndexesArr) = partitionSchema.fields.toSeq.zipWithIndex.filter(p => !mandatoryFields.contains(p._1.name)).unzip // The schema of the partition cols we want to append the value instead of reading from the file @@ -202,6 +213,8 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, val requestedAvroSchema = AvroSchemaUtils.pruneDataSchema(avroTableSchema, AvroConversionUtils.convertStructTypeToAvroSchema(requestedSchema, sanitizedTableName), exclusionFields) val dataAvroSchema = AvroSchemaUtils.pruneDataSchema(avroTableSchema, AvroConversionUtils.convertStructTypeToAvroSchema(dataSchema, sanitizedTableName), exclusionFields) + spark.sessionState.conf.setConfString("spark.sql.parquet.enableVectorizedReader", supportVectorizedRead.toString) + val baseFileReader = spark.sparkContext.broadcast(buildBaseFileReader(spark, options, augmentedStorageConf.unwrap(), dataSchema, supportVectorizedRead)) val fileGroupBaseFileReader = if (isMOR && supportVectorizedRead) { // for file group reader to perform read, we always need to read the record without vectorized reader because our merging is based on row level. @@ -222,6 +235,7 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, .builder().setConf(augmentedStorageConf).setBasePath(tablePath).build (file: PartitionedFile) => { + // executor val storageConf = new HadoopStorageConfiguration(broadcastedStorageConf.value.value) val iter = file.partitionValues match { // Snapshot or incremental queries. @@ -231,6 +245,7 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, fileSliceMapping.getSlice(fileGroupName) match { case Some(fileSlice) if !isCount && (requiredSchema.nonEmpty || fileSlice.getLogFiles.findAny().isPresent) => val readerContext = new SparkFileFormatInternalRowReaderContext(fileGroupBaseFileReader.value, filters, requiredFilters, storageConf, metaClient.getTableConfig) + readerContext.setEnableLogicalTimestampFieldRepair(storageConf.getBoolean("logicalTimestampField.repair.enable", true)) val props = metaClient.getTableConfig.getProps options.foreach(kv => props.setProperty(kv._1, kv._2)) props.put(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE.key(), String.valueOf(maxMemoryPerCompaction)) @@ -365,20 +380,21 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, } } + // executor private def readBaseFile(file: PartitionedFile, parquetFileReader: SparkColumnarFileReader, requestedSchema: StructType, remainingPartitionSchema: StructType, fixedPartitionIndexes: Set[Int], requiredSchema: StructType, partitionSchema: StructType, outputSchema: StructType, filters: Seq[Filter], storageConf: StorageConfiguration[Configuration]): Iterator[InternalRow] = { if (remainingPartitionSchema.fields.length == partitionSchema.fields.length) { //none of partition fields are read from the file, so the reader will do the appending for us - parquetFileReader.read(file, requiredSchema, partitionSchema, internalSchemaOpt, filters, storageConf) + parquetFileReader.read(file, requiredSchema, partitionSchema, internalSchemaOpt, filters, storageConf, tableSchemaAsMessageType) } else if (remainingPartitionSchema.fields.length == 0) { //we read all of the partition fields from the file val pfileUtils = sparkAdapter.getSparkPartitionedFileUtils //we need to modify the partitioned file so that the partition values are empty val modifiedFile = pfileUtils.createPartitionedFile(InternalRow.empty, pfileUtils.getPathFromPartitionedFile(file), file.start, file.length) //and we pass an empty schema for the partition schema - parquetFileReader.read(modifiedFile, outputSchema, new StructType(), internalSchemaOpt, filters, storageConf) + parquetFileReader.read(modifiedFile, outputSchema, new StructType(), internalSchemaOpt, filters, storageConf, tableSchemaAsMessageType) } else { //need to do an additional projection here. The case in mind is that partition schema is "a,b,c" mandatoryFields is "a,c", //then we will read (dataSchema + a + c) and append b. So the final schema will be (data schema + a + c +b) @@ -386,7 +402,7 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, val pfileUtils = sparkAdapter.getSparkPartitionedFileUtils val partitionValues = getFixedPartitionValues(file.partitionValues, partitionSchema, fixedPartitionIndexes) val modifiedFile = pfileUtils.createPartitionedFile(partitionValues, pfileUtils.getPathFromPartitionedFile(file), file.start, file.length) - val iter = parquetFileReader.read(modifiedFile, requestedSchema, remainingPartitionSchema, internalSchemaOpt, filters, storageConf) + val iter = parquetFileReader.read(modifiedFile, requestedSchema, remainingPartitionSchema, internalSchemaOpt, filters, storageConf, tableSchemaAsMessageType) projectIter(iter, StructType(requestedSchema.fields ++ remainingPartitionSchema.fields), outputSchema) } } 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 5fb471f2aec52..14b334866a93d 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 @@ -43,6 +43,7 @@ abstract class SparkParquetReaderBase(enableVectorizedReader: Boolean, capacity: Int, returningBatch: Boolean, enableRecordFilter: Boolean, + enableLogicalTimestampRepair: Boolean, timeZoneId: Option[String]) extends SparkColumnarFileReader { /** * Read an individual parquet file @@ -53,6 +54,7 @@ abstract class SparkParquetReaderBase(enableVectorizedReader: Boolean, * @param internalSchemaOpt option of internal schema for schema.on.read * @param filters filters for data skipping. Not guaranteed to be used; the spark plan will also apply the filters. * @param storageConf the hadoop conf + * @param tableSchemaOpt option of table schema for timestamp precision conversion * @return iterator of rows read from the file output type says [[InternalRow]] but could be [[ColumnarBatch]] */ final def read(file: PartitionedFile, @@ -60,7 +62,8 @@ abstract class SparkParquetReaderBase(enableVectorizedReader: Boolean, partitionSchema: StructType, internalSchemaOpt: util.Option[InternalSchema], filters: Seq[Filter], - storageConf: StorageConfiguration[Configuration]): Iterator[InternalRow] = { + storageConf: StorageConfiguration[Configuration], + tableSchemaOpt: util.Option[org.apache.parquet.schema.MessageType] = util.Option.empty()): Iterator[InternalRow] = { val conf = storageConf.unwrapCopy() conf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) conf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) @@ -78,7 +81,7 @@ abstract class SparkParquetReaderBase(enableVectorizedReader: Boolean, } ParquetWriteSupport.setSchema(requiredSchema, conf) - doRead(file, requiredSchema, partitionSchema, internalSchemaOpt, filters, conf) + doRead(file, requiredSchema, partitionSchema, internalSchemaOpt, filters, conf, tableSchemaOpt) } /** @@ -90,6 +93,7 @@ abstract class SparkParquetReaderBase(enableVectorizedReader: Boolean, * @param internalSchemaOpt option of internal schema for schema.on.read * @param filters filters for data skipping. Not guaranteed to be used; the spark plan will also apply the filters. * @param sharedConf the hadoop conf + * @param tableSchemaOpt option of table schema for timestamp precision conversion * @return iterator of rows read from the file output type says [[InternalRow]] but could be [[ColumnarBatch]] */ protected def doRead(file: PartitionedFile, @@ -97,7 +101,8 @@ abstract class SparkParquetReaderBase(enableVectorizedReader: Boolean, partitionSchema: StructType, internalSchemaOpt: util.Option[InternalSchema], filters: Seq[Filter], - sharedConf: Configuration): Iterator[InternalRow] + sharedConf: Configuration, + tableSchemaOpt: util.Option[org.apache.parquet.schema.MessageType]): Iterator[InternalRow] } trait SparkParquetReaderBuilder { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala index d49476f5e1805..6ebdebb4a9279 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala @@ -20,10 +20,10 @@ package org.apache.spark.sql.hudi.command.payload import org.apache.hudi.AvroConversionUtils.{convertAvroSchemaToStructType, convertStructTypeToAvroSchema} import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.SparkAdapterSupport.sparkAdapter -import org.apache.hudi.avro.AvroSchemaUtils.{isNullable, resolveNullableSchema} +import org.apache.hudi.avro.AvroSchemaUtils.{getNonNullTypeFromUnion, isNullable} import org.apache.hudi.avro.HoodieAvroUtils -import org.apache.hudi.avro.HoodieAvroUtils.{bytesToAvro, createNewSchemaField} -import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodiePayloadProps, HoodieRecord, HoodieRecordPayload, OverwriteWithLatestAvroPayload, SerializableIndexedRecord} +import org.apache.hudi.avro.HoodieAvroUtils.createNewSchemaField +import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodiePayloadProps, HoodieRecord, HoodieRecordPayload, OverwriteWithLatestAvroPayload} import org.apache.hudi.common.util.{BinaryUtil, ConfigUtils, HoodieRecordUtils, Option => HOption, OrderingValues, StringUtils, ValidationUtils} import org.apache.hudi.common.util.ValidationUtils.checkState import org.apache.hudi.config.HoodieWriteConfig @@ -549,8 +549,8 @@ object ExpressionPayload { .zipWithIndex .foreach { case ((expectedField, targetField), idx) => - val expectedFieldSchema = resolveNullableSchema(expectedField.schema()) - val targetFieldSchema = resolveNullableSchema(targetField.schema()) + val expectedFieldSchema = getNonNullTypeFromUnion(expectedField.schema()) + val targetFieldSchema = getNonNullTypeFromUnion(targetField.schema()) val equal = Objects.equals(expectedFieldSchema, targetFieldSchema) ValidationUtils.checkState(equal, 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 5c0f18fab7ee9..363c8c34eec55 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 @@ -271,8 +271,8 @@ public HoodieWriteConfig.Builder getConfigBuilder() { private List generateInserts(String instant, long ts, int count) { Schema schema = getSchema(); Schema decimalSchema = schema.getField("decimal_field").schema(); - Schema nestedSchema = AvroSchemaUtils.resolveNullableSchema(schema.getField("nested_record").schema()); - Schema enumSchema = AvroSchemaUtils.resolveNullableSchema(schema.getField("enum_field").schema()); + Schema nestedSchema = AvroSchemaUtils.getNonNullTypeFromUnion(schema.getField("nested_record").schema()); + Schema enumSchema = AvroSchemaUtils.getNonNullTypeFromUnion(schema.getField("enum_field").schema()); Random random = new Random(0); return IntStream.range(0, count) .mapToObj(i -> { diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_cow_read_v6.zip b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_cow_read_v6.zip new file mode 100644 index 0000000000000..1de8eadd923e6 Binary files /dev/null and b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_cow_read_v6.zip differ diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_cow_read_v8.zip b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_cow_read_v8.zip new file mode 100644 index 0000000000000..165dd4376cf29 Binary files /dev/null and b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_cow_read_v8.zip differ diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_cow_read_v9.zip b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_cow_read_v9.zip new file mode 100644 index 0000000000000..68030ad293c2f Binary files /dev/null and b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_cow_read_v9.zip differ diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v6.zip b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v6.zip new file mode 100644 index 0000000000000..e75f8c4909133 Binary files /dev/null and b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v6.zip differ diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v6_parquet_log.zip b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v6_parquet_log.zip new file mode 100644 index 0000000000000..22c849cc5ed37 Binary files /dev/null and b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v6_parquet_log.zip differ diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v8.zip b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v8.zip new file mode 100644 index 0000000000000..7d774b7349580 Binary files /dev/null and b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v8.zip differ diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v8_parquet_log.zip b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v8_parquet_log.zip new file mode 100644 index 0000000000000..230d2b6945a09 Binary files /dev/null and b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v8_parquet_log.zip differ diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v9.zip b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v9.zip new file mode 100644 index 0000000000000..3a93b28687189 Binary files /dev/null and b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v9.zip differ diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v9_parquet_log.zip b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v9_parquet_log.zip new file mode 100644 index 0000000000000..1f5d2e8de56d4 Binary files /dev/null and b/hudi-spark-datasource/hudi-spark/src/test/resources/trips_logical_types_json_mor_read_v9_parquet_log.zip differ diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index 8738030189395..abe4f714f0db2 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -17,7 +17,7 @@ package org.apache.hudi.functional -import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, QuickstartUtils, ScalaAssertionSupport} +import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkUtils, QuickstartUtils, ScalaAssertionSupport} import org.apache.hudi.DataSourceWriteOptions.{INLINE_CLUSTERING_ENABLE, KEYGENERATOR_CLASS_NAME} import org.apache.hudi.HoodieConversionUtils.toJavaOption import org.apache.hudi.QuickstartUtils.{convertToStringList, getQuickstartWriteConfigs} @@ -61,6 +61,8 @@ import org.junit.jupiter.api.function.Executable import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{Arguments, CsvSource, EnumSource, MethodSource, ValueSource} +import java.net.URI +import java.nio.file.Paths import java.sql.{Date, Timestamp} import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.function.Consumer @@ -1825,6 +1827,78 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup }) } + @ParameterizedTest + @CsvSource(Array("true, 6", "false, 6", "true, 8", "false, 8", "true, 9", "false, 9")) + def testLogicalTypesReadRepair(vectorizedReadEnabled: Boolean, tableVersion: Int): Unit = { + // Note: for spark 3.3 and 3.4 we should fall back to nonvectorized reader + // if that is not happening then this test will fail + val prevValue = spark.conf.get("spark.sql.parquet.enableVectorizedReader") + val prevTimezone = spark.conf.get("spark.sql.session.timeZone") + val propertyValue: String = System.getProperty("spark.testing") + try { + if (HoodieSparkUtils.isSpark3_3) { + System.setProperty("spark.testing", "true") + } + spark.conf.set("spark.sql.parquet.enableVectorizedReader", vectorizedReadEnabled.toString) + spark.conf.set("spark.sql.session.timeZone", "UTC") + val tableName = "trips_logical_types_json_cow_read_v" + tableVersion + val dataPath = "file://" + basePath + "/" + tableName + val zipOutput = Paths.get(new URI(dataPath)) + HoodieTestUtils.extractZipToDirectory("/" + tableName + ".zip", zipOutput, getClass) + val tableBasePath = zipOutput.toString + + val df = spark.read.format("org.apache.hudi") + .option("hoodie.metadata.enable", "false") + .load(tableBasePath) + + val rows = df.collect() + assertEquals(20, rows.length) + for (row <- rows) { + val hash = row.get(6).asInstanceOf[String].hashCode() + if ((hash & 1) == 0) { + assertEquals("2020-01-01T00:00:00.001Z", row.get(15).asInstanceOf[Timestamp].toInstant.toString) + assertEquals("2020-06-01T12:00:00.000001Z", row.get(16).asInstanceOf[Timestamp].toInstant.toString) + assertEquals("2015-05-20T12:34:56.001", row.get(17).toString) + assertEquals("2017-07-07T07:07:07.000001", row.get(18).toString) + } else { + assertEquals("2019-12-31T23:59:59.999Z", row.get(15).asInstanceOf[Timestamp].toInstant.toString) + assertEquals("2020-06-01T11:59:59.999999Z", row.get(16).asInstanceOf[Timestamp].toInstant.toString) + assertEquals("2015-05-20T12:34:55.999", row.get(17).toString) + assertEquals("2017-07-07T07:07:06.999999", row.get(18).toString) + } + } + assertEquals(10, df.filter("ts_millis > timestamp('2020-01-01 00:00:00Z')").count()) + assertEquals(10, df.filter("ts_millis < timestamp('2020-01-01 00:00:00Z')").count()) + assertEquals(0, df.filter("ts_millis > timestamp('2020-01-01 00:00:00.001Z')").count()) + assertEquals(0, df.filter("ts_millis < timestamp('2019-12-31 23:59:59.999Z')").count()) + + assertEquals(10, df.filter("ts_micros > timestamp('2020-06-01 12:00:00Z')").count()) + assertEquals(10, df.filter("ts_micros < timestamp('2020-06-01 12:00:00Z')").count()) + assertEquals(0, df.filter("ts_micros > timestamp('2020-06-01 12:00:00.000001Z')").count()) + assertEquals(0, df.filter("ts_micros < timestamp('2020-06-01 11:59:59.999999Z')").count()) + + assertEquals(10, df.filter("local_ts_millis > CAST('2015-05-20 12:34:56' AS TIMESTAMP_NTZ)").count()) + assertEquals(10, df.filter("local_ts_millis < CAST('2015-05-20 12:34:56' AS TIMESTAMP_NTZ)").count()) + assertEquals(0, df.filter("local_ts_millis > CAST('2015-05-20 12:34:56.001' AS TIMESTAMP_NTZ)").count()) + assertEquals(0, df.filter("local_ts_millis < CAST('2015-05-20 12:34:55.999' AS TIMESTAMP_NTZ)").count()) + + assertEquals(10, df.filter("local_ts_micros > CAST('2017-07-07 07:07:07' AS TIMESTAMP_NTZ)").count()) + assertEquals(10, df.filter("local_ts_micros < CAST('2017-07-07 07:07:07' AS TIMESTAMP_NTZ)").count()) + assertEquals(0, df.filter("local_ts_micros > CAST('2017-07-07 07:07:07.000001' AS TIMESTAMP_NTZ)").count()) + assertEquals(0, df.filter("local_ts_micros < CAST('2017-07-07 07:07:06.999999' AS TIMESTAMP_NTZ)").count()) + } finally { + spark.conf.set("spark.sql.parquet.enableVectorizedReader", prevValue) + spark.conf.set("spark.sql.session.timeZone", prevTimezone) + if (HoodieSparkUtils.isSpark3_3) { + if (propertyValue == null) { + System.clearProperty("spark.testing") + } else { + System.setProperty("spark.testing", propertyValue) + } + } + } + } + def getWriterReaderOpts(recordType: HoodieRecordType = HoodieRecordType.AVRO, opt: Map[String, String] = CommonOptionUtils.commonOpts, enableFileIndex: Boolean = DataSourceReadOptions.ENABLE_HOODIE_FILE_INDEX.defaultValue()): diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala index f98344df2d96b..1bc32c0669c97 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala @@ -17,7 +17,7 @@ package org.apache.hudi.functional -import org.apache.hudi.{ColumnStatsIndexSupport, DataSourceReadOptions, DataSourceUtils, DataSourceWriteOptions, DefaultSparkRecordMerger, HoodieDataSourceHelpers, ScalaAssertionSupport, SparkDatasetMixin} +import org.apache.hudi.{ColumnStatsIndexSupport, DataSourceReadOptions, DataSourceUtils, DataSourceWriteOptions, DefaultSparkRecordMerger, HoodieDataSourceHelpers, HoodieSparkUtils, ScalaAssertionSupport, SparkDatasetMixin} import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.HoodieConversionUtils.toJavaOption import org.apache.hudi.client.SparkRDDWriteClient @@ -53,7 +53,9 @@ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{Arguments, CsvSource, EnumSource, MethodSource, ValueSource} import java.io.File -import java.nio.file.Files +import java.net.URI +import java.nio.file.{Files, Paths} +import java.sql.Timestamp import java.util.function.Consumer import java.util.stream.Collectors @@ -1259,6 +1261,78 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin spark.read.format("hudi").load(basePath).count()) } + @ParameterizedTest + @CsvSource(Array("avro, 6", "parquet, 6", "avro, 8", "parquet, 8", "avro, 9", "parquet, 9")) + def testLogicalTypesReadRepair(logBlockFormat: String, tableVersion: Int): Unit = { + val logBlockString = if (logBlockFormat == "avro") { + "" + } else { + "_parquet_log" + } + val prevTimezone = spark.conf.get("spark.sql.session.timeZone") + val propertyValue: String = System.getProperty("spark.testing") + try { + if (HoodieSparkUtils.isSpark3_3) { + System.setProperty("spark.testing", "true") + } + spark.conf.set("spark.sql.session.timeZone", "UTC") + val tableName = "trips_logical_types_json_mor_read_v" + tableVersion + logBlockString + val dataPath = "file://" + basePath + "/" + tableName + val zipOutput = Paths.get(new URI(dataPath)) + HoodieTestUtils.extractZipToDirectory("/" + tableName + ".zip", zipOutput, getClass) + val tableBasePath = zipOutput.toString + + val df = spark.read.format("org.apache.hudi") + .option("hoodie.metadata.enable", "false") + .load(tableBasePath) + + val rows = df.collect() + assertEquals(20, rows.length) + for (row <- rows) { + val hash = row.get(6).asInstanceOf[String].hashCode() + if ((hash & 1)== 0) { + assertEquals("2020-01-01T00:00:00.001Z", row.get(15).asInstanceOf[Timestamp].toInstant.toString) + assertEquals("2020-06-01T12:00:00.000001Z", row.get(16).asInstanceOf[Timestamp].toInstant.toString) + assertEquals("2015-05-20T12:34:56.001", row.get(17).toString) + assertEquals("2017-07-07T07:07:07.000001", row.get(18).toString) + } else { + assertEquals("2019-12-31T23:59:59.999Z", row.get(15).asInstanceOf[Timestamp].toInstant.toString) + assertEquals("2020-06-01T11:59:59.999999Z", row.get(16).asInstanceOf[Timestamp].toInstant.toString) + assertEquals("2015-05-20T12:34:55.999", row.get(17).toString) + assertEquals("2017-07-07T07:07:06.999999", row.get(18).toString) + } + } + assertEquals(10, df.filter("ts_millis > timestamp('2020-01-01 00:00:00Z')").count()) + assertEquals(10, df.filter("ts_millis < timestamp('2020-01-01 00:00:00Z')").count()) + assertEquals(0, df.filter("ts_millis > timestamp('2020-01-01 00:00:00.001Z')").count()) + assertEquals(0, df.filter("ts_millis < timestamp('2019-12-31 23:59:59.999Z')").count()) + + assertEquals(10, df.filter("ts_micros > timestamp('2020-06-01 12:00:00Z')").count()) + assertEquals(10, df.filter("ts_micros < timestamp('2020-06-01 12:00:00Z')").count()) + assertEquals(0, df.filter("ts_micros > timestamp('2020-06-01 12:00:00.000001Z')").count()) + assertEquals(0, df.filter("ts_micros < timestamp('2020-06-01 11:59:59.999999Z')").count()) + + assertEquals(10, df.filter("local_ts_millis > CAST('2015-05-20 12:34:56' AS TIMESTAMP_NTZ)").count()) + assertEquals(10, df.filter("local_ts_millis < CAST('2015-05-20 12:34:56' AS TIMESTAMP_NTZ)").count()) + assertEquals(0, df.filter("local_ts_millis > CAST('2015-05-20 12:34:56.001' AS TIMESTAMP_NTZ)").count()) + assertEquals(0, df.filter("local_ts_millis < CAST('2015-05-20 12:34:55.999' AS TIMESTAMP_NTZ)").count()) + + assertEquals(10, df.filter("local_ts_micros > CAST('2017-07-07 07:07:07' AS TIMESTAMP_NTZ)").count()) + assertEquals(10, df.filter("local_ts_micros < CAST('2017-07-07 07:07:07' AS TIMESTAMP_NTZ)").count()) + assertEquals(0, df.filter("local_ts_micros > CAST('2017-07-07 07:07:07.000001' AS TIMESTAMP_NTZ)").count()) + assertEquals(0, df.filter("local_ts_micros < CAST('2017-07-07 07:07:06.999999' AS TIMESTAMP_NTZ)").count()) + } finally { + spark.conf.set("spark.sql.session.timeZone", prevTimezone) + if (HoodieSparkUtils.isSpark3_3) { + if (propertyValue == null) { + System.clearProperty("spark.testing") + } else { + System.setProperty("spark.testing", propertyValue) + } + } + } + } + /** * This tests the case that query by with a specified partition condition on hudi table which is * different between the value of the partition field and the actual partition path, diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala index 71bf1e542d1a3..474ce8afc6798 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala @@ -323,6 +323,7 @@ class Spark33LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val readSupport = new HoodieParquetReadSupport( convertTz, enableVectorizedReader = false, + enableTimestampFieldRepair = true, datetimeRebaseSpec, int96RebaseSpec) diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetReader.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetReader.scala index 50fc06ea3187b..332200c5c0f27 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetReader.scala @@ -31,11 +31,14 @@ import org.apache.parquet.filter2.compat.FilterCompat import org.apache.parquet.filter2.predicate.FilterApi import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS import org.apache.parquet.hadoop._ +import org.apache.parquet.hadoop.metadata.{FileMetaData, ParquetMetadata} +import org.apache.parquet.schema.SchemaRepair import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.JoinedRow import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.parquet.Spark33ParquetReader.repairFooterSchema import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -57,6 +60,7 @@ class Spark33ParquetReader(enableVectorizedReader: Boolean, capacity: Int, returningBatch: Boolean, enableRecordFilter: Boolean, + enableLogicalTimestampRepair: Boolean, timeZoneId: Option[String]) extends SparkParquetReaderBase( enableVectorizedReader = enableVectorizedReader, enableParquetFilterPushDown = enableParquetFilterPushDown, @@ -70,6 +74,7 @@ class Spark33ParquetReader(enableVectorizedReader: Boolean, capacity = capacity, returningBatch = returningBatch, enableRecordFilter = enableRecordFilter, + enableLogicalTimestampRepair = enableLogicalTimestampRepair, timeZoneId = timeZoneId) { /** @@ -89,7 +94,8 @@ class Spark33ParquetReader(enableVectorizedReader: Boolean, partitionSchema: StructType, internalSchemaOpt: org.apache.hudi.common.util.Option[InternalSchema], filters: Seq[Filter], - sharedConf: Configuration): Iterator[InternalRow] = { + sharedConf: Configuration, + tableSchemaOpt: org.apache.hudi.common.util.Option[org.apache.parquet.schema.MessageType]): Iterator[InternalRow] = { assert(file.partitionValues.numFields == partitionSchema.size) val filePath = new Path(new URI(file.filePath)) @@ -98,8 +104,14 @@ class Spark33ParquetReader(enableVectorizedReader: Boolean, val schemaEvolutionUtils = new ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema, internalSchemaOpt) - lazy val footerFileMetaData = - ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData + lazy val originalFooter = ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS) + lazy val fileFooter = if (enableLogicalTimestampRepair) { + repairFooterSchema(originalFooter, tableSchemaOpt) + } else { + originalFooter + } + + lazy val footerFileMetaData = fileFooter.getFileMetaData val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) @@ -192,8 +204,10 @@ class Spark33ParquetReader(enableVectorizedReader: Boolean, val readSupport = new HoodieParquetReadSupport( convertTz, enableVectorizedReader = false, + enableLogicalTimestampRepair, datetimeRebaseSpec, - int96RebaseSpec) + int96RebaseSpec, + tableSchemaOpt) val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) new ParquetRecordReader[InternalRow](readSupport, parquetFilter) @@ -252,9 +266,10 @@ object Spark33ParquetReader extends SparkParquetReaderBuilder { sqlConf.getConfString("spark.sql.legacy.parquet.nanosAsLong", "false").toBoolean ) + val enableLogicalTimestampRepair = hadoopConf.getBoolean("logicalTimestampField.repair.enable", true) // Should always be set by FileSourceScanExec while creating this. // Check conf before checking the option, to allow working around an issue by changing conf. - val returningBatch = sqlConf.parquetVectorizedReaderEnabled && + val returningBatch = vectorized && sqlConf.parquetVectorizedReaderEnabled && options.get(FileFormat.OPTION_RETURNING_BATCH) .getOrElse { throw new IllegalArgumentException( @@ -279,6 +294,23 @@ object Spark33ParquetReader extends SparkParquetReaderBuilder { capacity = sqlConf.parquetVectorizedReaderBatchSize, returningBatch = returningBatch, enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + enableLogicalTimestampRepair = enableLogicalTimestampRepair, timeZoneId = Some(sqlConf.sessionLocalTimeZone)) } + + // Helper to repair the schema if needed + def repairFooterSchema(original: ParquetMetadata, + tableSchemaOpt: org.apache.hudi.common.util.Option[org.apache.parquet.schema.MessageType]): ParquetMetadata = { + val repairedSchema = SchemaRepair.repairLogicalTypes(original.getFileMetaData.getSchema, tableSchemaOpt) + val oldMeta = original.getFileMetaData + new ParquetMetadata( + new FileMetaData( + repairedSchema, + oldMeta.getKeyValueMetaData, + oldMeta.getCreatedBy, + oldMeta.getFileDecryptor + ), + original.getBlocks + ) + } } diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala index 58f195f7253aa..419c1fc827aa9 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala @@ -334,6 +334,7 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val readSupport = new HoodieParquetReadSupport( convertTz, enableVectorizedReader = false, + enableTimestampFieldRepair = true, datetimeRebaseSpec, int96RebaseSpec) diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetReader.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetReader.scala index 9692b343dd667..aa43401a605d7 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetReader.scala @@ -30,11 +30,14 @@ import org.apache.parquet.filter2.compat.FilterCompat import org.apache.parquet.filter2.predicate.FilterApi import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS import org.apache.parquet.hadoop._ +import org.apache.parquet.hadoop.metadata.{FileMetaData, ParquetMetadata} +import org.apache.parquet.schema.SchemaRepair import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.JoinedRow import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.parquet.Spark34ParquetReader.repairFooterSchema import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -54,6 +57,7 @@ class Spark34ParquetReader(enableVectorizedReader: Boolean, capacity: Int, returningBatch: Boolean, enableRecordFilter: Boolean, + enableLogicalTimestampRepair: Boolean, timeZoneId: Option[String]) extends SparkParquetReaderBase( enableVectorizedReader = enableVectorizedReader, enableParquetFilterPushDown = enableParquetFilterPushDown, @@ -67,6 +71,7 @@ class Spark34ParquetReader(enableVectorizedReader: Boolean, capacity = capacity, returningBatch = returningBatch, enableRecordFilter = enableRecordFilter, + enableLogicalTimestampRepair = enableLogicalTimestampRepair, timeZoneId = timeZoneId) { /** @@ -86,7 +91,8 @@ class Spark34ParquetReader(enableVectorizedReader: Boolean, partitionSchema: StructType, internalSchemaOpt: org.apache.hudi.common.util.Option[InternalSchema], filters: Seq[Filter], - sharedConf: Configuration): Iterator[InternalRow] = { + sharedConf: Configuration, + tableSchemaOpt: org.apache.hudi.common.util.Option[org.apache.parquet.schema.MessageType]): Iterator[InternalRow] = { assert(file.partitionValues.numFields == partitionSchema.size) val filePath = file.toPath @@ -95,8 +101,14 @@ class Spark34ParquetReader(enableVectorizedReader: Boolean, val schemaEvolutionUtils = new ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema, internalSchemaOpt) - lazy val footerFileMetaData = - ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData + lazy val originalFooter = ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS) + lazy val fileFooter = if (enableLogicalTimestampRepair) { + repairFooterSchema(originalFooter, tableSchemaOpt) + } else { + originalFooter + } + + lazy val footerFileMetaData = fileFooter.getFileMetaData val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) @@ -189,8 +201,10 @@ class Spark34ParquetReader(enableVectorizedReader: Boolean, val readSupport = new HoodieParquetReadSupport( convertTz, enableVectorizedReader = false, + enableLogicalTimestampRepair, datetimeRebaseSpec, - int96RebaseSpec) + int96RebaseSpec, + tableSchemaOpt) val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) new ParquetRecordReader[InternalRow](readSupport, parquetFilter) @@ -254,6 +268,7 @@ object Spark34ParquetReader extends SparkParquetReaderBuilder { ) hadoopConf.setBoolean(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, sqlConf.parquetInferTimestampNTZEnabled) + val enableLogicalTimestampRepair = hadoopConf.getBoolean("logicalTimestampField.repair.enable", true) val returningBatch = sqlConf.parquetVectorizedReaderEnabled && options.getOrElse(FileFormat.OPTION_RETURNING_BATCH, throw new IllegalArgumentException( @@ -278,6 +293,23 @@ object Spark34ParquetReader extends SparkParquetReaderBuilder { capacity = sqlConf.parquetVectorizedReaderBatchSize, returningBatch = returningBatch, enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + enableLogicalTimestampRepair = enableLogicalTimestampRepair, timeZoneId = Some(sqlConf.sessionLocalTimeZone)) } + + // Helper to repair the schema if needed + def repairFooterSchema(original: ParquetMetadata, + tableSchemaOpt: org.apache.hudi.common.util.Option[org.apache.parquet.schema.MessageType]): ParquetMetadata = { + val repairedSchema = SchemaRepair.repairLogicalTypes(original.getFileMetaData.getSchema, tableSchemaOpt) + val oldMeta = original.getFileMetaData + new ParquetMetadata( + new FileMetaData( + repairedSchema, + oldMeta.getKeyValueMetaData, + oldMeta.getCreatedBy, + oldMeta.getFileDecryptor + ), + original.getBlocks + ) + } } diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35LegacyHoodieParquetFileFormat.scala index e8fc3b3f6479d..d547af7f8ed83 100644 --- a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35LegacyHoodieParquetFileFormat.scala @@ -335,6 +335,7 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val readSupport = new HoodieParquetReadSupport( convertTz, enableVectorizedReader = false, + enableTimestampFieldRepair = true, datetimeRebaseSpec, int96RebaseSpec) diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetReader.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetReader.scala index 3d192dd3a170c..6b32cda273743 100644 --- a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetReader.scala @@ -28,12 +28,15 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.parquet.filter2.compat.FilterCompat import org.apache.parquet.filter2.predicate.FilterApi import org.apache.parquet.hadoop.{ParquetInputFormat, ParquetRecordReader} +import org.apache.parquet.hadoop.metadata.{FileMetaData, ParquetMetadata} +import org.apache.parquet.schema.SchemaRepair import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.JoinedRow import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.{DataSourceUtils, FileFormat, PartitionedFile, RecordReaderIterator, SparkColumnarFileReader} +import org.apache.spark.sql.execution.datasources.parquet.Spark35ParquetReader.repairFooterSchema import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType @@ -53,6 +56,7 @@ class Spark35ParquetReader(enableVectorizedReader: Boolean, capacity: Int, returningBatch: Boolean, enableRecordFilter: Boolean, + enableLogicalTimestampRepair: Boolean, timeZoneId: Option[String]) extends SparkParquetReaderBase( enableVectorizedReader = enableVectorizedReader, enableParquetFilterPushDown = enableParquetFilterPushDown, @@ -66,6 +70,7 @@ class Spark35ParquetReader(enableVectorizedReader: Boolean, capacity = capacity, returningBatch = returningBatch, enableRecordFilter = enableRecordFilter, + enableLogicalTimestampRepair = enableLogicalTimestampRepair, timeZoneId = timeZoneId) { /** @@ -85,7 +90,8 @@ class Spark35ParquetReader(enableVectorizedReader: Boolean, partitionSchema: StructType, internalSchemaOpt: org.apache.hudi.common.util.Option[InternalSchema], filters: scala.Seq[Filter], - sharedConf: Configuration): Iterator[InternalRow] = { + sharedConf: Configuration, + tableSchemaOpt: org.apache.hudi.common.util.Option[org.apache.parquet.schema.MessageType]): Iterator[InternalRow] = { assert(file.partitionValues.numFields == partitionSchema.size) val filePath = file.toPath @@ -94,7 +100,7 @@ class Spark35ParquetReader(enableVectorizedReader: Boolean, val schemaEvolutionUtils = new ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema, internalSchemaOpt) - val fileFooter = if (enableVectorizedReader) { + val originalFooter = if (enableVectorizedReader) { // When there are vectorized reads, we can avoid reading the footer twice by reading // all row groups in advance and filter row groups according to filters that require // push down (no need to read the footer metadata again). @@ -103,6 +109,12 @@ class Spark35ParquetReader(enableVectorizedReader: Boolean, ParquetFooterReader.readFooter(sharedConf, file, ParquetFooterReader.SKIP_ROW_GROUPS) } + val fileFooter = if (enableLogicalTimestampRepair) { + repairFooterSchema(originalFooter, tableSchemaOpt); + } else { + originalFooter + } + val footerFileMetaData = fileFooter.getFileMetaData val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( footerFileMetaData.getKeyValueMetaData.get, @@ -196,8 +208,10 @@ class Spark35ParquetReader(enableVectorizedReader: Boolean, val readSupport = new HoodieParquetReadSupport( convertTz, enableVectorizedReader = false, + enableLogicalTimestampRepair, datetimeRebaseSpec, - int96RebaseSpec) + int96RebaseSpec, + tableSchemaOpt) val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) new ParquetRecordReader[InternalRow](readSupport, parquetFilter) @@ -261,6 +275,7 @@ object Spark35ParquetReader extends SparkParquetReaderBuilder { ) hadoopConf.setBoolean(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, sqlConf.parquetInferTimestampNTZEnabled) + val enableLogicalTimestampRepair = hadoopConf.getBoolean("logicalTimestampField.repair.enable", true) val returningBatch = sqlConf.parquetVectorizedReaderEnabled && options.getOrElse(FileFormat.OPTION_RETURNING_BATCH, throw new IllegalArgumentException( @@ -285,6 +300,25 @@ object Spark35ParquetReader extends SparkParquetReaderBuilder { capacity = sqlConf.parquetVectorizedReaderBatchSize, returningBatch = returningBatch, enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + enableLogicalTimestampRepair = enableLogicalTimestampRepair, timeZoneId = Some(sqlConf.sessionLocalTimeZone)) } + + + // Helper to repair the schema if needed + def repairFooterSchema(original: ParquetMetadata, + tableSchemaOpt: org.apache.hudi.common.util.Option[org.apache.parquet.schema.MessageType]): ParquetMetadata = { + val repairedSchema = SchemaRepair.repairLogicalTypes(original.getFileMetaData.getSchema, tableSchemaOpt) + val oldMeta = original.getFileMetaData + new ParquetMetadata( + new FileMetaData( + repairedSchema, + oldMeta.getKeyValueMetaData, + oldMeta.getCreatedBy, + oldMeta.getEncryptionType, + oldMeta.getFileDecryptor + ), + original.getBlocks + ) + } } diff --git a/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark40LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark40LegacyHoodieParquetFileFormat.scala index 75190bc135c3c..9f408dce46bea 100644 --- a/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark40LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark40LegacyHoodieParquetFileFormat.scala @@ -332,6 +332,7 @@ class Spark40LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val readSupport = new HoodieParquetReadSupport( convertTz, enableVectorizedReader = false, + enableTimestampFieldRepair = true, datetimeRebaseSpec, int96RebaseSpec) diff --git a/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark40ParquetReader.scala b/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark40ParquetReader.scala index 5e8cea9f11a2a..e5b717330b371 100644 --- a/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark40ParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark40ParquetReader.scala @@ -28,12 +28,15 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.parquet.filter2.compat.FilterCompat import org.apache.parquet.filter2.predicate.FilterApi import org.apache.parquet.hadoop.{ParquetInputFormat, ParquetRecordReader} +import org.apache.parquet.hadoop.metadata.{FileMetaData, ParquetMetadata} +import org.apache.parquet.schema.SchemaRepair import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.JoinedRow import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.{DataSourceUtils, FileFormat, PartitionedFile, RecordReaderIterator, SparkColumnarFileReader} +import org.apache.spark.sql.execution.datasources.parquet.Spark40ParquetReader.repairFooterSchema import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType @@ -53,6 +56,7 @@ class Spark40ParquetReader(enableVectorizedReader: Boolean, capacity: Int, returningBatch: Boolean, enableRecordFilter: Boolean, + enableLogicalTimestampRepair: Boolean, timeZoneId: Option[String]) extends SparkParquetReaderBase( enableVectorizedReader = enableVectorizedReader, enableParquetFilterPushDown = enableParquetFilterPushDown, @@ -66,6 +70,7 @@ class Spark40ParquetReader(enableVectorizedReader: Boolean, capacity = capacity, returningBatch = returningBatch, enableRecordFilter = enableRecordFilter, + enableLogicalTimestampRepair = enableLogicalTimestampRepair, timeZoneId = timeZoneId) { /** @@ -85,7 +90,8 @@ class Spark40ParquetReader(enableVectorizedReader: Boolean, partitionSchema: StructType, internalSchemaOpt: org.apache.hudi.common.util.Option[InternalSchema], filters: scala.Seq[Filter], - sharedConf: Configuration): Iterator[InternalRow] = { + sharedConf: Configuration, + tableSchemaOpt: org.apache.hudi.common.util.Option[org.apache.parquet.schema.MessageType]): Iterator[InternalRow] = { assert(file.partitionValues.numFields == partitionSchema.size) val filePath = file.toPath @@ -94,7 +100,7 @@ class Spark40ParquetReader(enableVectorizedReader: Boolean, val schemaEvolutionUtils = new ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema, internalSchemaOpt) - val fileFooter = if (enableVectorizedReader) { + val originalFooter = if (enableVectorizedReader) { // When there are vectorized reads, we can avoid reading the footer twice by reading // all row groups in advance and filter row groups according to filters that require // push down (no need to read the footer metadata again). @@ -103,6 +109,12 @@ class Spark40ParquetReader(enableVectorizedReader: Boolean, ParquetFooterReader.readFooter(sharedConf, file, ParquetFooterReader.SKIP_ROW_GROUPS) } + val fileFooter = if (enableLogicalTimestampRepair) { + repairFooterSchema(originalFooter, tableSchemaOpt) + } else { + originalFooter + } + val footerFileMetaData = fileFooter.getFileMetaData val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( footerFileMetaData.getKeyValueMetaData.get, @@ -196,8 +208,10 @@ class Spark40ParquetReader(enableVectorizedReader: Boolean, val readSupport = new HoodieParquetReadSupport( convertTz, enableVectorizedReader = false, + enableLogicalTimestampRepair, datetimeRebaseSpec, - int96RebaseSpec) + int96RebaseSpec, + tableSchemaOpt) val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) new ParquetRecordReader[InternalRow](readSupport, parquetFilter) @@ -261,6 +275,7 @@ object Spark40ParquetReader extends SparkParquetReaderBuilder { ) hadoopConf.setBoolean(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, sqlConf.parquetInferTimestampNTZEnabled) + val enableLogicalTimestampRepair = hadoopConf.getBoolean("logicalTimestampField.repair.enable", true) val returningBatch = sqlConf.parquetVectorizedReaderEnabled && options.getOrElse(FileFormat.OPTION_RETURNING_BATCH, throw new IllegalArgumentException( @@ -285,6 +300,24 @@ object Spark40ParquetReader extends SparkParquetReaderBuilder { capacity = sqlConf.parquetVectorizedReaderBatchSize, returningBatch = returningBatch, enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + enableLogicalTimestampRepair = enableLogicalTimestampRepair, timeZoneId = Some(sqlConf.sessionLocalTimeZone)) } + + // Helper to repair the schema if needed + def repairFooterSchema(original: ParquetMetadata, + tableSchemaOpt: org.apache.hudi.common.util.Option[org.apache.parquet.schema.MessageType]): ParquetMetadata = { + val repairedSchema = SchemaRepair.repairLogicalTypes(original.getFileMetaData.getSchema, tableSchemaOpt) + val oldMeta = original.getFileMetaData + new ParquetMetadata( + new FileMetaData( + repairedSchema, + oldMeta.getKeyValueMetaData, + oldMeta.getCreatedBy, + oldMeta.getEncryptionType, + oldMeta.getFileDecryptor + ), + original.getBlocks + ) + } } 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 2dc9d4da67acb..b0edc6f5fcc3c 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 @@ -35,6 +35,7 @@ import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.DefaultHoodieRecordPayload; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; @@ -1008,6 +1009,344 @@ private void assertBoundaryCounts(Dataset df, String exprZero, String exprT assertEquals(totalCount, df.filter(exprTotal).count(), exprTotal); } + @ParameterizedTest + @CsvSource(value = {"SIX,AVRO,CLUSTER", "EIGHT,AVRO,CLUSTER", "CURRENT,AVRO,NONE", "CURRENT,AVRO,CLUSTER", "CURRENT,SPARK,NONE", "CURRENT,SPARK,CLUSTER"}) + public void testCOWLogicalRepair(String tableVersion, String recordType, String operation) throws Exception { + TestMercifulJsonToRowConverterBase.timestampNTZCompatibility(() -> { + String dirName = "trips_logical_types_json_cow_write"; + String dataPath = basePath + "/" + dirName; + java.nio.file.Path zipOutput = Paths.get(new URI(dataPath)); + HoodieTestUtils.extractZipToDirectory("logical-repair/" + dirName + ".zip", zipOutput, getClass()); + String tableBasePath = zipOutput.toString(); + + TypedProperties properties = new TypedProperties(); + String schemaPath = getClass().getClassLoader().getResource("logical-repair/schema.avsc").toURI().toString(); + properties.setProperty("hoodie.streamer.schemaprovider.source.schema.file", schemaPath); + properties.setProperty("hoodie.streamer.schemaprovider.target.schema.file", schemaPath); + String inputDataPath = getClass().getClassLoader().getResource("logical-repair/cow_write_updates/2").toURI().toString(); + properties.setProperty("hoodie.streamer.source.dfs.root", inputDataPath); + + String mergerClass = getMergerClassForRecordType(recordType); + String tableVersionString = getTableVersionCode(tableVersion); + + properties.setProperty(HoodieWriteConfig.RECORD_MERGE_IMPL_CLASSES.key(), mergerClass); + properties.setProperty("hoodie.datasource.write.recordkey.field", "_row_key"); + properties.setProperty("hoodie.datasource.write.precombine.field", "timestamp"); + properties.setProperty("hoodie.datasource.write.partitionpath.field", "partition_path"); + properties.setProperty("hoodie.datasource.write.keygenerator.class", "org.apache.hudi.keygen.SimpleKeyGenerator"); + properties.setProperty("hoodie.cleaner.policy", "KEEP_LATEST_COMMITS"); + properties.setProperty("hoodie.compact.inline", "false"); + properties.setProperty("hoodie.metadtata.enable", "false"); + properties.setProperty("hoodie.parquet.small.file.limit", "-1"); + properties.setProperty("hoodie.cleaner.commits.retained", "10"); + properties.setProperty(HoodieWriteConfig.WRITE_TABLE_VERSION.key(), tableVersionString); + + Option propt = Option.of(properties); + + new HoodieStreamer(prepCfgForCowLogicalRepair(tableBasePath, "456"), jsc, propt).sync(); + + + inputDataPath = getClass().getClassLoader().getResource("logical-repair/cow_write_updates/3").toURI().toString(); + propt.get().setProperty("hoodie.streamer.source.dfs.root", inputDataPath); + if ("CLUSTER".equals(operation)) { + propt.get().setProperty("hoodie.clustering.inline", "true"); + propt.get().setProperty("hoodie.clustering.inline.max.commits", "1"); + propt.get().setProperty("hoodie.clustering.plan.strategy.single.group.clustering.enabled", "true"); + propt.get().setProperty("hoodie.clustering.plan.strategy.sort.columns", "ts_millis,_row_key"); + } + new HoodieStreamer(prepCfgForCowLogicalRepair(tableBasePath, "789"), jsc, propt).sync(); + + String prevTimezone = sparkSession.conf().get("spark.sql.session.timeZone"); + try { + sparkSession.conf().set("spark.sql.session.timeZone", "UTC"); + Dataset df = sparkSession.read() + .format("org.apache.hudi") + .option("hoodie.metadata.enable", "false") + .load(tableBasePath); + + assertDataframe(df, 15, 15); + + if ("CLUSTER".equals(operation)) { + // after we cluster, the raw parquet should be correct + + // Validate raw parquet files + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() + .setConf(storage.getConf()) + .setBasePath(tableBasePath) + .build(); + + HoodieTimeline completedCommitsTimeline = metaClient.getCommitsTimeline().filterCompletedInstants(); + Option latestInstant = completedCommitsTimeline.lastInstant(); + assertTrue(latestInstant.isPresent(), "No completed commits found"); + + List baseFilePaths = collectLatestBaseFilePaths(metaClient); + + assertEquals(4, baseFilePaths.size()); + + // Read raw parquet files + Dataset rawParquetDf = sparkSession.read().parquet(baseFilePaths.toArray(new String[0])); + assertDataframe(rawParquetDf, 15, 15); + } + } finally { + sparkSession.conf().set("spark.sql.session.timeZone", prevTimezone); + } + }); + } + + @ParameterizedTest + @CsvSource(value = {"SIX,AVRO,CLUSTER,AVRO", "EIGHT,AVRO,CLUSTER,AVRO", + "CURRENT,AVRO,NONE,AVRO", "CURRENT,AVRO,CLUSTER,AVRO", "CURRENT,AVRO,COMPACT,AVRO", + "CURRENT,AVRO,NONE,PARQUET", "CURRENT,AVRO,CLUSTER,PARQUET", "CURRENT,AVRO,COMPACT,PARQUET", + "CURRENT,SPARK,NONE,PARQUET", "CURRENT,SPARK,CLUSTER,PARQUET", "CURRENT,SPARK,COMPACT,PARQUET"}) + public void testMORLogicalRepair(String tableVersion, String recordType, String operation, String logBlockType) throws Exception { + TestMercifulJsonToRowConverterBase.timestampNTZCompatibility(() -> { + String tableSuffix; + String logFormatValue; + if ("AVRO".equals(logBlockType)) { + logFormatValue = "avro"; + tableSuffix = "avro_log"; + } else { + logFormatValue = "parquet"; + tableSuffix = "parquet_log"; + } + + String dirName = "trips_logical_types_json_mor_write_" + tableSuffix; + String dataPath = basePath + "/" + dirName; + java.nio.file.Path zipOutput = Paths.get(new URI(dataPath)); + HoodieTestUtils.extractZipToDirectory("logical-repair/" + dirName + ".zip", zipOutput, getClass()); + String tableBasePath = zipOutput.toString(); + + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() + .setConf(storage.getConf()) + .setBasePath(tableBasePath) + .build(); + + // validate no compaction and clustering instants present in the timeline + HoodieTimeline completedTimeline = metaClient.getActiveTimeline().filterCompletedInstants(); + assertFalse(completedTimeline.getInstants().stream().anyMatch(i -> i.getAction().equals(HoodieTimeline.COMPACTION_ACTION))); + assertFalse(completedTimeline.getInstants().stream().anyMatch(i -> i.getAction().equals(HoodieTimeline.CLUSTERING_ACTION))); + + TypedProperties properties = new TypedProperties(); + String schemaPath = getClass().getClassLoader().getResource("logical-repair/schema.avsc").toURI().toString(); + properties.setProperty("hoodie.streamer.schemaprovider.source.schema.file", schemaPath); + properties.setProperty("hoodie.streamer.schemaprovider.target.schema.file", schemaPath); + String inputDataPath = getClass().getClassLoader().getResource("logical-repair/mor_write_updates/5").toURI().toString(); + properties.setProperty("hoodie.streamer.source.dfs.root", inputDataPath); + String mergerClass = getMergerClassForRecordType(recordType); + String tableVersionString = getTableVersionCode(tableVersion); + + properties.setProperty(HoodieWriteConfig.RECORD_MERGE_IMPL_CLASSES.key(), mergerClass); + properties.setProperty("hoodie.datasource.write.recordkey.field", "_row_key"); + properties.setProperty("hoodie.datasource.write.precombine.field", "timestamp"); + properties.setProperty("hoodie.datasource.write.partitionpath.field", "partition_path"); + properties.setProperty("hoodie.datasource.write.keygenerator.class", "org.apache.hudi.keygen.SimpleKeyGenerator"); + properties.setProperty("hoodie.cleaner.policy", "KEEP_LATEST_COMMITS"); + properties.setProperty("hoodie.metadtata.enable", "false"); + properties.setProperty("hoodie.parquet.small.file.limit", "-1"); + properties.setProperty("hoodie.cleaner.commits.retained", "10"); + properties.setProperty(HoodieWriteConfig.WRITE_TABLE_VERSION.key(), tableVersionString); + properties.setProperty(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key(), logFormatValue); + + boolean disableCompaction; + if ("COMPACT".equals(operation)) { + properties.setProperty("hoodie.compact.inline", "true"); + properties.setProperty("hoodie.compact.inline.max.delta.commits", "1"); + disableCompaction = false; + // validate that there are no completed compaction (commit) instants in timeline. + } else { + properties.setProperty("hoodie.compact.inline", "false"); + disableCompaction = true; + } + + if ("CLUSTER".equals(operation)) { + properties.setProperty("hoodie.clustering.inline", "true"); + properties.setProperty("hoodie.clustering.inline.max.commits", "1"); + properties.setProperty("hoodie.clustering.plan.strategy.single.group.clustering.enabled", "true"); + properties.setProperty("hoodie.clustering.plan.strategy.sort.columns", "ts_millis,_row_key"); + } + + Option propt = Option.of(properties); + + new HoodieStreamer(prepCfgForMorLogicalRepair(tableBasePath, dirName, "123", disableCompaction), jsc, propt).sync(); + + String prevTimezone = sparkSession.conf().get("spark.sql.session.timeZone"); + try { + if (!HoodieSparkUtils.gteqSpark3_5()) { + sparkSession.conf().set("spark.sql.parquet.enableVectorizedReader", "false"); + } + sparkSession.conf().set("spark.sql.session.timeZone", "UTC"); + Dataset df = sparkSession.read() + .format("org.apache.hudi") + .option("hoodie.metadata.enable", "false") + .load(tableBasePath); + + assertDataframe(df, 12, 14); + + metaClient = HoodieTableMetaClient.builder() + .setConf(storage.getConf()) + .setBasePath(tableBasePath) + .build(); + + if ("CLUSTER".equals(operation)) { + // after we cluster, the raw parquet should be correct + + // Validate raw parquet files + HoodieTimeline completedCommitsTimeline = metaClient.getCommitsTimeline().filterCompletedInstants(); + Option latestInstant = completedCommitsTimeline.lastInstant(); + assertTrue(latestInstant.isPresent(), "No completed commits found"); + + List baseFilePaths = collectLatestBaseFilePaths(metaClient); + + assertEquals(3, baseFilePaths.size()); + + // Read raw parquet files + Dataset rawParquetDf = sparkSession.read().parquet(baseFilePaths.toArray(new String[0])); + assertDataframe(rawParquetDf, 12, 14); + } else if ("COMPACT".equals(operation)) { + // after compaction some files should be ok + + // Validate raw parquet files + HoodieTimeline completedCommitsTimeline = metaClient.getCommitsTimeline().filterCompletedInstants(); + Option latestInstant = completedCommitsTimeline.lastInstant(); + assertTrue(latestInstant.isPresent(), "No completed commits found"); + + List baseFilePaths = collectLatestBaseFilePaths(metaClient); + + assertEquals(7, baseFilePaths.size()); + + // Read raw parquet files + Dataset rawParquetDf = sparkSession.read().parquet(baseFilePaths.stream() + // only read the compacted ones, the others are still incorrect + .filter(path -> path.contains(latestInstant.get().requestedTime())) + .toArray(String[]::new)); + assertDataframe(rawParquetDf, 2, 3); + } + } finally { + sparkSession.conf().set("spark.sql.session.timeZone", prevTimezone); + if (!HoodieSparkUtils.gteqSpark3_5()) { + sparkSession.conf().set("spark.sql.parquet.enableVectorizedReader", "true"); + } + } + }); + } + + public static void assertDataframe(Dataset df, int above, int below) { + List rows = df.collectAsList(); + assertEquals(above + below, rows.size()); + + for (Row row : rows) { + String val = row.getString(6); + int hash = val.hashCode(); + + if ((hash & 1) == 0) { + assertEquals("2020-01-01T00:00:00.001Z", row.getTimestamp(15).toInstant().toString()); + assertEquals("2020-06-01T12:00:00.000001Z", row.getTimestamp(16).toInstant().toString()); + assertEquals("2015-05-20T12:34:56.001", row.get(17).toString()); + assertEquals("2017-07-07T07:07:07.000001", row.get(18).toString()); + } else { + assertEquals("2019-12-31T23:59:59.999Z", row.getTimestamp(15).toInstant().toString()); + assertEquals("2020-06-01T11:59:59.999999Z", row.getTimestamp(16).toInstant().toString()); + assertEquals("2015-05-20T12:34:55.999", row.get(17).toString()); + assertEquals("2017-07-07T07:07:06.999999", row.get(18).toString()); + } + } + + assertEquals(above, df.filter("ts_millis > timestamp('2020-01-01 00:00:00Z')").count()); + assertEquals(below, df.filter("ts_millis < timestamp('2020-01-01 00:00:00Z')").count()); + assertEquals(0, df.filter("ts_millis > timestamp('2020-01-01 00:00:00.001Z')").count()); + assertEquals(0, df.filter("ts_millis < timestamp('2019-12-31 23:59:59.999Z')").count()); + + assertEquals(above, df.filter("ts_micros > timestamp('2020-06-01 12:00:00Z')").count()); + assertEquals(below, df.filter("ts_micros < timestamp('2020-06-01 12:00:00Z')").count()); + assertEquals(0, df.filter("ts_micros > timestamp('2020-06-01 12:00:00.000001Z')").count()); + assertEquals(0, df.filter("ts_micros < timestamp('2020-06-01 11:59:59.999999Z')").count()); + + assertEquals(above, df.filter("local_ts_millis > CAST('2015-05-20 12:34:56' AS TIMESTAMP_NTZ)").count()); + assertEquals(below, df.filter("local_ts_millis < CAST('2015-05-20 12:34:56' AS TIMESTAMP_NTZ)").count()); + assertEquals(0, df.filter("local_ts_millis > CAST('2015-05-20 12:34:56.001' AS TIMESTAMP_NTZ)").count()); + assertEquals(0, df.filter("local_ts_millis < CAST('2015-05-20 12:34:55.999' AS TIMESTAMP_NTZ)").count()); + + assertEquals(above, df.filter("local_ts_micros > CAST('2017-07-07 07:07:07' AS TIMESTAMP_NTZ)").count()); + assertEquals(below, df.filter("local_ts_micros < CAST('2017-07-07 07:07:07' AS TIMESTAMP_NTZ)").count()); + assertEquals(0, df.filter("local_ts_micros > CAST('2017-07-07 07:07:07.000001' AS TIMESTAMP_NTZ)").count()); + assertEquals(0, df.filter("local_ts_micros < CAST('2017-07-07 07:07:06.999999' AS TIMESTAMP_NTZ)").count()); + } + + private List collectLatestBaseFilePaths(HoodieTableMetaClient metaClient) { + List baseFilePaths = new ArrayList<>(); + try (HoodieTableFileSystemView fsView = FileSystemViewManager.createInMemoryFileSystemView( + new HoodieLocalEngineContext(metaClient.getStorageConf()), + metaClient, + HoodieMetadataConfig.newBuilder().enable(false).build())) { + + fsView.loadAllPartitions(); + fsView.getPartitionNames().forEach(partitionName -> + fsView.getLatestFileSlices(partitionName).forEach(fileSlice -> { + assertFalse(fileSlice.hasLogFiles(), "File slice should not have log files"); + Option latestBaseFile = fileSlice.getBaseFile(); + assertTrue(latestBaseFile.isPresent(), "Base file should be present"); + baseFilePaths.add(latestBaseFile.get().getPath()); + })); + } + return baseFilePaths; + } + + private String getMergerClassForRecordType(String recordType) { + switch (recordType) { + case "AVRO": + return HoodieAvroRecordMerger.class.getName(); + case "SPARK": + return DefaultSparkRecordMerger.class.getName(); + default: + throw new IllegalArgumentException("Invalid record type: " + recordType); + } + } + + private String getTableVersionCode(String tableVersion) { + switch (tableVersion) { + case "SIX": + return String.valueOf(HoodieTableVersion.SIX.versionCode()); + case "EIGHT": + return String.valueOf(HoodieTableVersion.EIGHT.versionCode()); + case "CURRENT": + return String.valueOf(HoodieTableVersion.current().versionCode()); + default: + throw new IllegalArgumentException("Invalid table version: " + tableVersion); + } + } + + private HoodieStreamer.Config prepCfgForCowLogicalRepair(String tableBasePath, + String ignoreCheckpoint) throws Exception { + HoodieStreamer.Config cfg = new HoodieStreamer.Config(); + cfg.targetBasePath = tableBasePath; + cfg.tableType = "COPY_ON_WRITE"; + cfg.targetTableName = "trips_logical_types_json_cow_write"; + cfg.sourceClassName = "org.apache.hudi.utilities.sources.JsonDFSSource"; + cfg.schemaProviderClassName = "org.apache.hudi.utilities.schema.FilebasedSchemaProvider"; + cfg.sourceOrderingFields = "timestamp"; + cfg.ignoreCheckpoint = ignoreCheckpoint; + cfg.operation = WriteOperationType.UPSERT; + cfg.forceDisableCompaction = true; + return cfg; + } + + private HoodieStreamer.Config prepCfgForMorLogicalRepair(String tableBasePath, + String tableName, + String ignoreCheckpoint, + boolean disableCompaction) throws Exception { + HoodieStreamer.Config cfg = new HoodieStreamer.Config(); + cfg.targetBasePath = tableBasePath; + cfg.tableType = "MERGE_ON_READ"; + cfg.targetTableName = tableName; + cfg.sourceClassName = "org.apache.hudi.utilities.sources.JsonDFSSource"; + cfg.schemaProviderClassName = "org.apache.hudi.utilities.schema.FilebasedSchemaProvider"; + cfg.sourceOrderingFields = "timestamp"; + cfg.ignoreCheckpoint = ignoreCheckpoint; + cfg.operation = WriteOperationType.UPSERT; + cfg.forceDisableCompaction = disableCompaction; + return cfg; + } + private static Stream continuousModeArgs() { return Stream.of( Arguments.of("AVRO", "CURRENT"), diff --git a/hudi-utilities/src/test/resources/logical-repair/README.md b/hudi-utilities/src/test/resources/logical-repair/README.md new file mode 100644 index 0000000000000..3aea5f2ae6906 --- /dev/null +++ b/hudi-utilities/src/test/resources/logical-repair/README.md @@ -0,0 +1,88 @@ + + + +Test assets + +trips_logical_types_json_cow_write.zip: + +this table was created with two deltastreamer writes: + +write 0 with 0.15.0: +inserts to partition 1, partition 2, partition 3 + +write 1 with 0.15.0: +inserts to partition 3 + +this gives us a table with 3 partitions, partition 1 and 2 have 1 file each and partition 3 has 2. + +Then we provide updates in cow_write_updates: + +write 2 done in the test: +inserts to partition 3, partition 4 + +write 3 done in the test: +updates to partition 3 + +this gives us a final table: + +partition 1: +1 base file written with 0.15.0 +partition 2: +1 base file written with 0.15.0 +1 base file written with 1.1 +partition 3: +1 base file written with 1.1 that contains some 0.15.0 written records +1 base file written with 0.15.0 +1 base file written with 1.1 +partition 4: +1 base file written with 1.1 + + +trips_logical_types_json_mor_write_avro_log.zip/trips_logical_types_json_mor_write_parquet_log.zip +the two tables were created with the same steps, but the avro table uses avro log files and the parquet table uses parquet files + +write 0 with 0.15.0: +inserts to partition 1, 2, 3 + +write 1 with 0.15.0: +inserts to partition 3 + +write 2 with 0.15.0: +updates to 1 file in partition 3 and 1 file in partition 2 + +write 3 with 0.15.0: +inserts to partition 3 + +write 4 with 0.15.0 +inserts to partition 3 and updates to 1 file in partition 3 + +write 5 done in the tests: +updates to 2 files in partition 3 and inserts to partition 3 + +The final table will be + +partition 1: +fg1: base file with 0.15.0 +partition 2: +fg1: base file with 0.15.0 + log file with 0.15.0 +partition 3: +fg1: base file with 0.15.0 + log file with 0.15.0 + log file with 1.1 +fg2: base file with 0.15.0 + log file with 1.1 +fg3: base file with 1.1 +fg4: base file with 0.15 + log file with 0.15 +fg5: base file with 0.15 diff --git a/hudi-utilities/src/test/resources/logical-repair/cow_write_updates/2/data.json b/hudi-utilities/src/test/resources/logical-repair/cow_write_updates/2/data.json new file mode 100644 index 0000000000000..dd6e2a05616be --- /dev/null +++ b/hudi-utilities/src/test/resources/logical-repair/cow_write_updates/2/data.json @@ -0,0 +1,6 @@ +{"timestamp": 1761335069636, "_row_key": "3f3ef947-c3e9-48a5-b08f-6cabbc6d6533", "partition_path": "2016/03/15", "trip_type": "BLACK", "rider": "rider-002", "driver": "driver-002", "begin_lat": 0.3017074196681322, "begin_lon": 0.5626109152945691, "end_lat": 0.5649785382157525, "end_lon": 0.6160079798524531, "ts_millis": 1577836799999, "ts_micros": 1591012799999999, "local_ts_millis": 1432125295999, "local_ts_micros": 1499411226999999, "event_date": 10956, "dec_fixed_small": [0, -44, 48], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 20], "_hoodie_is_deleted": false, "partition": "2015/03/16"} +{"timestamp": 1761335069636, "_row_key": "d8725d53-826a-45d8-9b70-b812d06d9dd0", "partition_path": "2016/03/15", "trip_type": "UBERX", "rider": "rider-002", "driver": "driver-002", "begin_lat": 0.27641854803317645, "begin_lon": 0.31700440770954075, "end_lat": 0.16654733508021524, "end_lon": 0.3555821110759497, "ts_millis": 1577836800001, "ts_micros": 1591012800000001, "local_ts_millis": 1432125296001, "local_ts_micros": 1499411227000001, "event_date": 10958, "dec_fixed_small": [0, -44, 50], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 22], "_hoodie_is_deleted": false, "partition": "2015/03/17"} +{"timestamp": 1761335069636, "_row_key": "1d4f0480-5300-4f68-8ebb-1ff70ff5c6ea", "partition_path": "2016/03/15", "trip_type": "UBERX", "rider": "rider-002", "driver": "driver-002", "begin_lat": 0.9074176919785227, "begin_lon": 0.4117236492462387, "end_lat": 0.6994811148788228, "end_lon": 0.3772709703853857, "ts_millis": 1577836800001, "ts_micros": 1591012800000001, "local_ts_millis": 1432125296001, "local_ts_micros": 1499411227000001, "event_date": 10958, "dec_fixed_small": [0, -44, 50], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 22], "_hoodie_is_deleted": false, "partition": "2015/03/16"} +{"timestamp": 1761335069636, "_row_key": "a6816ca4-60c8-4bab-a77a-31fa2c000987", "partition_path": "2015/03/18", "trip_type": "BLACK", "rider": "rider-002", "driver": "driver-002", "begin_lat": 0.4444402031840541, "begin_lon": 0.33692826304653933, "end_lat": 0.3043284603831268, "end_lon": 0.11042503421042937, "ts_millis": 1577836800001, "ts_micros": 1591012800000001, "local_ts_millis": 1432125296001, "local_ts_micros": 1499411227000001, "event_date": 10958, "dec_fixed_small": [0, -44, 50], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 22], "_hoodie_is_deleted": false, "partition": "2015/03/17"} +{"timestamp": 1761335069636, "_row_key": "b1555002-2fe1-4687-bb60-406a52f16bb5", "partition_path": "2015/03/18", "trip_type": "BLACK", "rider": "rider-002", "driver": "driver-002", "begin_lat": 0.43246488469100974, "begin_lon": 0.3648005645136184, "end_lat": 0.3781839595846225, "end_lon": 0.4638740649211893, "ts_millis": 1577836799999, "ts_micros": 1591012799999999, "local_ts_millis": 1432125295999, "local_ts_micros": 1499411226999999, "event_date": 10956, "dec_fixed_small": [0, -44, 48], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 20], "_hoodie_is_deleted": false, "partition": "2015/03/16"} +{"timestamp": 1761335069636, "_row_key": "615c45b5-57ad-489d-aad8-6fe563b513f7", "partition_path": "2015/03/18", "trip_type": "BLACK", "rider": "rider-002", "driver": "driver-002", "begin_lat": 0.75916203985879, "begin_lon": 0.49855855157343465, "end_lat": 0.7432577319020379, "end_lon": 0.33072999799294, "ts_millis": 1577836800001, "ts_micros": 1591012800000001, "local_ts_millis": 1432125296001, "local_ts_micros": 1499411227000001, "event_date": 10958, "dec_fixed_small": [0, -44, 50], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 22], "_hoodie_is_deleted": false, "partition": "2015/03/16"} diff --git a/hudi-utilities/src/test/resources/logical-repair/cow_write_updates/3/data.json b/hudi-utilities/src/test/resources/logical-repair/cow_write_updates/3/data.json new file mode 100644 index 0000000000000..105ce19370e16 --- /dev/null +++ b/hudi-utilities/src/test/resources/logical-repair/cow_write_updates/3/data.json @@ -0,0 +1,3 @@ +{"timestamp": 1761335069637, "_row_key": "fe818873-3af0-4fcd-90ea-f5ad0e7565fb", "partition_path": "2016/03/15", "trip_type": "BLACK", "rider": "rider-003", "driver": "driver-003", "begin_lat": 0.9122211080491403, "begin_lon": 0.23232697706220873, "end_lat": 0.8967870566670471, "end_lon": 0.05065495500664263, "ts_millis": 1577836800001, "ts_micros": 1591012800000001, "local_ts_millis": 1432125296001, "local_ts_micros": 1499411227000001, "event_date": 10958, "dec_fixed_small": [0, -44, 50], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 22], "_hoodie_is_deleted": false, "partition": "2016/03/15"} +{"timestamp": 1761335069637, "_row_key": "0af34a1a-c231-4c6b-8a5c-50d8e36a0ff1", "partition_path": "2016/03/15", "trip_type": "BLACK", "rider": "rider-003", "driver": "driver-003", "begin_lat": 0.26483577112225265, "begin_lon": 0.26862954952340434, "end_lat": 0.2727902211619275, "end_lon": 0.9138712331657564, "ts_millis": 1577836799999, "ts_micros": 1591012799999999, "local_ts_millis": 1432125295999, "local_ts_micros": 1499411226999999, "event_date": 10956, "dec_fixed_small": [0, -44, 48], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 20], "_hoodie_is_deleted": false, "partition": "2016/03/15"} +{"timestamp": 1761335069637, "_row_key": "f448d495-69cf-4d28-afa8-3af2459636ee", "partition_path": "2016/03/15", "trip_type": "UBERX", "rider": "rider-003", "driver": "driver-003", "begin_lat": 0.8873308149149347, "begin_lon": 0.358940823441969, "end_lat": 0.2646927323955117, "end_lon": 0.4091537968746116, "ts_millis": 1577836799999, "ts_micros": 1591012799999999, "local_ts_millis": 1432125295999, "local_ts_micros": 1499411226999999, "event_date": 10956, "dec_fixed_small": [0, -44, 48], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 20], "_hoodie_is_deleted": false, "partition": "2016/03/15"} diff --git a/hudi-utilities/src/test/resources/logical-repair/mor_write_updates/5/data.json b/hudi-utilities/src/test/resources/logical-repair/mor_write_updates/5/data.json new file mode 100644 index 0000000000000..48109422242fe --- /dev/null +++ b/hudi-utilities/src/test/resources/logical-repair/mor_write_updates/5/data.json @@ -0,0 +1,3 @@ +{"timestamp": 1761341703299, "_row_key": "092ed4ad-0e67-4df7-a051-c66ff30f08a7", "partition_path": "2015/03/17", "trip_type": "UBERX", "rider": "rider-005", "driver": "driver-005", "begin_lat": 0.20841305367042184, "begin_lon": 0.41269017191959156, "end_lat": 0.6266431410358951, "end_lon": 0.4514006891788409, "ts_millis": 1577836799999, "ts_micros": 1591012799999999, "local_ts_millis": 1432125295999, "local_ts_micros": 1499411226999999, "event_date": 10956, "dec_fixed_small": [0, -44, 48], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 20], "_hoodie_is_deleted": false, "partition": "2015/03/17"} +{"timestamp": 1761341703299, "_row_key": "5affef5c-f36f-4374-80f3-e7c6d3c38d25", "partition_path": "2015/03/17", "trip_type": "BLACK", "rider": "rider-005", "driver": "driver-005", "begin_lat": 0.3287214805934826, "begin_lon": 0.4292459075453131, "end_lat": 0.8027879467022967, "end_lon": 0.07863000273562926, "ts_millis": 1577836800001, "ts_micros": 1591012800000001, "local_ts_millis": 1432125296001, "local_ts_micros": 1499411227000001, "event_date": 10958, "dec_fixed_small": [0, -44, 50], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 22], "_hoodie_is_deleted": false, "partition": "2015/03/17"} +{"timestamp": 1761341703299, "_row_key": "5b22bcfd-e2a2-4cb9-b6c5-f643223c48c8", "partition_path": "2015/03/17", "trip_type": "UBERX", "rider": "rider-005", "driver": "driver-005", "begin_lat": 0.2218333443775823, "begin_lon": 0.441127835026775, "end_lat": 0.39946947397642374, "end_lon": 0.5064153585372088, "ts_millis": 1577836800001, "ts_micros": 1591012800000001, "local_ts_millis": 1432125296001, "local_ts_micros": 1499411227000001, "event_date": 10958, "dec_fixed_small": [0, -44, 50], "dec_fixed_large": [13, -76, -38, 95, 75, 113, 119, 22], "_hoodie_is_deleted": false, "partition": "2015/03/17"} diff --git a/hudi-utilities/src/test/resources/logical-repair/schema.avsc b/hudi-utilities/src/test/resources/logical-repair/schema.avsc new file mode 100644 index 0000000000000..6392fa92d7826 --- /dev/null +++ b/hudi-utilities/src/test/resources/logical-repair/schema.avsc @@ -0,0 +1,112 @@ +/* + * 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" : [ "null", "string" ], + "default" : null + }, { + "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" : "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 + } + }, { + "name" : "_hoodie_is_deleted", + "type" : "boolean", + "default" : false + } ] +} diff --git a/hudi-utilities/src/test/resources/logical-repair/trips_logical_types_json_cow_write.zip b/hudi-utilities/src/test/resources/logical-repair/trips_logical_types_json_cow_write.zip new file mode 100644 index 0000000000000..901120035f1e1 Binary files /dev/null and b/hudi-utilities/src/test/resources/logical-repair/trips_logical_types_json_cow_write.zip differ diff --git a/hudi-utilities/src/test/resources/logical-repair/trips_logical_types_json_mor_write_avro_log.zip b/hudi-utilities/src/test/resources/logical-repair/trips_logical_types_json_mor_write_avro_log.zip new file mode 100644 index 0000000000000..f6ef371ffea7e Binary files /dev/null and b/hudi-utilities/src/test/resources/logical-repair/trips_logical_types_json_mor_write_avro_log.zip differ diff --git a/hudi-utilities/src/test/resources/logical-repair/trips_logical_types_json_mor_write_parquet_log.zip b/hudi-utilities/src/test/resources/logical-repair/trips_logical_types_json_mor_write_parquet_log.zip new file mode 100644 index 0000000000000..c3d02f98eaa12 Binary files /dev/null and b/hudi-utilities/src/test/resources/logical-repair/trips_logical_types_json_mor_write_parquet_log.zip differ