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


##########
hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java:
##########
@@ -391,22 +425,70 @@ protected Option<T> merge(Option<T> older, Map<String, 
Object> olderInfoMap,
           return newer;
         case CUSTOM:
         default:
-          Option<Pair<HoodieRecord, Schema>> mergedRecord = recordMerger.merge(
-              readerContext.constructHoodieRecord(older, olderInfoMap), 
(Schema) olderInfoMap.get(INTERNAL_META_SCHEMA),
-              readerContext.constructHoodieRecord(newer, newerInfoMap), 
(Schema) newerInfoMap.get(INTERNAL_META_SCHEMA), props);
-
-          if (mergedRecord.isPresent()
-              && 
!mergedRecord.get().getLeft().isDelete(mergedRecord.get().getRight(), props)) {
-            if (!mergedRecord.get().getRight().equals(readerSchema)) {
-              return Option.ofNullable((T) 
mergedRecord.get().getLeft().rewriteRecordWithNewSchema(mergedRecord.get().getRight(),
 null, readerSchema).getData());
+
+          if (payloadClass.isPresent()) {
+            ValidationUtils.checkArgument(!Objects.equals(payloadClass, 
OverwriteWithLatestAvroPayload.class.getCanonicalName())
+                && !Objects.equals(payloadClass, 
DefaultHoodieRecordPayload.class.getCanonicalName()));
+            HoodieRecord oldHoodieRecord = 
constructHoodieAvroRecord(readerContext, older, olderInfoMap);
+            HoodieRecord newHoodieRecord = 
constructHoodieAvroRecord(readerContext, newer, newerInfoMap);
+            Option<Pair<HoodieRecord, Schema>> mergedRecord = 
recordMerger.get().merge(
+                oldHoodieRecord, getSchemaForAvroPayloadMerge(oldHoodieRecord, 
olderInfoMap),
+                newHoodieRecord, getSchemaForAvroPayloadMerge(newHoodieRecord, 
newerInfoMap), props);
+            if (mergedRecord.isPresent()
+                && 
!mergedRecord.get().getLeft().isDelete(mergedRecord.get().getRight(), props)) {
+              if (!mergedRecord.get().getRight().equals(readerSchema)) {
+                return Option.ofNullable(
+                    readerContext.convertAvroRecord((IndexedRecord) 
mergedRecord.get().getLeft().rewriteRecordWithNewSchema(mergedRecord.get().getRight(),
 null, readerSchema).getData()));
+              }
+              return 
Option.ofNullable(readerContext.convertAvroRecord((IndexedRecord)  
mergedRecord.get().getLeft().getData()));
+            }
+          } else {
+            Option<Pair<HoodieRecord, Schema>> mergedRecord = 
recordMerger.get().merge(
+                readerContext.constructHoodieRecord(older, olderInfoMap), 
(Schema) olderInfoMap.get(INTERNAL_META_SCHEMA),
+                readerContext.constructHoodieRecord(newer, newerInfoMap), 
(Schema) newerInfoMap.get(INTERNAL_META_SCHEMA), props);
+            if (mergedRecord.isPresent()
+                && 
!mergedRecord.get().getLeft().isDelete(mergedRecord.get().getRight(), props)) {
+              if (!mergedRecord.get().getRight().equals(readerSchema)) {
+                return Option.ofNullable((T) 
mergedRecord.get().getLeft().rewriteRecordWithNewSchema(mergedRecord.get().getRight(),
 null, readerSchema).getData());
+              }
+              return Option.ofNullable((T) 
mergedRecord.get().getLeft().getData());
             }
-            return Option.ofNullable((T) 
mergedRecord.get().getLeft().getData());
           }
           return Option.empty();
       }
     }
   }
 
+  /**
+   * Constructs a new {@link HoodieAvroRecord} for payload based merging
+   *
+   * @param readerContext reader context
+   * @param recordOption An option of the record in engine-specific type if 
exists.
+   * @param metadataMap  The record metadata.
+   * @return A new instance of {@link HoodieRecord}.
+   */
+  private HoodieRecord constructHoodieAvroRecord(HoodieReaderContext<T> 
readerContext, Option<T> recordOption, Map<String, Object> metadataMap) {

Review Comment:
   We discussed that this logic of converting engine-specific record (e.g., 
InternalRow) to the Avro record is only used by the new file group reader, and 
eventually we'll make the read path efficient by avoid unnecessary conversions 
end-to-end.  This direction provides better performance in the long run by 
allowing row-based merging directly through the file group reader (right now, 
we need this adapter for custom payload merging to fall back to Avro-based 
merging).



-- 
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