yihua commented on code in PR #9883: URL: https://github.com/apache/hudi/pull/9883#discussion_r1373980774
########## hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/merge/SparkRecordMergingUtils.java: ########## @@ -0,0 +1,192 @@ +/* + * 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.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +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<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 props Configuration in {@link TypedProperties}. + * @return The merged record. + */ + public static Pair<HoodieRecord, Schema> mergeCompleteOrPartialRecords(HoodieSparkRecord older, + Schema oldSchema, + HoodieSparkRecord newer, + Schema newSchema, + TypedProperties props) { + Pair<Map<Integer, StructField>, Pair<StructType, Schema>> mappingSchemaPair = + getCachedMergedSchema(oldSchema, newSchema); + boolean isNewerPartial = isPartial(newSchema, mappingSchemaPair.getRight().getRight()); + if (isNewerPartial) { + InternalRow oldRow = older.getData(); + InternalRow newPartialRow = newer.getData(); + + Map<Integer, StructField> mergedIdToFieldMapping = mappingSchemaPair.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); + if (newPartialNameToIdMapping.containsKey(structField.name())) { + // pick from new + int ordInPartialUpdate = newPartialNameToIdMapping.get(structField.name()); + values.add(newPartialRow.get(ordInPartialUpdate, structField.dataType())); + } else { + // pick from old + values.add(oldRow.get(fieldId, structField.dataType())); Review Comment: Here `new` updates `old` after determining the ordering, either based on commit, event time, or custom logic. Both old and new schema can be partial, so there can be intermediate partial updates. For example, consider the following case: ``` base file: schema {col1, col2, col3, col4} log1: partial update schema {col2} log2: partial update schema {col4} ``` When merging partial update records between log1 (`{col2}`) and log2 (`{col4}`), the log reader generates partial updates again with schema `{col2, col4}` for some records. Finally there's record merging between base (`{col1, col2, col3, col4}`) and logs (`{col2, col4}`). -- 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]
