xushiyan commented on code in PR #8490:
URL: https://github.com/apache/hudi/pull/8490#discussion_r1196153851


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergedReadHandle.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.io;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieLogFile;
+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> {
+
+  protected final Schema readerSchema;
+
+  public HoodieMergedReadHandle(HoodieWriteConfig config,
+                                Option<String> instantTime,
+                                HoodieTable<T, I, K, O> hoodieTable,
+                                Pair<String, String> partitionPathFileIDPair) {
+    super(config, instantTime, hoodieTable, partitionPathFileIDPair);
+    readerSchema = HoodieAvroUtils.addMetadataFields(new 
Schema.Parser().parse(config.getSchema()), 
config.allowOperationMetadataField());
+  }
+
+  public List<HoodieRecord<T>> getMergedRecords() {
+    Option<FileSlice> fileSliceOpt = getLatestFileSlice();
+    if (!fileSliceOpt.isPresent()) {
+      return Collections.emptyList();
+    }
+    checkState(nonEmpty(instantTime), String.format("Expected a valid instant 
time but got `%s`", instantTime));
+    final FileSlice fileSlice = fileSliceOpt.get();
+    final HoodieRecordLocation currentLocation = new 
HoodieRecordLocation(instantTime, fileSlice.getFileId());
+    Option<HoodieFileReader> baseFileReader = Option.empty();
+    HoodieMergedLogRecordScanner logRecordScanner = null;
+    try {
+      baseFileReader = getBaseFileReader(fileSlice);
+      logRecordScanner = getLogRecordScanner(fileSlice);
+      List<HoodieRecord<T>> mergedRecords = new ArrayList<>();
+      doMergedRead(baseFileReader, logRecordScanner).forEach(r -> {
+        r.unseal();
+        r.setCurrentLocation(currentLocation);
+        r.seal();
+        mergedRecords.add(r);
+      });
+      return mergedRecords;
+    } catch (IOException e) {
+      throw new HoodieIndexException("Error in reading " + fileSlice, e);
+    } finally {
+      if (baseFileReader.isPresent()) {
+        baseFileReader.get().close();
+      }
+      if (logRecordScanner != null) {
+        logRecordScanner.close();
+      }
+    }
+  }
+
+  private Option<FileSlice> getLatestFileSlice() {
+    if (nonEmpty(instantTime)
+        && 
hoodieTable.getMetaClient().getCommitsTimeline().filterCompletedInstants().lastInstant().isPresent())
 {
+      return Option.fromJavaOptional(hoodieTable
+          .getHoodieView()
+          
.getLatestMergedFileSlicesBeforeOrOn(partitionPathFileIDPair.getLeft(), 
instantTime)
+          .filter(fileSlice -> 
fileSlice.getFileId().equals(partitionPathFileIDPair.getRight()))
+          .findFirst());
+    }
+    return Option.empty();
+  }
+
+  private Option<HoodieFileReader> getBaseFileReader(FileSlice fileSlice) 
throws IOException {
+    if (fileSlice.getBaseFile().isPresent()) {
+      return Option.of(createNewFileReader(fileSlice.getBaseFile().get()));
+    }
+    return Option.empty();
+  }
+
+  private HoodieMergedLogRecordScanner getLogRecordScanner(FileSlice 
fileSlice) {
+    List<String> logFilePaths = 
fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
+        .map(l -> l.getPath().toString()).collect(toList());
+    return HoodieMergedLogRecordScanner.newBuilder()
+        .withFileSystem(hoodieTable.getMetaClient().getFs())
+        .withBasePath(hoodieTable.getMetaClient().getBasePathV2().toString())
+        .withLogFilePaths(logFilePaths)
+        .withReaderSchema(readerSchema)
+        .withLatestInstantTime(instantTime)
+        
.withMaxMemorySizeInBytes(IOUtils.getMaxMemoryPerCompaction(hoodieTable.getTaskContextSupplier(),
 config))
+        .withReadBlocksLazily(config.getCompactionLazyBlockReadEnabled())
+        .withReverseReader(config.getCompactionReverseLogReadEnabled())
+        .withBufferSize(config.getMaxDFSStreamBufferSize())
+        .withSpillableMapBasePath(config.getSpillableMapBasePath())
+        .withPartition(fileSlice.getPartitionPath())
+        .withOptimizedLogBlocksScan(config.enableOptimizedLogBlocksScan())
+        .withDiskMapType(config.getCommonConfig().getSpillableDiskMapType())
+        
.withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
+        .withRecordMerger(config.getRecordMerger())
+        .build();
+  }
+
+  private List<HoodieRecord<T>> doMergedRead(Option<HoodieFileReader> 
baseFileReaderOpt, HoodieMergedLogRecordScanner logRecordScanner) throws 
IOException {
+    List<HoodieRecord<T>> mergedRecords = new ArrayList<>();
+    Map<String, HoodieRecord> deltaRecordMap = logRecordScanner.getRecords();
+    Set<String> deltaRecordKeys = new HashSet<>(deltaRecordMap.keySet());
+
+    if (baseFileReaderOpt.isPresent()) {
+      HoodieFileReader baseFileReader = baseFileReaderOpt.get();
+      HoodieRecordMerger recordMerger = config.getRecordMerger();
+      ClosableIterator<HoodieRecord<T>> baseFileItr = 
baseFileReader.getRecordIterator(readerSchema);
+      HoodieTableConfig tableConfig = 
hoodieTable.getMetaClient().getTableConfig();
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt =
+          tableConfig.populateMetaFields() ? Option.empty() : 
Option.of(Pair.of(tableConfig.getRecordKeyFieldProp(), 
tableConfig.getPartitionFieldProp()));
+      while (baseFileItr.hasNext()) {
+        HoodieRecord<T> record = 
baseFileItr.next().wrapIntoHoodieRecordPayloadWithParams(readerSchema,
+            config.getProps(), simpleKeyGenFieldsOpt, 
logRecordScanner.isWithOperationField(), 
logRecordScanner.getPartitionNameOverride(), false, Option.empty());
+        String key = record.getRecordKey();
+        if (deltaRecordMap.containsKey(key)) {
+          deltaRecordKeys.remove(key);
+          Option<Pair<HoodieRecord, Schema>> mergeResult = recordMerger

Review Comment:
   this merge result needs to be wrapped back to the original payload so that 
caller won't have to do it. fixed in https://github.com/apache/hudi/pull/8736



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java:
##########
@@ -175,4 +186,106 @@ 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 the incoming record with the matching existing record loaded via 
{@link HoodieMergedReadHandle}. The existing record is the latest version in 
the table.
+   */
+  private static <R> Option<HoodieRecord<R>> 
mergeIncomingWithExistingRecord(HoodieRecord<R> incoming, HoodieRecord<R> 
existing, HoodieWriteConfig config) throws IOException {
+    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());
+    // prepend the hoodie meta fields as the incoming record does not have them
+    HoodieRecord incomingPrepended = incoming
+        .prependMetaFields(writeSchema, writeSchemaWithMetaFields, new 
MetadataValues().setRecordKey(incoming.getRecordKey()).setPartitionPath(incoming.getPartitionPath()),
 config.getProps());
+    // after prepend the meta fields, convert the record back to the original 
payload
+    HoodieRecord incomingWithMetaFields = incomingPrepended
+        .wrapIntoHoodieRecordPayloadWithParams(writeSchema, config.getProps(), 
Option.empty(), config.allowOperationMetadataField(), Option.empty(), false, 
Option.empty());
+    Option<Pair<HoodieRecord, Schema>> mergeResult = config.getRecordMerger()
+        .merge(existing, existingSchema, incomingWithMetaFields, 
writeSchemaWithMetaFields, config.getProps());
+    if (mergeResult.isPresent()) {
+      // the merged record needs to be converted back to the original payload
+      HoodieRecord<R> merged = 
mergeResult.get().getLeft().wrapIntoHoodieRecordPayloadWithParams(
+          writeSchemaWithMetaFields, config.getProps(), Option.empty(),
+          config.allowOperationMetadataField(), Option.empty(), false, 
Option.of(writeSchema));
+      return Option.of(merged);
+    } else {
+      return Option.empty();
+    }
+  }
+
+  /**
+   * 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);
+
+    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();

Review Comment:
   this needs to use `isDelete()` api to check and `incoming`'s key need to be 
overwritten to the existing's key. fixed in 
https://github.com/apache/hudi/pull/8736



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