yihua commented on code in PR #18478:
URL: https://github.com/apache/hudi/pull/18478#discussion_r3046544058


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergedReadHandle.java:
##########
@@ -56,26 +56,24 @@ public class HoodieMergedReadHandle<T, I, K, O> extends 
HoodieReadHandle<T, I, K
   protected final Schema baseFileReaderSchema;
   private final Option<FileSlice> fileSliceOpt;
 
-  public HoodieMergedReadHandle(HoodieWriteConfig config,
-                                Option<String> instantTime,
-                                HoodieTable<T, I, K, O> hoodieTable,
-                                Pair<String, String> partitionPathFileIDPair) {
-    this(config, instantTime, hoodieTable, partitionPathFileIDPair, 
Option.empty());
+  public HoodieMergedReadHandle(HoodieWriteConfig config, Option<String> 
instantTime,
+                                HoodieTable<T, I, K, O> hoodieTable, 
Pair<String, String> partitionPathFileIDPair,
+                                Schema baseFileReaderSchema, boolean 
hasTimestampFields) {
+    this(config, instantTime, hoodieTable, partitionPathFileIDPair, 
baseFileReaderSchema, hasTimestampFields, Option.empty());
   }
 
-  public HoodieMergedReadHandle(HoodieWriteConfig config,
-                                Option<String> instantTime,
-                                HoodieTable<T, I, K, O> hoodieTable,
-                                Pair<String, String> partitionPathFileIDPair,
+  public HoodieMergedReadHandle(HoodieWriteConfig config, Option<String> 
instantTime,
+                                HoodieTable<T, I, K, O> hoodieTable, 
Pair<String, String> partitionPathFileIDPair,
+                                Schema baseFileReaderSchema, boolean 
hasTimestampFields,
                                 Option<FileSlice> fileSliceOption) {
     super(config, instantTime, hoodieTable, partitionPathFileIDPair);
     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());
+    this.baseFileReaderSchema = baseFileReaderSchema;
     fileSliceOpt = fileSliceOption.isPresent() ? fileSliceOption : 
getLatestFileSlice();
     // Repair reader schema.
     // Assume writer schema should be correct. If not, no repair happens.
-    readerSchema = AvroSchemaUtils.getRepairedSchema(orignalReaderSchema, 
baseFileReaderSchema);
+    readerSchema = hasTimestampFields ? 
AvroSchemaUtils.getRepairedSchema(orignalReaderSchema, 
this.baseFileReaderSchema) : orignalReaderSchema;

Review Comment:
   🤖 The guard `hasTimestampFields` is checked via `hasTimestampMillisField`, 
which returns true only for `TimestampMillis` and `LocalTimestampMillis`. 
However, `needsLogicalTypeRepair` in `AvroSchemaRepair` also repairs the case 
where the file schema is a plain LONG and the table schema has 
`LocalTimestampMicros`. In that scenario, `hasTimestampMillisField` returns 
false but the repair would still be needed. Could you verify this gap doesn't 
affect any realistic schema canonicalization path, or broaden the check to 
cover all timestamp logical types that `needsLogicalTypeRepair` handles?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##########
@@ -584,8 +587,10 @@ private static HoodieData<HoodieRecord> 
readRecordKeysFromFileSliceSnapshot(Hood
       final String partition = partitionAndFileSlice.getKey();
       final FileSlice fileSlice = partitionAndFileSlice.getValue();
       final String fileId = fileSlice.getFileId();
-      return new HoodieMergedReadHandle(dataWriteConfig, instantTime, 
hoodieTable, Pair.of(partition, fileSlice.getFileId()),
-          Option.of(fileSlice)).getMergedRecords().stream().map(record -> {
+      Schema baseFileReaderSchema = HoodieAvroUtils.addMetadataFields(new 
Schema.Parser().parse(dataWriteConfig.getWriteSchema()), 
dataWriteConfig.allowOperationMetadataField());

Review Comment:
   🤖 The schema parsing and `hasTimestampMillisField` computation are inside 
the `flatMap` lambda, so they're re-evaluated for every partition/file-slice 
pair. The PR description says this should be computed on the driver side. Could 
you hoist `baseFileReaderSchema` and `hasTimestampFields` above the `flatMap`, 
similar to how it's done in `HoodieIndexUtils.getExistingRecords`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to