nsivabalan commented on code in PR #8490:
URL: https://github.com/apache/hudi/pull/8490#discussion_r1184683949
##########
hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java:
##########
@@ -143,6 +144,31 @@ public static <R> HoodieRecord<R>
convertToHoodieRecordPayload(GenericRecord rec
return (HoodieRecord<R>) hoodieRecord;
}
+ public static <R> HoodieRecord<R>
convertToHoodieRecordPayload2(GenericRecord record, String payloadClazz,
Review Comment:
`convertToHoodieRecordPayload2` needs to be fixed for naming
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java:
##########
@@ -175,4 +188,92 @@ public static boolean checkIfValidCommit(HoodieTimeline
commitTimeline, String c
// 2) is less than the first commit ts in the timeline
return !commitTimeline.empty() &&
commitTimeline.containsOrBeforeTimelineStarts(commitTs);
}
+
+ /**
+ * Read existing records based on the given partition path and {@link
HoodieRecordLocation} info.
+ * <p>
+ * This will perform merged read for MOR table, in case a FileGroup contains
log files.
+ *
+ * @return {@link HoodieRecord}s that have the current location being set.
+ */
+ private static <R> HoodieData<HoodieRecord<R>> getExistingRecords(
+ HoodieData<Pair<String, HoodieRecordLocation>> partitionLocations,
HoodieWriteConfig config, HoodieTable hoodieTable) {
+ final Option<String> instantTime = hoodieTable
+ .getMetaClient()
+ .getCommitsTimeline()
+ .filterCompletedInstants()
+ .lastInstant()
+ .map(HoodieInstant::getTimestamp);
+ return partitionLocations.flatMap(p -> {
+ String partitionPath = p.getLeft();
+ String fileId = p.getRight().getFileId();
+ return new HoodieMergedReadHandle(config, instantTime, hoodieTable,
Pair.of(partitionPath, fileId))
+ .getMergedRecords().iterator();
+ });
+ }
+
+ /**
+ * Merge tagged incoming records with existing records in case of partition
path updated.
+ */
+ public static <R> HoodieData<HoodieRecord<R>> mergeForPartitionUpdates(
+ HoodieData<Pair<HoodieRecord<R>, Option<Pair<String,
HoodieRecordLocation>>>> taggedHoodieRecords, HoodieWriteConfig config,
HoodieTable hoodieTable) {
+ // completely new records
+ HoodieData<HoodieRecord<R>> newRecords = taggedHoodieRecords.filter(p ->
!p.getRight().isPresent()).map(Pair::getLeft);
+ // the records tagged to existing base files
+ HoodieData<HoodieRecord<R>> updatingRecords = taggedHoodieRecords.filter(p
-> p.getRight().isPresent()).map(Pair::getLeft)
+ .distinctWithKey(HoodieRecord::getRecordKey,
config.getGlobalIndexReconcileParallelism());
+ // the tagging partitions and locations
+ HoodieData<Pair<String, HoodieRecordLocation>> partitionLocations =
taggedHoodieRecords
+ .filter(p -> p.getRight().isPresent())
+ .map(p -> p.getRight().get())
+ .distinct(config.getGlobalIndexReconcileParallelism());
+ // merged existing records with current locations being set
+ HoodieData<HoodieRecord<R>> existingRecords =
getExistingRecords(partitionLocations, config, hoodieTable);
+
+ TypedProperties updatedProps =
HoodieAvroRecordMerger.Config.withLegacyOperatingModePreCombining(config.getProps());
+ HoodieData<HoodieRecord<R>> taggedUpdatingRecords =
updatingRecords.mapToPair(r -> Pair.of(r.getRecordKey(), r))
+ .leftOuterJoin(existingRecords.mapToPair(r ->
Pair.of(r.getRecordKey(), r)))
+ .values().flatMap(entry -> {
+ HoodieRecord<R> incoming = entry.getLeft();
+ Option<HoodieRecord<R>> existingOpt = entry.getRight();
+ if (!existingOpt.isPresent()) {
+ // existing record not found (e.g., due to delete log not merged
to base file): tag as a new record
+ return Collections.singletonList(getTaggedRecord(incoming,
Option.empty())).iterator();
+ }
+ HoodieRecord<R> existing = existingOpt.get();
+ if (incoming.getData() instanceof EmptyHoodieRecordPayload) {
+ // incoming is a delete: force tag the incoming to the old
partition
+ return Collections.singletonList(getTaggedRecord(incoming,
Option.of(existing.getCurrentLocation()))).iterator();
+ }
+ Schema existingSchema = HoodieAvroUtils.addMetadataFields(new
Schema.Parser().parse(config.getSchema()),
config.allowOperationMetadataField());
+ Schema writeSchema = new
Schema.Parser().parse(config.getWriteSchema());
+ Schema writeSchemaWithMetaFields =
HoodieAvroUtils.addMetadataFields(writeSchema,
config.allowOperationMetadataField());
+ HoodieRecord incomingPrepended = incoming
+ .prependMetaFields(writeSchema, writeSchemaWithMetaFields, new
MetadataValues().setRecordKey(incoming.getRecordKey()).setPartitionPath(incoming.getPartitionPath()),
config.getProps());
+ HoodieRecord incomingWithMetaFields = incomingPrepended
+ .wrapIntoHoodieRecordPayloadWithParams(writeSchema,
config.getProps(), Option.empty(), config.allowOperationMetadataField(),
Option.empty(), false);
+ Option<Pair<HoodieRecord, Schema>> mergeResult =
config.getRecordMerger()
+ .merge(existing, existingSchema, incomingWithMetaFields,
writeSchemaWithMetaFields, config.getProps());
+ if (!mergeResult.isPresent()) {
+ // merge resulted in delete: force tag the incoming to the old
partition
+ return Collections.singletonList(getTaggedRecord(incoming,
Option.of(existing.getCurrentLocation()))).iterator();
+ }
+ HoodieRecord mergedRaw = mergeResult.get().getLeft();
+ HoodieRecord<R> merged =
((HoodieAvroIndexedRecord)mergedRaw).wrapIntoHoodieRecordPayloadWithoutMetaFields(
Review Comment:
lets add a java doc on the purpose.
##########
hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java:
##########
@@ -143,6 +144,31 @@ public static <R> HoodieRecord<R>
convertToHoodieRecordPayload(GenericRecord rec
return (HoodieRecord<R>) hoodieRecord;
}
+ public static <R> HoodieRecord<R>
convertToHoodieRecordPayload2(GenericRecord record, String payloadClazz,
Review Comment:
add java docs on the purpose of the method.
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergedReadHandle.java:
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.io;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.HoodieRecordMerger;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.ClosableIterator;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.avro.Schema;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.hudi.common.util.StringUtils.nonEmpty;
+import static org.apache.hudi.common.util.ValidationUtils.checkState;
+
+public class HoodieMergedReadHandle<T, I, K, O> extends HoodieReadHandle<T, I,
K, O> {
Review Comment:
we need tests for this .
--
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]