Skip to content

fix: timestamp logical types#18132

Merged
danny0405 merged 60 commits into
apache:release-0.14.2-prepfrom
linliu-code:pr-18100-1
Apr 10, 2026
Merged

fix: timestamp logical types#18132
danny0405 merged 60 commits into
apache:release-0.14.2-prepfrom
linliu-code:pr-18100-1

Conversation

@linliu-code
Copy link
Copy Markdown
Collaborator

@linliu-code linliu-code commented Feb 8, 2026

Describe the issue this Pull Request addresses

This PR combines mainly two PRs that fixing timestamp_millis logical type issue.

  1. fix(ingest): Repair affected logical timestamp milli tables #14161
  2. feat(metadata): Improve Logical Type Handling on Col Stats #13711

Summary and Changelog

The below is the PR description from #14161.

This pr #9743 adds more schema evolution functionality and schema processing. However, we used the InternalSchema system to do various operations such as fix null ordering, reorder, and add columns. At the time, InternalSchema only had a single Timestamp type. When converting back to avro, this was assumed to be micros. Therefore, if the schema provider had any millis columns, the processed schema would end up with those columns as micros.

In this pr to update column stats with better support for logical types: #13711, the schema issues were fixed, as well as additional issues with handling and conversion of timestamps during ingestion.

this pr aims to add functionality to spark and hive readers and writers to automatically repair affected tables.
After switching to use the 1.1 binary, the affected columns will undergo evolution from timestamp-micros to timestamp-mills. Normally a lossy evolution that is not supported, this evolution is ok because the data is actually still timestamp-millis it is just mislabeled as micros in the parquet and table schemas

Impact

When reading from a hudi table using spark or hive reader if the table schema has a column as millis, but the data schema is micros, we will assume that this column is affected and read it as a millis value instead of a micros value. This correction is also applied to all readers that the default write paths use. As a table is rewritten the parquet files will be correct. A table's latest snapshot can be immediately fixed by writing one commit with the 1.1 binary.

Risk Level

High, extensive testing was done and functional tests were added.

Documentation Update

Contributor's checklist

  • Read through contributor's guide
  • Enough context is provided in the sections above
  • Adequate tests were added if applicable

@github-actions github-actions Bot added the size:XL PR with lines of changes > 1000 label Feb 8, 2026
val enableVectorizedReader: Boolean =
sqlConf.parquetVectorizedReaderEnabled &&
resultSchema.forall(_.dataType.isInstanceOf[AtomicType])
ParquetUtils.isBatchReadSupportedForSchema(sqlConf, resultSchema)
Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is the fix.

@apache apache deleted a comment from hudi-bot Feb 10, 2026
@linliu-code linliu-code marked this pull request as ready for review February 10, 2026 04:41
Copy link
Copy Markdown
Collaborator

@lokeshj1703 lokeshj1703 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@linliu-code Found few optimisation which can be added. These would be needed in branch-0.x as well.


Schema writerSchema = mergeHandle.getWriterSchemaWithMetaFields();
Schema readerSchema = baseFileReader.getSchema();
Schema readerSchema = AvroSchemaUtils.getRepairedSchema(baseFileReader.getSchema(), writerSchema);
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I can see this code getting executed in executor. Ran test: org.apache.hudi.functional.TestRecordLevelIndex#testRLIUpsert

This might be a problem with branch-0.x as well.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can you help me understand, why are we not trying to infer isLogicalTimestampRepairEnabled in the executor code?
isn't the idea to parse the schema once in the driver and infer it in the executor and avoid schema repair code altogether if table does not contain any logical types at all

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is difficult to fix. Multiple nested callers and functions like handleUpdateInternal are involved here.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

but why can't we add it to hadoopConfiguration thats part of table.getHadoopConf() in the driver and then fetch it from here

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Addressed

Schema orignalReaderSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()), config.allowOperationMetadataField());
// config.getSchema is not canonicalized, while config.getWriteSchema is canonicalized. So, we have to use the canonicalized schema to read the existing data.
baseFileReaderSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getWriteSchema()), config.allowOperationMetadataField());
// Repair reader schema.
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This would also be executed in executor and we can probably optimise by adding a flag in the caller. Could not validate though.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lets fix this. should not take much effort to fix.
lets fix 0.15.1 if need be.

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Addressed
#18478 for 0.15

this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
this.enableOptimizedLogBlocksScan = enableOptimizedLogBlocksScan;
this.enableLogicalTimestampFieldRepair = !hoodieTableMetaClient.isMetadataTable() && fs.getConf().getBoolean(HoodieFileReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR,
readerSchema != null && AvroSchemaUtils.hasTimestampMillisField(readerSchema));
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is also getting executed in the executor code path. We can optimise by adding a flag in the caller instead. Validated in the logical repair tests added in TestHoodieDeltaStreamer.

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we took an informed decision to add optimisation for just metadata table here. We can probably check the others.

@linliu-code
Copy link
Copy Markdown
Collaborator Author

Will address these comments after cherry-pick commits from branch-0.x

