n3nash commented on a change in pull request #1704:
URL: https://github.com/apache/hudi/pull/1704#discussion_r439533536



##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordPayload.java
##########
@@ -50,8 +50,25 @@
    * @param schema Schema used for record
    * @return new combined/merged value to be written back to storage. EMPTY to 
skip writing this record.
    */
+  @Deprecated
   Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, 
Schema schema) throws IOException;
 
+  /**
+   * This methods lets you write custom merging/combining logic to produce new 
values as a function of current value on
+   * storage and whats contained in this object.
+   * <p>
+   * eg: 1) You are updating counters, you may want to add counts to 
currentValue and write back updated counts 2) You
+   * may be reading DB redo logs, and merge them with current image for a 
database row on storage
+   *
+   * @param currentValue Current value in storage, to merge/combine this 
payload with
+   * @param schema Schema used for record
+   * @param props Payload related properties. For example pass the ordering 
field(s) name to extract from value in storage.
+   * @return new combined/merged value to be written back to storage. EMPTY to 
skip writing this record.
+   */
+  default Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord 
currentValue, Schema schema, Map<String, String> props) throws IOException {

Review comment:
       Let me explain. The current interface 
`combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)` doesn't 
allow the user to pass the ordering field names. You're trying to introduce 
this and have a Map<> to allow the user to do this. (The description link in 
the JIRA ticket doesn't work so I needed more context). 
   Now, since the user can have their their own implementation of 
`combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)`, they can 
have the ordering field name in their own class and maintain it themselves, 
thus not requiring the need to explicitly pass these names through a method. 
   I see that the intention of this PR to support this functionality 
specifically for the `OverwriteWithLatestAvroPayload`. For preCombine, will we 
just continue to support natural ordering which may not be the same as for 
combineAndGetUpdateValue, correct ?




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

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


Reply via email to