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..44f264850640b 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 @@ -27,6 +27,8 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.minicluster.HdfsTestService; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.hadoop.avro.HoodieTimestampAwareParquetInputFormat; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StorageConfiguration; import org.apache.hudi.storage.hadoop.HoodieHadoopStorage; @@ -37,6 +39,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat; +import org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; import org.apache.hadoop.hive.serde2.SerDeException; import org.apache.hadoop.hive.serde2.avro.HiveTypeUtils; @@ -94,7 +97,18 @@ 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) -> { + if (HoodieColumnProjectionUtils.supportTimestamp(jobConf)) { + try { + return new ParquetRecordReaderWrapper(new HoodieTimestampAwareParquetInputFormat(Option.empty(), Option.of(dataSchema)), inputSplit, jobConf, null); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } else { + return 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..c6d4aa4b2ad6e 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 messageTypeOption = Option.empty(); private Option structTypeOption = Option.empty(); private Option schemaOption = Option.empty(); @@ -116,11 +120,12 @@ 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 requestNonNull = AvroSchemaUtils.resolveNullableSchema(requestedSchema); + StructType structSchema = HoodieInternalRowUtils.getCachedSchema(requestNonNull); + Option messageSchema = Option.of(getAvroSchemaConverter(storage.getConf().unwrapAs(Configuration.class)).convert(requestNonNull)); + MessageType dataMessageType = SchemaRepair.repairLogicalTypes(getMessageType(), messageSchema); + StructType dataStructType = convertToStruct(dataMessageType); + 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); @@ -128,7 +133,7 @@ public ClosableIterator getUnsafeRowIterator(StructType requestedSche 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, 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 getMessageType() { + if (messageTypeOption.isEmpty()) { + MessageType messageType = ((ParquetUtils) parquetUtils).readSchema(storage, path); + messageTypeOption = Option.of(messageType); + } + return messageTypeOption.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 = getMessageType(); + 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 = getMessageType(); + 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/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index 537189909dbe1..29c77ddaed695 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 @@ -75,19 +78,28 @@ class SparkFileFormatInternalRowReaderContext(baseFileReader: SparkColumnarFileR if (hasRowIndexField) { assert(getRecordContext.supportsParquetRowIndex()) } - val structType = HoodieInternalRowUtils.getCachedSchema(requiredSchema) if (FSUtils.isLogFile(filePath)) { 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..ca8017eb1d940 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 @@ -35,16 +35,16 @@ class HoodieParquetReadSupport( convertTz: Option[ZoneId], enableVectorizedReader: 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 + val requestedParquetSchema = SchemaRepair.repairLogicalTypes(readContext.getRequestedSchema, tableSchemaOpt) 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..6ad47c19012d8 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 @@ -43,14 +43,14 @@ class MultipleColumnarFileFormatReader(parquetReader: SparkColumnarFileReader, o * @param storageConf the hadoop conf * @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-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java index 2dd40387c1d0e..be4d818c6b7c5 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 @@ -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); } @@ -1485,6 +1487,10 @@ public static boolean recordNeedsRewriteForExtendedAvroTypePromotion(Schema writ case DOUBLE: case FLOAT: case LONG: + if (readerSchema.getLogicalType() instanceof LogicalTypes.TimestampMillis + && writerSchema.getLogicalType() instanceof LogicalTypes.TimestampMicros) { + return true; + } return !(writerSchema.getType().equals(Schema.Type.INT) || writerSchema.getType().equals(Schema.Type.LONG)); default: return !writerSchema.getType().equals(readerSchema.getType()); 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..f1d6bea5fc058 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; @@ -187,11 +188,12 @@ private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content) this.totalRecords = this.dis.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); } } @@ -272,11 +274,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/parquet/schema/AvroSchemaRepair.java b/hudi-common/src/main/java/org/apache/parquet/schema/AvroSchemaRepair.java new file mode 100644 index 0000000000000..476401b0ec46a --- /dev/null +++ b/hudi-common/src/main/java/org/apache/parquet/schema/AvroSchemaRepair.java @@ -0,0 +1,217 @@ +/* + * 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.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; + +import org.apache.hudi.avro.AvroSchemaCache; +import org.apache.hudi.avro.AvroSchemaUtils; + +import java.util.ArrayList; +import java.util.List; + +public class AvroSchemaRepair { + public static Schema repairLogicalTypes(Schema requestedSchema, Schema tableSchema) { + Schema repairedSchema = repairAvroSchema(requestedSchema, tableSchema); + if (repairedSchema != requestedSchema) { + return AvroSchemaCache.intern(repairedSchema); + } + return requestedSchema; + } + + /** + * Performs schema repair on a schema, handling nullable unions. + */ + private static Schema repairAvroSchema(Schema requested, Schema table) { + // Always resolve nullable schemas first (returns unchanged if not a union) + Schema requestedNonNull = AvroSchemaUtils.resolveNullableSchema(requested); + Schema tableNonNull = AvroSchemaUtils.resolveNullableSchema(table); + + // Perform repair on the non-null types + Schema repairedNonNull = repairAvroSchemaNonNull(requestedNonNull, tableNonNull); + + // If nothing changed, return the original schema + if (repairedNonNull == requestedNonNull) { + return requested; + } + + // If the original was a union, wrap the repaired schema back in a nullable union + if (requested.getType() == Schema.Type.UNION) { + return AvroSchemaUtils.createNullableSchema(repairedNonNull); + } + + return repairedNonNull; + } + + /** + * Repairs non-nullable schemas (after unions have been resolved). + */ + private static Schema repairAvroSchemaNonNull(Schema requested, Schema table) { + // If schemas are already equal, nothing to repair + if (requested.equals(table)) { + return requested; + } + + // If types are different, no repair can be done + if (requested.getType() != table.getType()) { + return requested; + } + + // Handle record types (nested structs) + if (requested.getType() == Schema.Type.RECORD) { + return repairRecord(requested, table); + } + + // Handle array types + if (requested.getType() == Schema.Type.ARRAY) { + Schema repairedElementSchema = repairAvroSchema(requested.getElementType(), table.getElementType()); + // If element didn't change, return original array schema + if (repairedElementSchema == requested.getElementType()) { + return requested; + } + return Schema.createArray(repairedElementSchema); + } + + // Handle map types + if (requested.getType() == Schema.Type.MAP) { + Schema repairedValueSchema = repairAvroSchema(requested.getValueType(), table.getValueType()); + // If value didn't change, return original map schema + if (repairedValueSchema == requested.getValueType()) { + return requested; + } + return Schema.createMap(repairedValueSchema); + } + + // Check primitive if we need to repair + if (needsLogicalTypeRepair(requested, table)) { + // If we need to repair, return the table schema + return table; + } + + // Default: return requested schema + return requested; + } + + /** + * Quick check if a logical type repair is needed (no allocations). + */ + private static boolean needsLogicalTypeRepair(Schema requested, Schema table) { + if (requested.getType() != Schema.Type.LONG || table.getType() != Schema.Type.LONG) { + return false; + } + + LogicalType reqLogical = requested.getLogicalType(); + LogicalType tblLogical = table.getLogicalType(); + + // if requested has no logical type, and the table has a local timestamp, then we need to repair + if (reqLogical == null) { + return tblLogical instanceof LogicalTypes.LocalTimestampMillis + || tblLogical instanceof LogicalTypes.LocalTimestampMicros; + } + + // if requested is timestamp-micros, and the table is timestamp-millis, then we need to repair + return reqLogical instanceof LogicalTypes.TimestampMicros + && tblLogical instanceof LogicalTypes.TimestampMillis; + } + + /** + * Performs record repair, returning the original schema if nothing changed. + */ + private static Schema repairRecord(Schema requestedSchema, Schema tableSchema) { + List fields = requestedSchema.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 requestedSchema; + } + + // 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( + requestedSchema.getName(), + requestedSchema.getDoc(), + requestedSchema.getNamespace(), + requestedSchema.isError(), + repairedFields + ); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java index 4f8511f2e4cea..ced158dae3b20 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java @@ -293,6 +293,44 @@ protected static List> hoodieRecordsToIndexedRecords .collect(Collectors.toList()); } + @Test + public void testTimestampRepair() throws Exception { + Map writeConfigs = new HashMap<>( + getCommonConfigs(RecordMergeMode.EVENT_TIME_ORDERING, true)); + HoodieTestDataGenerator.SchemaEvolutionConfigs schemaEvolutionConfigs = HoodieTestDataGenerator.SchemaEvolutionConfigs.createMinimalConfig(); + writeConfigs.put(HoodieTableConfig.BASE_FILE_FORMAT.key(), HoodieFileFormat.PARQUET.name()); + + try (HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(TRIP_EXAMPLE_SCHEMA, 0xDEEF)) { + //start with micros + dataGen.extendSchemaBeforeEvolutionWithTimestampLogicalType(schemaEvolutionConfigs, false); + + // Write a base file with schema A + List firstRecords = dataGen.generateInsertsForPartition("001", 5, "any_partition"); + List> firstIndexedRecords = hoodieRecordsToIndexedRecords(firstRecords, dataGen.getExtendedSchema()); + commitToTable(firstRecords, INSERT.value(), true, writeConfigs, dataGen.getExtendedSchema().toString()); + validateOutputFromFileGroupReaderWithNativeRecords( + getStorageConf(), getBasePath(), + true, 0, RecordMergeMode.EVENT_TIME_ORDERING, + firstIndexedRecords); + + // Evolve schema + dataGen.extendSchemaAfterEvolution(schemaEvolutionConfigs); + + // Write another base file with schema B + List secondRecords = dataGen.generateInsertsForPartition("002", 5, "new_partition"); + List> secondIndexedRecords = hoodieRecordsToIndexedRecords(secondRecords, dataGen.getExtendedSchema()); + commitToTable(secondRecords, INSERT.value(), false, writeConfigs, dataGen.getExtendedSchema().toString()); + // replace first index records with repaired schema + firstIndexedRecords = hoodieRecordsToIndexedRecords(firstRecords, dataGen.getExtendedSchema()); + List> mergedRecords = CollectionUtils.combine(firstIndexedRecords, secondIndexedRecords); + validateOutputFromFileGroupReaderWithNativeRecords( + getStorageConf(), getBasePath(), + true, 0, RecordMergeMode.EVENT_TIME_ORDERING, + mergedRecords); + } + } + + /** * Write a base file with schema A, then write another base file with schema B. */ 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 4aab20e0c2c01..8e2dc3973a25a 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 @@ -1573,6 +1573,28 @@ public static class SchemaEvolutionConfigs { // Bytes public boolean bytesToStringSupport = true; + + /** + * Creates a config with minimal schema evolution support (all type promotions disabled). + * Useful for testing basic schema evolution without type promotion complexity. + */ + public static SchemaEvolutionConfigs createMinimalConfig() { + SchemaEvolutionConfigs config = new SchemaEvolutionConfigs(); + config.addNewFieldSupport = false; + config.intToLongSupport = false; + config.intToFloatSupport = false; + config.intToDoubleSupport = false; + config.intToStringSupport = false; + config.longToFloatSupport = false; + config.longToDoubleSupport = false; + config.longToStringSupport = false; + config.floatToDoubleSupport = false; + config.floatToStringSupport = false; + config.doubleToStringSupport = false; + config.stringToBytesSupport = false; + config.bytesToStringSupport = false; + return config; + } } private enum SchemaEvolutionTypePromotionCase { @@ -1607,6 +1629,21 @@ private enum SchemaEvolutionTypePromotionCase { } public void extendSchema(SchemaEvolutionConfigs configs, boolean isBefore) { + extendSchemaInternal(configs, isBefore, Option.empty()); + } + + /** + * Extends the schema with timestamp logical types for LONG fields. + * + * @param configs Schema evolution configuration + * @param isBefore Whether this is the "before" schema (pre-evolution) + * @param useMillis If true, uses timestamp-millis logical type; if false, uses timestamp-micros + */ + public void extendSchemaWithTimestampLogicalType(SchemaEvolutionConfigs configs, boolean isBefore, boolean useMillis) { + extendSchemaInternal(configs, isBefore, Option.of(useMillis)); + } + + private void extendSchemaInternal(SchemaEvolutionConfigs configs, boolean isBefore, Option useMillis) { List baseFields = new ArrayList<>(); for (SchemaEvolutionTypePromotionCase evolution : SchemaEvolutionTypePromotionCase.values()) { if (evolution.isEnabled.test(configs)) { @@ -1619,7 +1656,7 @@ public void extendSchema(SchemaEvolutionConfigs configs, boolean isBefore) { baseFields.add(Schema.Type.BOOLEAN); } - this.extendedSchema = Option.of(generateExtendedSchema(configs, new ArrayList<>(baseFields))); + this.extendedSchema = Option.of(generateExtendedSchema(configs, new ArrayList<>(baseFields), useMillis)); } public void extendSchemaBeforeEvolution(SchemaEvolutionConfigs configs) { @@ -1630,32 +1667,46 @@ public void extendSchemaAfterEvolution(SchemaEvolutionConfigs configs) { extendSchema(configs, false); } + /** + * Extends the schema before evolution with timestamp logical types. + */ + public void extendSchemaBeforeEvolutionWithTimestampLogicalType(SchemaEvolutionConfigs configs, boolean useMillis) { + extendSchemaWithTimestampLogicalType(configs, true, useMillis); + } + + /** + * Extends the schema after evolution with timestamp logical types. + */ + public void extendSchemaAfterEvolutionWithTimestampLogicalType(SchemaEvolutionConfigs configs, boolean useMillis) { + extendSchemaWithTimestampLogicalType(configs, false, useMillis); + } + public Schema getExtendedSchema() { return extendedSchema.orElseThrow(IllegalArgumentException::new); } - private static Schema generateExtendedSchema(SchemaEvolutionConfigs configs, List baseFields) { - return generateExtendedSchema(configs.schema, configs, baseFields, "customField", true); + private static Schema generateExtendedSchema(SchemaEvolutionConfigs configs, List baseFields, Option useMillis) { + return generateExtendedSchema(configs.schema, configs, baseFields, "customField", true, useMillis); } - private static Schema generateExtendedSchema(Schema baseSchema, SchemaEvolutionConfigs configs, List baseFields, String fieldPrefix, boolean toplevel) { + private static Schema generateExtendedSchema(Schema baseSchema, SchemaEvolutionConfigs configs, List baseFields, String fieldPrefix, boolean toplevel, Option useMillis) { List fields = baseSchema.getFields(); List finalFields = new ArrayList<>(fields.size() + baseFields.size()); boolean addedFields = false; for (Schema.Field field : fields) { if (configs.nestedSupport && field.name().equals("fare") && field.schema().getType() == Schema.Type.RECORD) { - finalFields.add(createNewSchemaField(field.name(), generateExtendedSchema(field.schema(), configs, baseFields, "customFare", false), field.doc(), field.defaultVal())); + finalFields.add(createNewSchemaField(field.name(), generateExtendedSchema(field.schema(), configs, baseFields, "customFare", false, useMillis), field.doc(), field.defaultVal())); } else if (configs.anyArraySupport || !field.name().equals("tip_history")) { //TODO: [HUDI-9603] remove the if condition when the issue is fixed if (field.name().equals("_hoodie_is_deleted")) { addedFields = true; - addFields(configs, finalFields, baseFields, fieldPrefix, baseSchema.getNamespace(), toplevel); + addFields(configs, finalFields, baseFields, fieldPrefix, baseSchema.getNamespace(), toplevel, useMillis); } finalFields.add(createNewSchemaField(field)); } } if (!addedFields) { - addFields(configs, finalFields, baseFields, fieldPrefix, baseSchema.getNamespace(), toplevel); + addFields(configs, finalFields, baseFields, fieldPrefix, baseSchema.getNamespace(), toplevel, useMillis); } Schema finalSchema = Schema.createRecord(baseSchema.getName(), baseSchema.getDoc(), baseSchema.getNamespace(), baseSchema.isError()); @@ -1663,34 +1714,51 @@ private static Schema generateExtendedSchema(Schema baseSchema, SchemaEvolutionC return finalSchema; } - private static void addFields(SchemaEvolutionConfigs configs, List finalFields, List baseFields, String fieldPrefix, String namespace, boolean toplevel) { + private static void addFields(SchemaEvolutionConfigs configs, List finalFields, List baseFields, + String fieldPrefix, String namespace, boolean toplevel, Option useMillis) { if (toplevel) { if (configs.mapSupport) { List mapFields = new ArrayList<>(baseFields.size()); - addFieldsHelper(mapFields, baseFields, fieldPrefix + "Map"); + addFieldsHelper(mapFields, baseFields, fieldPrefix + "Map", useMillis); finalFields.add(new Schema.Field(fieldPrefix + "Map", Schema.createMap(Schema.createRecord("customMapRecord", "", namespace, false, mapFields)), "", null)); } if (configs.arraySupport) { List arrayFields = new ArrayList<>(baseFields.size()); - addFieldsHelper(arrayFields, baseFields, fieldPrefix + "Array"); + addFieldsHelper(arrayFields, baseFields, fieldPrefix + "Array", useMillis); finalFields.add(new Schema.Field(fieldPrefix + "Array", Schema.createArray(Schema.createRecord("customArrayRecord", "", namespace, false, arrayFields)), "", null)); } } - addFieldsHelper(finalFields, baseFields, fieldPrefix); + addFieldsHelper(finalFields, baseFields, fieldPrefix, useMillis); } - private static void addFieldsHelper(List finalFields, List baseFields, String fieldPrefix) { + private static void addFieldsHelper(List finalFields, List baseFields, String fieldPrefix, Option useMillis) { for (int i = 0; i < baseFields.size(); i++) { if (baseFields.get(i) == Schema.Type.BOOLEAN) { // boolean fields are added fields finalFields.add(new Schema.Field(fieldPrefix + i, AvroSchemaUtils.createNullableSchema(Schema.Type.BOOLEAN), "", null)); + } else if (baseFields.get(i) == Schema.Type.LONG && useMillis.isPresent()) { + // Apply timestamp logical type only when useMillis is present + if (useMillis.get()) { + finalFields.add(new Schema.Field(fieldPrefix + i, LogicalTypes.timestampMillis().addToSchema(Schema.create(baseFields.get(i))), "", null)); + } else { + finalFields.add(new Schema.Field(fieldPrefix + i, LogicalTypes.timestampMicros().addToSchema(Schema.create(baseFields.get(i))), "", null)); + } } else { finalFields.add(new Schema.Field(fieldPrefix + i, Schema.create(baseFields.get(i)), "", null)); } } } + // not truly random, but good enough for our purposes + private static long nextLong(Random random, long min, long max) { + if (min >= max) { + throw new IllegalArgumentException("max must be greater than min"); + } + double fraction = random.nextDouble(); // value between 0.0 and 1.0 + return min + (long)((max - min) * fraction); + } + private void generateCustomValues(GenericRecord rec, String customPrefix) { for (Schema.Field field : rec.getSchema().getFields()) { if (field.name().startsWith(customPrefix)) { @@ -1699,7 +1767,7 @@ private void generateCustomValues(GenericRecord rec, String customPrefix) { rec.put(field.name(), rand.nextInt()); break; case LONG: - rec.put(field.name(), rand.nextLong()); + rec.put(field.name(), nextLong(rand, 1129918320000L, 1761084755188L)); break; case FLOAT: rec.put(field.name(), rand.nextFloat()); 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..b567806f11f06 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/parquet/schema/TestAvroSchemaRepair.java @@ -0,0 +1,642 @@ +/* + * 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.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertSame; + +/** + * 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.resolveNullableSchema(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.resolveNullableSchema(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.resolveNullableSchema(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.resolveNullableSchema(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.resolveNullableSchema(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()); + } +} 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..ec91c124b3d9e 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 fileSchema = 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(fileSchema, schema)) { + AvroReadSupport.setAvroReadSchema(hadoopConf, fileSchema); + AvroReadSupport.setRequestedProjection(hadoopConf, fileSchema); 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..1bb6c569700bf 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,8 +42,11 @@ */ 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() { @@ -51,7 +57,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 +80,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..aecaf8c265e93 --- /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 requestedSchema, Option tableSchema) { + if (tableSchema.isEmpty()) { + return requestedSchema; + } + return repairLogicalTypes(requestedSchema, tableSchema.get()); + } + + static MessageType repairLogicalTypes(MessageType requestedSchema, MessageType tableSchema) { + List repairedFields = repairFields(requestedSchema.getFields(), tableSchema); + + // If nothing changed, return the original schema + if (repairedFields == null) { + return requestedSchema; + } + + return new MessageType(requestedSchema.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 requestedFields, GroupType tableSchema) { + // First pass: find the first field that changes + int firstChangedIndex = -1; + Type firstRepairedField = null; + + for (int i = 0; i < requestedFields.size(); i++) { + Type requestedField = requestedFields.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<>(requestedFields.size()); + + // Copy all fields before the first changed field + for (int i = 0; i < firstChangedIndex; i++) { + repairedFields.add(requestedFields.get(i)); + } + + // Add the first changed field (using cached repaired field) + repairedFields.add(firstRepairedField); + + // Process remaining fields + for (int i = firstChangedIndex + 1; i < requestedFields.size(); i++) { + Type requestedField = requestedFields.get(i); + Type repaired = requestedField; + if (tableSchema.containsField(requestedField.getName())) { + Type tableField = tableSchema.getType(requestedField.getName()); + repaired = repairField(requestedField, tableField); + } + repairedFields.add(repaired); + } + + return repairedFields; + } + + private static Type repairField(Type requested, Type table) { + if (requested.isPrimitive() && table.isPrimitive()) { + return repairPrimitiveType(requested.asPrimitiveType(), table.asPrimitiveType()); + } else if (!requested.isPrimitive() && !table.isPrimitive()) { + // recurse into nested structs + GroupType reqGroup = requested.asGroupType(); + GroupType tblGroup = table.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 requested; + } + + return new GroupType( + reqGroup.getRepetition(), + reqGroup.getName(), + reqGroup.getLogicalTypeAnnotation(), + repairedFields + ); + } else { + // fallback: keep requested + return requested; + } + } + + private static PrimitiveType repairPrimitiveType(PrimitiveType requested, PrimitiveType table) { + // Quick check if repair is needed (no allocations) + if (needsLogicalTypeRepair(requested, table)) { + return Types.primitive(table.getPrimitiveTypeName(), requested.getRepetition()) + .as(table.getLogicalTypeAnnotation()) + .named(requested.getName()); + } + return requested; + } + + /** + * Quick check if a logical type repair is needed (no allocations). + */ + private static boolean needsLogicalTypeRepair(PrimitiveType requested, PrimitiveType table) { + if (requested.getPrimitiveTypeName() != PrimitiveType.PrimitiveTypeName.INT64 + || table.getPrimitiveTypeName() != PrimitiveType.PrimitiveTypeName.INT64) { + return false; + } + LogicalTypeAnnotation reqLogical = requested.getLogicalTypeAnnotation(); + LogicalTypeAnnotation tblLogical = table.getLogicalTypeAnnotation(); + + // if requested has no logical type, and the table has a local timestamp, then we need to repair + if (reqLogical == null) { + return tblLogical instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation + && !((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) tblLogical).isAdjustedToUTC(); + } + + // if requested is timestamp-micros and table is timestamp-millis then we need to repair + return reqLogical instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation + && tblLogical instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation + && ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) reqLogical).getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS + && ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) tblLogical).getUnit() == LogicalTypeAnnotation.TimeUnit.MILLIS + && ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) reqLogical).isAdjustedToUTC() + && ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) tblLogical).isAdjustedToUTC(); + } +} 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-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..fbf2312b23214 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, requiredSchema); 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..e4e6231b4f1d2 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 @@ -18,9 +18,7 @@ package org.apache.hudi.hadoop.avro; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.hadoop.HoodieColumnProjectionUtils; import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils; import org.apache.avro.Schema; @@ -44,8 +42,6 @@ import org.apache.parquet.schema.MessageType; import java.io.IOException; -import java.util.Arrays; -import java.util.List; import static org.apache.parquet.avro.HoodieAvroParquetSchemaConverter.getAvroSchemaConverter; import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter; @@ -55,31 +51,26 @@ 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); - } - 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-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..a28131b083bb9 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 @@ -25,7 +25,7 @@ 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.collection.ClosableIterator import org.apache.hudi.data.CloseableIteratorListener @@ -87,6 +87,12 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, private lazy val avroTableSchema = new Schema.Parser().parse(tableSchema.avroSchemaStr) + private lazy val tableSchemaAsMessageType: org.apache.hudi.common.util.Option[org.apache.parquet.schema.MessageType] = { + org.apache.hudi.common.util.Option.ofNullable( + ParquetTableSchemaResolver.convertAvroSchemaToParquet(avroTableSchema, new Configuration()) + ) + } + override def shortName(): String = "HudiFileGroup" override def toString: String = "HoodieFileGroupReaderBasedFileFormat" @@ -371,14 +377,14 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, 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 +392,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..014131b74202c 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 @@ -53,6 +53,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 +61,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 +80,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 +92,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 +100,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/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..9d22e8a0d7081 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,68 @@ 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 = { + // vectorized reader not working for 3.3 and 3.4 yet + if (!vectorizedReadEnabled || HoodieSparkUtils.gteqSpark3_5) { + val prevValue = spark.conf.get("spark.sql.parquet.enableVectorizedReader", "true") + val prevTimezone = spark.conf.get("spark.sql.session.timeZone") + try { + 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: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) + } + } + } + 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 39179e966222b..ed9a18827c204 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,67 @@ 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") + try { + 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: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) + } + } + /** * 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/Spark33ParquetReader.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetReader.scala index 50fc06ea3187b..bae2ea95f79dc 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._ @@ -89,7 +92,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 +102,10 @@ 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 fileFooter = repairFooterSchema( + ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS), tableSchemaOpt) + + lazy val footerFileMetaData = fileFooter.getFileMetaData val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) @@ -281,4 +287,20 @@ object Spark33ParquetReader extends SparkParquetReaderBuilder { enableRecordFilter = sqlConf.parquetRecordFilterEnabled, 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/Spark34ParquetReader.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetReader.scala index 9692b343dd667..76d306b248a58 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._ @@ -86,7 +89,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 +99,10 @@ 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 fileFooter = repairFooterSchema( + ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS), tableSchemaOpt) + + lazy val footerFileMetaData = fileFooter.getFileMetaData val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) @@ -280,4 +286,20 @@ object Spark34ParquetReader extends SparkParquetReaderBuilder { enableRecordFilter = sqlConf.parquetRecordFilterEnabled, 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/Spark35ParquetReader.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetReader.scala index 3d192dd3a170c..3b242df8e8afc 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 @@ -85,7 +88,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,14 +98,14 @@ class Spark35ParquetReader(enableVectorizedReader: Boolean, val schemaEvolutionUtils = new ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema, internalSchemaOpt) - val fileFooter = if (enableVectorizedReader) { + val fileFooter = repairFooterSchema(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). ParquetFooterReader.readFooter(sharedConf, file, ParquetFooterReader.WITH_ROW_GROUPS) } else { ParquetFooterReader.readFooter(sharedConf, file, ParquetFooterReader.SKIP_ROW_GROUPS) - } + }, tableSchemaOpt) val footerFileMetaData = fileFooter.getFileMetaData val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( @@ -197,7 +201,8 @@ class Spark35ParquetReader(enableVectorizedReader: Boolean, convertTz, enableVectorizedReader = false, datetimeRebaseSpec, - int96RebaseSpec) + int96RebaseSpec, + tableSchemaOpt) val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) new ParquetRecordReader[InternalRow](readSupport, parquetFilter) @@ -287,4 +292,22 @@ object Spark35ParquetReader extends SparkParquetReaderBuilder { enableRecordFilter = sqlConf.parquetRecordFilterEnabled, 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/Spark40ParquetReader.scala b/hudi-spark-datasource/hudi-spark4.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark40ParquetReader.scala index 5e8cea9f11a2a..6c9b0009649dd 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 @@ -85,7 +88,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,14 +98,14 @@ class Spark40ParquetReader(enableVectorizedReader: Boolean, val schemaEvolutionUtils = new ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema, internalSchemaOpt) - val fileFooter = if (enableVectorizedReader) { + val fileFooter = repairFooterSchema(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). ParquetFooterReader.readFooter(sharedConf, file, ParquetFooterReader.WITH_ROW_GROUPS) } else { ParquetFooterReader.readFooter(sharedConf, file, ParquetFooterReader.SKIP_ROW_GROUPS) - } + }, tableSchemaOpt) val footerFileMetaData = fileFooter.getFileMetaData val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( @@ -287,4 +291,21 @@ object Spark40ParquetReader extends SparkParquetReaderBuilder { enableRecordFilter = sqlConf.parquetRecordFilterEnabled, 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 89781741e39f6..80e1f8f14ef78 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; @@ -925,6 +926,328 @@ 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 { + 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 { + 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(); + + 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; + } 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 { + 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); + + 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(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 + 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(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); + } + } + + 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: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/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..eba6ab4fe9068 --- /dev/null +++ b/hudi-utilities/src/test/resources/logical-repair/schema.avsc @@ -0,0 +1,95 @@ +{ + "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