linliu-code commented on code in PR #13623:
URL: https://github.com/apache/hudi/pull/13623#discussion_r2230963336


##########
hudi-common/src/main/java/org/apache/hudi/common/table/read/KeepValuesPartialMergingUtils.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.table.read;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.util.collection.Pair;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Generic util class to merge records of type T that may contain partial 
updates.
+ */
+public class KeepValuesPartialMergingUtils<T> {
+  private static final Map<Schema, Map<Integer, String>>
+      FIELD_ID_TO_NAME_MAPPING_CACHE = new ConcurrentHashMap<>();
+  private static final Map<Schema, Map<String, Integer>>
+      FIELD_NAME_TO_ID_MAPPING_CACHE = new ConcurrentHashMap<>();
+  private static final Map<Pair<Pair<Schema, Schema>, Schema>, Schema>
+      MERGED_SCHEMA_CACHE = new ConcurrentHashMap<>();
+
+  /**
+   * Merges records which can contain partial updates.
+   *
+   * @param older         Older record of type {@BufferedRecord<T>}.
+   * @param oldSchema     Schema of the older record.
+   * @param newer         Newer record of type {@BufferedRecord<T>}.
+   * @param newSchema     Schema of the newer record.
+   * @param readerSchema  Reader schema containing all the fields to read. 
This is used to maintain
+   *                      the ordering of the fields of the merged record.
+   * @param readerContext ReaderContext instance.
+   * @return The merged record and schema.
+   */
+  public Pair<BufferedRecord<T>, Schema> mergePartialRecords(BufferedRecord<T> 
older,
+                                                             Schema oldSchema,
+                                                             BufferedRecord<T> 
newer,
+                                                             Schema newSchema,
+                                                             Schema 
readerSchema,
+                                                             
HoodieReaderContext<T> readerContext) {
+    // The merged schema contains fields that only appear in either older 
and/or newer record.
+    Schema mergedSchema =
+        getCachedMergedSchema(oldSchema, newSchema, readerSchema);
+    boolean isNewerPartial = isPartial(newSchema, mergedSchema);
+    if (!isNewerPartial) {
+      return Pair.of(newer, newSchema);
+    }
+    Set<String> fieldNamesInNewRecord =
+        getCachedFieldNameToIdMapping(newSchema).keySet();
+    // Collect field values.
+    List<Object> values = new ArrayList<>();
+    List<Schema.Field> mergedSchemaFields = mergedSchema.getFields();
+    for (int index = 0; index < mergedSchemaFields.size(); index++) {
+      String fieldName = mergedSchemaFields.get(index).name();
+      if (fieldNamesInNewRecord.contains(fieldName)) { // field present in 
newer record.
+        values.add(readerContext.getValue(newer.getRecord(), newSchema, 
fieldName));
+      } else { // if not present in newer record pick from old record
+        values.add(readerContext.getValue(older.getRecord(), oldSchema, 
fieldName));
+      }
+    }
+    // Build merged record.
+    T engineRecord = readerContext.createEngineRecord(mergedSchema, values);
+    BufferedRecord<T> mergedRecord = new BufferedRecord<>(
+        newer.getRecordKey(),
+        newer.getOrderingValue(),
+        engineRecord,
+        readerContext.encodeAvroSchema(mergedSchema),
+        newer.isDelete());
+    return Pair.of(mergedRecord, mergedSchema);
+  }
+
+  /**
+   * @param avroSchema Avro schema.
+   * @return The field ID to field name mapping.
+   */
+  public static Map<Integer, String> getCachedFieldIdToNameMapping(Schema 
avroSchema) {

Review Comment:
   This function is not used anymore, can be removed.  I remember I have 
removed it. Maybe I haven't pushed the latest one.



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