@linliu-code linliu-code force-pushed the pr-18100-1 branch 7 times, most recently from 2b481e3 to 722842e Compare March 16, 2026 13:54
@apache apache deleted a comment from hudi-bot Mar 20, 2026
yihua
yihua previously approved these changes Apr 3, 2026
Copy link
Copy Markdown
Contributor

@yihua yihua left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

🤖 This review was generated by an AI agent and may contain mistakes. Please verify any suggestions before applying.

@nsivabalan
Copy link
Copy Markdown
Contributor

image

@yihua yihua dismissed their stale review April 5, 2026 04:36

Accidentally approved

@nsivabalan
Copy link
Copy Markdown
Contributor

why the PR desc is empty? Can we fill that in please

Copy link
Copy Markdown
Contributor

@nsivabalan nsivabalan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see we are removing SparkAvroPostProcessor.
can we just leave it and not add it by default.
in minor release, wondering if we should remove a utility class.

Schema orignalReaderSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()), config.allowOperationMetadataField());
// config.getSchema is not canonicalized, while config.getWriteSchema is canonicalized. So, we have to use the canonicalized schema to read the existing data.
baseFileReaderSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getWriteSchema()), config.allowOperationMetadataField());
// Repair reader schema.
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lets fix this. should not take much effort to fix.
lets fix 0.15.1 if need be.


Schema writerSchema = mergeHandle.getWriterSchemaWithMetaFields();
Schema readerSchema = baseFileReader.getSchema();
Schema readerSchema = AvroSchemaUtils.getRepairedSchema(baseFileReader.getSchema(), writerSchema);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1


Schema writerSchema = mergeHandle.getWriterSchemaWithMetaFields();
Schema readerSchema = baseFileReader.getSchema();
Schema readerSchema = AvroSchemaUtils.getRepairedSchema(baseFileReader.getSchema(), writerSchema);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can you help me understand, why are we not trying to infer isLogicalTimestampRepairEnabled in the executor code?
isn't the idea to parse the schema once in the driver and infer it in the executor and avoid schema repair code altogether if table does not contain any logical types at all

