stayrascal commented on a change in pull request #4724:
URL: https://github.com/apache/hudi/pull/4724#discussion_r811144363



##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/model/PartialOverwriteWithLatestAvroPayload.java
##########
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import org.apache.hudi.common.util.Option;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+
+import static org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro;
+
+/**
+ * The only difference with {@link OverwriteNonDefaultsWithLatestAvroPayload} 
is that it supports
+ * merging the latest non-null partial fields with the old record instead of 
replacing the whole record.
+ * And merging the non-null fields during preCombine multiple records with 
same record key instead of choosing the latest record based on ordering field.
+ *
+ * <p> Regarding #combineAndGetUpdateValue, Assuming a {@link GenericRecord} 
has row schema: (f0 int , f1 int, f2 int).
+ * The first record value is: (1, 2, 3), the second record value is: (4, 5, 
null) with the field f2 value as null.
+ * Calling the #combineAndGetUpdateValue method of the two records returns 
record: (4, 5, 3).
+ * Note that field f2 value is ignored because it is null. </p>
+ *
+ * <p> Regarding #preCombine, Assuming a {@link GenericRecord} has row schema: 
(f0 int , f1 int, f2 int, o1 int),
+ * and initial two {@link PartialOverwriteWithLatestAvroPayload} with 
different ordering value.
+ * The first record value is (1, null, 1, 1) with the filed f1 value as null, 
the second value is: (2, 2, null, 2) with the f2 value as null.
+ * Calling the #preCombine method of the two records returns record: (2, 2, 1, 
2).
+ * Note:
+ * <ol>
+ *   <li>the field f0 value is 2 because the ordering value of second record 
is bigger.</li>
+ *   <li>the filed f1 value is 2 because the f2 value of first record is 
null.</li>
+ *   <li>the filed f2 value is 1 because the f2 value of second record is 
null.</li>
+ *   <li>the filed o1 value is 2 because the ordering value of second record 
is bigger.</li>
+ * </ol>
+ *
+ * </p>
+ */
+public class PartialOverwriteWithLatestAvroPayload extends 
OverwriteWithLatestAvroPayload {
+
+  public PartialOverwriteWithLatestAvroPayload(GenericRecord record, 
Comparable orderingVal) {
+    this(record, orderingVal, null);
+  }
+
+  public PartialOverwriteWithLatestAvroPayload(GenericRecord record, 
Comparable orderingVal, String schema) {
+    super(record, orderingVal, schema);
+  }
+
+  public PartialOverwriteWithLatestAvroPayload(Option<GenericRecord> record) {
+    super(record); // natural order
+  }
+
+  @Override
+  public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord 
currentValue, Schema schema) throws IOException {
+    if (recordBytes.length == 0) {
+      return Option.empty();
+    }
+
+    GenericRecord incomingRecord = bytesToAvro(recordBytes, schema);
+    if (isDeleteRecord(incomingRecord)) {
+      return Option.empty();
+    }
+
+    GenericRecord currentRecord = (GenericRecord) currentValue;
+    List<Schema.Field> fields = schema.getFields();
+    fields.forEach(field -> {
+      Object value = incomingRecord.get(field.name());
+      if (Objects.nonNull(value)) {
+        currentRecord.put(field.name(), value);
+      }
+    });
+
+    return Option.of(currentRecord);
+  }
+
+  @Override
+  public int compareTo(OverwriteWithLatestAvroPayload oldValue) {
+    return orderingVal.compareTo(oldValue.orderingVal);
+  }
+
+  @Override
+  public OverwriteWithLatestAvroPayload 
preCombine(OverwriteWithLatestAvroPayload oldValue) {

Review comment:
       Hi @nsivabalan , thanks a lot for review this.
   
   Regarding adding new `preCombine` method with `Schema`, I considered this, 
but it means that the method caller who need to get the schema info at first, 
and currently, it seems that we only can get the schema info from 
Configuration(from `hoodie.avro.schema` field). Sometimes, the caller might 
hard to get the schema info, Especially for 
`FlinkWriteHeler.deduplicateRecords(List<HoodieRecord<T>> records, 
HoodieIndex<?, ?> index, int parallelism)`.
   
   But compare the performance, it seems that passing the schema in method 
might be a better approach.
   BTW, since we already have had the method `preCombine(T oldValue, Properties 
properties)`, how about put the schema string in properties, and then parse the 
schema string to Schema later, so that we don't need to create a new method any 
more. otherwise, I cannot image when will we will `Properties`.




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