3232import java .util .Properties ;
3333
3434/**
35- * subclass of OverwriteNonDefaultsWithLatestAvroPayload used for Partial update Hudi Table.
35+ * Payload clazz that is used for partial update Hudi Table.
3636 *
37- * Simplified partial update Logic:
38- * 1 preCombine
39- * For every record with duplicate record (same record key) in the same batch or in the delta logs that belongs to same File Group
40- * Check if one record's ordering value is larger than the other record. If yes,overwrite the exists one for specified fields
41- * that doesn't equal to null.
37+ * <p>Simplified partial update Logic:
38+ * <pre>
39+ * 1. #preCombine
40+ * For records with the same record key in one batch
41+ * or in the delta logs that belongs to same File Group,
42+ * Checks whether one record's ordering value is larger than the other record.
43+ * If yes, overwrites the existing one for specified fields that doesn't equal to null.
4244 *
43- * 2 combineAndGetUpdateValue
44- * For every incoming record with exists record in storage (same record key)
45- * Check if incoming record's ordering value is larger than exists record. If yes,overwrite the exists one for specified fields
46- * that doesn't equal to null.
47- * else overwrite the incoming one with exists record for specified fields that doesn't equal to null
48- * get a merged record, write to file .
45+ * 2. # combineAndGetUpdateValue
46+ * For every incoming record with existing record in storage (same record key)
47+ * Checks whether incoming record's ordering value is larger than the existing record.
48+ * If yes, overwrites the existing one for specified fields that doesn't equal to null.
49+ * else overwrites the incoming one with the existing record for specified fields that doesn't equal to null
50+ * and returns a merged record.
4951 *
5052 * Illustration with simple data.
5153 * let's say the order field is 'ts' and schema is :
6668 * id ts name price
6769 * 1 , 2 , null , price_2
6870 *
69- * Result data after preCombine or combineAndGetUpdateValue:
71+ * Result data after # preCombine or # combineAndGetUpdateValue:
7072 * id ts name price
7173 * 1 , 2 , name_1 , price_2
7274 *
8183 * Result data after preCombine or combineAndGetUpdateValue:
8284 * id ts name price
8385 * 1 , 2 , name_1 , price_1
84- *
85- *
86- * <ol>
87- * <li>preCombine - Picks the latest delta record for a key, based on an ordering field, then overwrite the older one for specified fields
88- * that doesn't equal null.
89- * <li>combineAndGetUpdateValue/getInsertValue - overwrite the older record for specified fields
90- * that doesn't equal null.
91- * </ol>
86+ *</pre>
9287 */
9388public class PartialUpdateAvroPayload extends OverwriteNonDefaultsWithLatestAvroPayload {
9489
@@ -106,16 +101,13 @@ public PartialUpdateAvroPayload preCombine(OverwriteWithLatestAvroPayload oldVal
106101 // use natural order for delete record
107102 return this ;
108103 }
109- boolean isOldRecordNewer = false ;
110- if (oldValue .orderingVal .compareTo (orderingVal ) > 0 ) {
111- // pick the payload with greatest ordering value as insert record
112- isOldRecordNewer = true ;
113- }
104+ // pick the payload with greater ordering value as insert record
105+ final boolean isOldRecordNewer = oldValue .orderingVal .compareTo (orderingVal ) > 0 ? true : false ;
114106 try {
115- GenericRecord indexedOldValue = (GenericRecord ) oldValue .getInsertValue (schema ).get ();
116- Option <IndexedRecord > optValue = combineAndGetUpdateValue ( indexedOldValue , schema , isOldRecordNewer );
117- if (optValue .isPresent ()) {
118- return new PartialUpdateAvroPayload ((GenericRecord ) optValue .get (),
107+ GenericRecord oldRecord = (GenericRecord ) oldValue .getInsertValue (schema ).get ();
108+ Option <IndexedRecord > mergedRecord = mergeOldRecord ( oldRecord , schema , isOldRecordNewer );
109+ if (mergedRecord .isPresent ()) {
110+ return new PartialUpdateAvroPayload ((GenericRecord ) mergedRecord .get (),
119111 isOldRecordNewer ? oldValue .orderingVal : this .orderingVal );
120112 }
121113 } catch (Exception ex ) {
@@ -124,59 +116,71 @@ public PartialUpdateAvroPayload preCombine(OverwriteWithLatestAvroPayload oldVal
124116 return this ;
125117 }
126118
127- private Option <IndexedRecord > combineAndGetUpdateValue (IndexedRecord currentValue , Schema schema , boolean shouldInsertCurrentValue ) throws IOException {
119+ @ Override
120+ public Option <IndexedRecord > combineAndGetUpdateValue (IndexedRecord currentValue , Schema schema ) throws IOException {
121+ return this .mergeOldRecord (currentValue , schema , false );
122+ }
123+
124+ @ Override
125+ public Option <IndexedRecord > combineAndGetUpdateValue (IndexedRecord currentValue , Schema schema , Properties prop ) throws IOException {
126+ return mergeOldRecord (currentValue , schema , isRecordNewer (currentValue , prop ));
127+ }
128+
129+ /**
130+ * Return true if value equals defaultValue otherwise false.
131+ */
132+ public Boolean overwriteField (Object value , Object defaultValue ) {
133+ return value == null ;
134+ }
135+
136+ // -------------------------------------------------------------------------
137+ // Utilities
138+ // -------------------------------------------------------------------------
139+
140+ private Option <IndexedRecord > mergeOldRecord (
141+ IndexedRecord oldRecord ,
142+ Schema schema ,
143+ boolean isOldRecordNewer ) throws IOException {
128144 Option <IndexedRecord > recordOption = getInsertValue (schema );
129145
130146 if (!recordOption .isPresent ()) {
147+ // use natural order for delete record
131148 return Option .empty ();
132149 }
133150
134- GenericRecord insertRecord ;
135- GenericRecord currentRecord ;
136- if (shouldInsertCurrentValue ) {
137- insertRecord = (GenericRecord ) currentValue ;
138- currentRecord = (GenericRecord ) recordOption .get ();
139- } else {
140- insertRecord = (GenericRecord ) recordOption .get ();
141- currentRecord = (GenericRecord ) currentValue ;
142- }
143-
144- return mergeRecords (schema , insertRecord , currentRecord );
145- }
151+ GenericRecord baseRecord = isOldRecordNewer ? (GenericRecord ) oldRecord : (GenericRecord ) recordOption .get ();
152+ GenericRecord mergedRecord = isOldRecordNewer ? (GenericRecord ) recordOption .get () : (GenericRecord ) oldRecord ;
146153
147- @ Override
148- public Option <IndexedRecord > combineAndGetUpdateValue (IndexedRecord currentValue , Schema schema ) throws IOException {
149- return this .combineAndGetUpdateValue (currentValue , schema , false );
154+ return mergeRecords (schema , baseRecord , mergedRecord );
150155 }
151156
152- @ Override
153- public Option <IndexedRecord > combineAndGetUpdateValue (IndexedRecord currentValue , Schema schema , Properties prop ) throws IOException {
154-
157+ /**
158+ * Returns whether the given record is newer than the record of this payload.
159+ *
160+ * @param record The record
161+ * @param prop The payload properties
162+ *
163+ * @return true if the given record is newer
164+ */
165+ private boolean isRecordNewer (IndexedRecord record , Properties prop ) {
155166 String orderingField = prop .getProperty (HoodiePayloadProps .PAYLOAD_ORDERING_FIELD_PROP_KEY );
156- boolean isOldRecordNewer = false ;
157-
158167 if (!StringUtils .isNullOrEmpty (orderingField )) {
159-
160168 boolean consistentLogicalTimestampEnabled = Boolean .parseBoolean (prop .getProperty (
161169 KeyGeneratorOptions .KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED .key (),
162170 KeyGeneratorOptions .KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED .defaultValue ()));
163171
164- Comparable oldOrderingVal = (Comparable )HoodieAvroUtils .getNestedFieldVal ((GenericRecord ) currentValue ,
165- orderingField ,
166- true , consistentLogicalTimestampEnabled );
167- if (oldOrderingVal != null && ReflectionUtils .isSameClass (oldOrderingVal , orderingVal )
168- && oldOrderingVal .compareTo (orderingVal ) > 0 ) {
169- // pick the payload with greatest ordering value as insert record
170- isOldRecordNewer = true ;
171- }
172+ Comparable oldOrderingVal =
173+ (Comparable ) HoodieAvroUtils .getNestedFieldVal (
174+ (GenericRecord ) record ,
175+ orderingField ,
176+ true ,
177+ consistentLogicalTimestampEnabled );
178+
179+ // pick the payload with greater ordering value as insert record
180+ return oldOrderingVal != null
181+ && ReflectionUtils .isSameClass (oldOrderingVal , orderingVal )
182+ && oldOrderingVal .compareTo (orderingVal ) > 0 ;
172183 }
173- return combineAndGetUpdateValue (currentValue , schema , isOldRecordNewer );
174- }
175-
176- /**
177- * Return true if value equals defaultValue otherwise false.
178- */
179- public Boolean overwriteField (Object value , Object defaultValue ) {
180- return value == null ;
184+ return false ;
181185 }
182186}
0 commit comments