*/
def hasTimestampMillisField(schema: Schema): Boolean = {
if (schema == null) {
true
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why do we return true if the schema is null?

Copy link
Copy Markdown
Contributor

@nsivabalan nsivabalan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I need to pull down the changes and do some additional inspection. for now, you can address existing feedback.

* @param avroSchema the table's Avro schema
* @return set of field names whose type is timestamp-millis or local-timestamp-millis
*/
def getTimestampMillisColumns(avroSchema: org.apache.avro.Schema): Set[String] = {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

are we considering only top level fields here?

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems like its only top level fields being considered here

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Apache master uses index definition to get source fields and validates those source fields

ss.emptyDataFrame
// Avoid calling isEmpty() which can cause serialization issues with Ordering$Reverse
// Check partition count instead, which doesn't require task serialization
val structType = convertAvroSchemaToStructType(new Schema.Parser().parse(schemaStr))
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is this optimization applied to 0.15.1 as well ?

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The code is different in 0.15.1. This optimisation is not required there.


Schema writerSchema = mergeHandle.getWriterSchemaWithMetaFields();
Schema readerSchema = baseFileReader.getSchema();
Schema readerSchema = AvroSchemaUtils.getRepairedSchema(baseFileReader.getSchema(), writerSchema);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

but why can't we add it to hadoopConfiguration thats part of table.getHadoopConf() in the driver and then fetch it from here

// 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)
if (!requestedSchema.isPresent()) {
Schema repairedFileSchema = getRepairedSchema(getSchema(), schema);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When we are instantiating the base file reader in L84 in HoodieMergeHelper, if we can embed a boolean flag in hadoopConf, we can fetch it again here and avoid repair calls for tables w/o any logical type.

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Addressed

conf.setBoolean(AvroReadSupport.AVRO_COMPATIBILITY, enableCompatibility);
}
return new HoodieAvroReadSupport<>(model);
return new HoodieAvroReadSupport<>(model, Option.ofNullable(tableSchema).map(schema -> getAvroSchemaConverter(conf).convert(schema)),
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if hadoopConf has the value for hasLogicalTsField, we can also avoid the additional call in L90

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Addressed

new HoodieParquetReadSupport(
convertTz,
enableVectorizedReader = false,
enableTimestampFieldRepair = true,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shouldn't we set the value to hasTimestampMillisFieldInTableSchema

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Addressed

}

@ParameterizedTest
@CsvSource(value = {"SIX,AVRO,CLUSTER", "CURRENT,AVRO,NONE", "CURRENT,AVRO,CLUSTER", "CURRENT,SPARK,NONE", "CURRENT,SPARK,CLUSTER"})
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

in 0.x, we don't have support for multiple writer versions. CURRENT and SIX are one and the same.
can we trim the unnecessary combinations.

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Addressed


@ParameterizedTest
@CsvSource(value = {
"SIX,AVRO,CLUSTER,AVRO",
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we trim the unnecessary combinations.

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Addressed

@nsivabalan
Copy link
Copy Markdown
Contributor

and can you point me to the places where we avoid calls to repairSchema if its metadata table. I don't remember seeing it.

Copy link
Copy Markdown
Contributor

@nsivabalan nsivabalan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

for MOR reads:

HoodieMergeOnReadRDD
L89 -> where we broadcast the hadoop conf, lets inject the hasLogicalTs into it in the driver.

and this eventually will get wired all the way to AbstractHoodieLogRecordReader.fs.getHadoopConf
This piece of code will be executed in executor. We can rely on the boolean flag and decide whether to call repairSchema or not.

this.forceFullScan = forceFullScan;
this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
this.enableOptimizedLogBlocksScan = enableOptimizedLogBlocksScan;
this.enableLogicalTimestampFieldRepair = !hoodieTableMetaClient.isMetadataTable() && fs.getConf().getBoolean(HoodieFileReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

so, this is the only place where we optimize for mdt is it?

@nsivabalan
Copy link
Copy Markdown
Contributor

nsivabalan commented Apr 8, 2026

for MOR reads:

HoodieMergeOnReadRDD L89 -> where we broadcast the hadoop conf, lets inject the hasLogicalTs into it in the driver.

and this eventually will get wired all the way to AbstractHoodieLogRecordReader.fs.getHadoopConf This piece of code will be executed in executor. We can rely on the boolean flag and decide whether to call repairSchema or not.

btw (HoodieMergeOnReadRDD L89) is going to be invoked N no of times for N file groups in driver, then we might be parsing the schema N no of times unless we cache the value of hasLogicalTs at a table level. So, lets add an instance variable in the class and make it lazy

@CsvSource(value = {"SIX,AVRO,CLUSTER", "CURRENT,AVRO,NONE", "CURRENT,AVRO,CLUSTER", "CURRENT,SPARK,NONE", "CURRENT,SPARK,CLUSTER"})
void testCOWLogicalRepair(String tableVersion, String recordType, String operation) throws Throwable {
@CsvSource(value = {"CLUSTER", "NONE"})
void testCOWLogicalRepair(String operation) throws Throwable {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lets include SPARK as well and not just AVRO

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Addressed

ParquetReader<IndexedRecord> reader =
new HoodieAvroParquetReaderBuilder<IndexedRecord>(path)
new HoodieAvroParquetReaderBuilder<IndexedRecord>(path,
AvroSchemaUtils.isLogicalTimestampRepairNeeded(conf, false) || schema == null || AvroSchemaUtils.hasTimestampMillisField(schema))
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why the default value is false here, but true in all other places?

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This has been removed now

Schema readerSchema = AvroSchemaUtils.getRepairedSchema(baseFileReader.getSchema(), writerSchema);

Schema readerSchema;
if (AvroSchemaUtils.isLogicalTimestampRepairNeeded(table.getHadoopConf(), true)) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this line is executed in the executor. we should parse the schema in the driver and set the boolean flag in hadoop conf at one of the caller site in the driver.

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is just a check in hadoop config, its not parsing the schema. I had checked the caller, I think it was HoodieCompactor.

if (AvroSchemaUtils.hasTimestampMillisField(writerSchema)) {
AvroSchemaUtils.setLogicalTimestampRepairNeeded(table.getHadoopConf());
}
AvroSchemaUtils.setLogicalTimestampRepairIfNotSet(table.getHadoopConf(), () -> AvroSchemaUtils.hasTimestampMillisField(writerSchema));
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do we need to check for mdt here and avoid setting the config?

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Addressed


jobConf.set(HoodieFileReader.ENABLE_LOGICAL_TIMESTAMP_REPAIR,
java.lang.Boolean.toString(hasTimestampMillisFieldInTableSchema))
AvroSchemaUtils.setLogicalTimestampRepairIfNotSet(jobConf, JFunction.toJavaSupplier(() => hasTimestampMillisFieldInTableSchema.asInstanceOf[java.lang.Boolean]))
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do we need to check for mdt here?

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Addressed

@hudi-bot
Copy link
Copy Markdown
Collaborator

hudi-bot commented Apr 9, 2026

CI report:

Bot commands @hudi-bot supports the following commands:
  • @hudi-bot run azure re-run the last Azure build

nsivabalan added a commit that referenced this pull request Apr 10, 2026
The PR adds optimisation for merged read handle so that schema is repaired only if schema has timestamp millis field. The existence of timestamp millis field is computed in driver.
It also addresses review comments in PR #18132 for 0.15.1 version so that repair is applied only in cases where timestamp millis field is present in schema.

---------

Co-authored-by: Lokesh Jain <ljain@Lokeshs-MacBook-Pro.local>
Co-authored-by: Lokesh Jain <ljain@192.168.1.5>
Co-authored-by: sivabalan <n.siva.b@gmail.com>
@danny0405 danny0405 merged commit a1ac307 into apache:release-0.14.2-prep Apr 10, 2026
89 of 92 checks passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

size:XL PR with lines of changes > 1000

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants