[ 
https://issues.apache.org/jira/browse/HUDI-2170?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17381402#comment-17381402
 ] 

ASF GitHub Bot commented on HUDI-2170:
--------------------------------------

nsivabalan commented on a change in pull request #3267:
URL: https://github.com/apache/hudi/pull/3267#discussion_r670539525



##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/model/OverwriteWithLatestAvroPayload.java
##########
@@ -90,6 +93,14 @@ protected boolean isDeleteRecord(GenericRecord 
genericRecord) {
     return (deleteMarker instanceof Boolean && (boolean) deleteMarker);
   }
 
+  /**
+   * Returns the ordering value of given record {@code record}.
+   */
+  protected static Object getOrderingVal(GenericRecord record, Properties 
properties) {
+    return getNestedFieldVal(record,
+        
properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), 
true);

Review comment:
       probably the last argument value differs depending on whether we are 
reading persisted record or an incoming record. So, might have to make it an 
arg to this method. 

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java
##########
@@ -113,8 +120,11 @@ protected AbstractHoodieLogRecordScanner(FileSystem fs, 
String basePath, List<St
     this.readerSchema = readerSchema;
     this.latestInstantTime = latestInstantTime;
     this.hoodieTableMetaClient = 
HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(basePath).build();
+    this.orderingField = 
this.hoodieTableMetaClient.getTableConfig().getPreCombineField();
     // load class from the payload fully qualified class name
     this.payloadClassFQN = 
this.hoodieTableMetaClient.getTableConfig().getPayloadClass();
+    // only the OverwriteWithLatestAvroPayload needs combining by orderingVal
+    this.naturalOrder = 
!OverwriteWithLatestAvroPayload.class.isAssignableFrom(ReflectionUtils.getClass(this.payloadClassFQN));

Review comment:
       Couple of points to ponder
   - We should try to be agnostic to payload impl's in this class. Should not 
leak OverwriteWithLatestAvroPayload
   - One another option is to add another constructor which takes in 
orderingField (instead of ordering Val) to the payload class. Basically this 
will move the line 318 to within Payload constructor itself. Basically one 
constructor will be used in the direct write path from user (ordering val) and 
another constructor will be used when reading a record already persisted. 

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/model/OverwriteWithLatestAvroPayload.java
##########
@@ -90,6 +93,14 @@ protected boolean isDeleteRecord(GenericRecord 
genericRecord) {
     return (deleteMarker instanceof Boolean && (boolean) deleteMarker);
   }
 
+  /**
+   * Returns the ordering value of given record {@code record}.
+   */
+  protected static Object getOrderingVal(GenericRecord record, Properties 
properties) {

Review comment:
       PAYLOAD_ORDERING_FIELD_PROP_KEY is something we introduced recently when 
we added DefaultHoodieRecordPayload. We need to ensure that this value will be 
set in all code paths even while using OverwriteWithLatestAvroPayload




-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Always choose the latest record for HoodieRecordPayload
> -------------------------------------------------------
>
>                 Key: HUDI-2170
>                 URL: https://issues.apache.org/jira/browse/HUDI-2170
>             Project: Apache Hudi
>          Issue Type: Improvement
>          Components: Common Core
>            Reporter: Danny Chen
>            Assignee: Danny Chen
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 0.9.0
>
>
> Now in {{OverwriteWithLatestAvroPayload.preCombine}}, we still choose the old 
> record when the new record has the same preCombine field with the old one, 
> actually it is more natural to keep the new incoming record instead. The 
> {{DefaultHoodieRecordPayload.combineAndGetUpdateValue}} method already did 
> that.
> See issue: https://github.com/apache/hudi/issues/3266.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to