danny0405 commented on code in PR #9883:
URL: https://github.com/apache/hudi/pull/9883#discussion_r1375161773


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/merge/SparkRecordMergingUtils.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.merge;
+
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordMerger;
+import org.apache.hudi.common.model.HoodieSparkRecord;
+import org.apache.hudi.common.util.collection.Pair;
+
+import org.apache.avro.Schema;
+import org.apache.spark.sql.HoodieInternalRowUtils;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Util class to merge records that may contain partial updates.
+ * This can be plugged into any Spark {@link HoodieRecordMerger} 
implementation.
+ */
+public class SparkRecordMergingUtils {
+  private static final Map<Schema, Map<Integer, StructField>> 
FIELD_ID_TO_FIELD_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>,
+      Pair<Map<Integer, StructField>, Pair<StructType, Schema>>> 
MERGED_SCHEMA_CACHE = new ConcurrentHashMap<>();
+
+  /**
+   * Merges records which can contain partial updates.
+   *
+   * @param older        Older {@link HoodieSparkRecord}.
+   * @param oldSchema    Old schema.
+   * @param newer        Newer {@link HoodieSparkRecord}.
+   * @param newSchema    New schema.
+   * @param readerSchema Reader schema containing all the fields to read.
+   * @param props        Configuration in {@link TypedProperties}.
+   * @return The merged record.
+   */
+  public static Pair<HoodieRecord, Schema> 
mergeCompleteOrPartialRecords(HoodieSparkRecord older,
+                                                                         
Schema oldSchema,
+                                                                         
HoodieSparkRecord newer,
+                                                                         
Schema newSchema,
+                                                                         
Schema readerSchema,
+                                                                         
TypedProperties props) {
+    Pair<Map<Integer, StructField>, Pair<StructType, Schema>> mergedSchemaPair 
=
+        getCachedMergedSchema(oldSchema, newSchema, readerSchema);
+    boolean isNewerPartial = isPartial(newSchema, 
mergedSchemaPair.getRight().getRight());
+    if (isNewerPartial) {
+      InternalRow oldRow = older.getData();
+      InternalRow newPartialRow = newer.getData();
+
+      Map<Integer, StructField> mergedIdToFieldMapping = 
mergedSchemaPair.getLeft();
+      Map<String, Integer> newPartialNameToIdMapping = 
getCachedFieldNameToIdMapping(newSchema);
+      List<Object> values = new ArrayList<>(mergedIdToFieldMapping.size());
+      for (int fieldId = 0; fieldId < mergedIdToFieldMapping.size(); 
fieldId++) {
+        StructField structField = mergedIdToFieldMapping.get(fieldId);
+        Integer ordInPartialUpdate = 
newPartialNameToIdMapping.get(structField.name());
+        if (ordInPartialUpdate != null) {
+          // pick from new
+          values.add(newPartialRow.get(ordInPartialUpdate, 
structField.dataType()));
+        } else {
+          // pick from old
+          values.add(oldRow.get(fieldId, structField.dataType()));
+        }
+      }
+      InternalRow mergedRow = new GenericInternalRow(values.toArray());
+
+      HoodieSparkRecord mergedSparkRecord = new HoodieSparkRecord(
+          mergedRow, mergedSchemaPair.getRight().getLeft());
+      return Pair.of(mergedSparkRecord, 
mergedSchemaPair.getRight().getRight());
+    } else {
+      return Pair.of(newer, newSchema);
+    }
+  }
+
+  /**
+   * @param avroSchema Avro schema.
+   * @return The field ID to {@link StructField} instance mapping.
+   */
+  public static Map<Integer, StructField> 
getCachedFieldIdToFieldMapping(Schema avroSchema) {
+    return FIELD_ID_TO_FIELD_MAPPING_CACHE.computeIfAbsent(avroSchema, schema 
-> {
+      StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+      Map<Integer, StructField> schemaFieldIdMapping = new HashMap<>();
+      int fieldId = 0;
+
+      for (StructField field : structType.fields()) {
+        schemaFieldIdMapping.put(fieldId, field);
+        fieldId++;
+      }
+
+      return schemaFieldIdMapping;
+    });
+  }
+
+  /**
+   * @param avroSchema Avro schema.
+   * @return The field name to ID mapping.
+   */
+  public static Map<String, Integer> getCachedFieldNameToIdMapping(Schema 
avroSchema) {
+    return FIELD_NAME_TO_ID_MAPPING_CACHE.computeIfAbsent(avroSchema, schema 
-> {
+      StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+      Map<String, Integer> schemaFieldIdMapping = new HashMap<>();
+      int fieldId = 0;
+
+      for (StructField field : structType.fields()) {
+        schemaFieldIdMapping.put(field.name(), fieldId);
+        fieldId++;
+      }
+
+      return schemaFieldIdMapping;
+    });
+  }
+
+  /**
+   * Merges the two schemas so the merged schema contains all the fields from 
the two schemas,
+   * with the same ordering of fields, and the fields of the old schema comes 
first.
+   *
+   * @param oldSchema Old schema.
+   * @param newSchema New schema.
+   * @return The ID to {@link StructField} instance mapping of the merged 
schema, and the
+   * {@link StructType} and Avro schema of the merged schema.
+   */
+  public static Pair<Map<Integer, StructField>, Pair<StructType, Schema>> 
getCachedMergedSchema(Schema oldSchema,
+                                                                               
                 Schema newSchema,
+                                                                               
                 Schema readerSchema) {
+    return MERGED_SCHEMA_CACHE.computeIfAbsent(
+        Pair.of(Pair.of(oldSchema, newSchema), readerSchema), schemaPair -> {
+          Schema schema1 = schemaPair.getLeft().getLeft();
+          Schema schema2 = schemaPair.getLeft().getRight();
+          Schema refSchema = schemaPair.getRight();
+          Map<String, Integer> nameToIdMapping1 = 
getCachedFieldNameToIdMapping(schema1);
+          Map<String, Integer> nameToIdMapping2 = 
getCachedFieldNameToIdMapping(schema2);
+          Map<Integer, StructField> refFieldIdToFieldMapping = 
getCachedFieldIdToFieldMapping(refSchema);
+          Set<String> fieldNameSet = new HashSet<>();
+          fieldNameSet.addAll(nameToIdMapping1.keySet());
+          fieldNameSet.addAll(nameToIdMapping2.keySet());
+          int fieldId = 0;
+          Map<Integer, StructField> mergedMapping = new HashMap<>();
+          List<StructField> mergedFieldList = new ArrayList<>();
+          for (int i = 0; i < refFieldIdToFieldMapping.size(); i++) {
+            StructField field = refFieldIdToFieldMapping.get(i);

Review Comment:
   `refFieldIdToFieldMapping` is a hash map, it does not guarantee the field 
sequence as what user decpares.



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to