fengjian428 commented on code in PR #4676:
URL: https://github.com/apache/hudi/pull/4676#discussion_r970357842


##########
hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateAvroPayload.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * subclass of OverwriteNonDefaultsWithLatestAvroPayload used for delta 
streamer.
+ *
+ * <ol>
+ * <li>preCombine - Picks the latest delta record for a key, based on an 
ordering field;
+ * <li>combineAndGetUpdateValue/getInsertValue - overwrite storage for 
specified fields
+ * that doesn't equal defaultValue.
+ * </ol>
+ */
+public class PartialUpdateAvroPayload extends 
OverwriteNonDefaultsWithLatestAvroPayload {
+
+  public PartialUpdateAvroPayload(GenericRecord record, Comparable 
orderingVal) {
+    super(record, orderingVal);
+  }
+
+  public PartialUpdateAvroPayload(Option<GenericRecord> record) {
+    super(record); // natural order
+  }
+
+  @Override
+  public PartialUpdateAvroPayload preCombine(OverwriteWithLatestAvroPayload 
oldValue, Properties properties, Schema schema) {
+    if (oldValue.recordBytes.length == 0) {
+      // use natural order for delete record
+      return this;
+    }
+    boolean isBaseRecordForMerge = false;
+    if (oldValue.orderingVal.compareTo(orderingVal) > 0) {
+      // pick the payload with greatest ordering value as insert record
+      isBaseRecordForMerge = true;
+    }
+    try {
+      GenericRecord indexedOldValue = (GenericRecord) 
oldValue.getInsertValue(schema).get();
+      Option<IndexedRecord> optValue = 
combineAndGetUpdateValue(indexedOldValue, schema, isBaseRecordForMerge);
+      if (optValue.isPresent()) {
+        return new PartialUpdateAvroPayload((GenericRecord) optValue.get(),
+            isBaseRecordForMerge ? oldValue.orderingVal : this.orderingVal);
+      }
+    } catch (Exception ex) {
+      return this;
+    }
+    return this;
+  }
+
+  public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord 
currentValue, Schema schema, boolean isBaseRecordForMerge) throws IOException {

Review Comment:
   In this Payload, if InsertCurrentValue is true means overwriting non-default 
values in the incoming value with the current value, otherwise overwriting 
non-default values in the current value with the incoming value
   should I use shouldInsertCurrentValueOverwriteIncoming to make it clear?



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