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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java:
##########
@@ -175,4 +186,72 @@ 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);
   }
+
+  public static <R> HoodieData<HoodieRecord<R>> 
getTaggedRecordsFromPartitionLocations(
+      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();
+    });
+  }
+
+  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());

Review Comment:
   I see we are doing distinctWithKey here. So, we assume that records may not 
be duplicated at all?
   what happens if there are duplicates already. for eg, some one ingested same 
batch of data w/ bulk_insert. may be we need to revisit overall end to end flow 
for this scenario of how our global index will work. 
   but trying to think through how it might surface after this fix? 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java:
##########
@@ -175,4 +186,72 @@ 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);
   }
+
+  public static <R> HoodieData<HoodieRecord<R>> 
getTaggedRecordsFromPartitionLocations(
+      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();
+    });
+  }
+
+  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)

Review Comment:
   I see we process taggedHoodieRecords.filter(p -> p.getRight().isPresent()) 
multiple times. should we take it out and cache it? 



##########
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:
   Do we have UTs for this class? 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java:
##########
@@ -175,4 +186,72 @@ 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);
   }
+
+  public static <R> HoodieData<HoodieRecord<R>> 
getTaggedRecordsFromPartitionLocations(
+      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();
+    });
+  }
+
+  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 = 
getTaggedRecordsFromPartitionLocations(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());

Review Comment:
   can we move L236 to 252 to a private method. helps keeping this method lean



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java:
##########
@@ -110,33 +110,31 @@ protected <R> HoodieData<HoodieRecord<R>> 
tagLocationBacktoRecords(
         keyLocationPairs.mapToPair(p -> new ImmutablePair<>(
             p.getKey().getRecordKey(), new ImmutablePair<>(p.getValue(), 
p.getKey())));
 
+    // Pair of a tagged record and if the record needs dedup

Review Comment:
   is this valid comment w/ latest code ? 



##########
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> {
+
+  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())
 {

Review Comment:
   can we move these checks to constructor. 



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceStorage.scala:
##########
@@ -133,4 +132,69 @@ class TestMORDataSourceStorage extends 
SparkClientFunctionalTestHarness {
     assertEquals(100, hudiSnapshotDF3.count())
     assertEquals(updatedVerificationVal, 
hudiSnapshotDF3.filter(col("_row_key") === 
verificationRowKey).select(verificationCol).first.getString(0))
   }
+
+  @ParameterizedTest
+  @CsvSource(Array(
+    "true,GLOBAL_SIMPLE",
+    "true,GLOBAL_BLOOM",
+    "false,GLOBAL_SIMPLE",
+    "false,GLOBAL_BLOOM"
+  ))
+  def testMergeOnReadStorageWithGlobalIndexUpdatePartition(isMetadataEnabled: 
Boolean, indexType: String): Unit = {
+    val totalRecords = 10
+    val parallelism = 2
+    val commonOpts = Map(
+      HoodieTableConfig.NAME.key -> "hudi_trips_mor_global_update_partition",
+      DataSourceWriteOptions.TABLE_TYPE.key -> 
HoodieTableType.MERGE_ON_READ.name,
+      DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
+      DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition_path",
+      DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
+      HoodieMetadataConfig.ENABLE.key -> isMetadataEnabled.toString,
+      HoodieIndexConfig.INDEX_TYPE.key -> indexType,
+      HoodieIndexConfig.SIMPLE_INDEX_UPDATE_PARTITION_PATH_ENABLE.key -> 
"true",
+      HoodieIndexConfig.GLOBAL_INDEX_RECONCILE_PARALLELISM.key -> 
parallelism.toString,
+      HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> parallelism.toString,
+      HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> parallelism.toString,
+      HoodieWriteConfig.BULKINSERT_PARALLELISM_VALUE.key -> 
parallelism.toString,
+      HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> parallelism.toString
+    )
+    val dataGen = new HoodieTestDataGenerator(0xDEEF)
+
+    def upsertAndValidate(records: java.util.List[HoodieRecord[_]], partition: 
String): Unit = {
+      // upsert records
+      
spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records), 
parallelism))
+        .write.format("hudi")
+        .options(commonOpts)
+        .mode(SaveMode.Append)
+        .save(basePath)
+
+      // validate all records are in the partition
+      val snapshotDF = spark.read.format("org.apache.hudi")
+        .option(DataSourceReadOptions.QUERY_TYPE.key, 
DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
+        .option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
+        .load(basePath)
+      assertEquals(totalRecords, snapshotDF.count)
+      assertEquals(totalRecords, snapshotDF.filter(s"partition_path = 
'$partition'").count)

Review Comment:
   can we compare the actual records match? and not just count. 
   for eg, the merge logic holds intact. 
   lets use DefaultHoodieRecordPayload and ingest out of order records and 
verify that new ones are ignored for out of order ones. 
   



##########
hudi-common/src/test/java/org/apache/hudi/common/testutils/RawTripTestPayload.java:
##########
@@ -86,7 +87,7 @@ public RawTripTestPayload(String jsonData) throws IOException 
{
     this.rowKey = jsonRecordMap.get("_row_key").toString();
     this.partitionPath = 
extractPartitionFromTimeField(jsonRecordMap.get("time").toString());
     this.isDeleted = false;
-    this.orderingVal = Integer.valueOf(jsonRecordMap.get("number").toString());
+    this.orderingVal = Integer.valueOf(jsonRecordMap.getOrDefault("number", 
0L).toString());

Review Comment:
   lets create a follow up ticket and dump your thoughts on what needs to be 
fixed here. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java:
##########
@@ -175,4 +186,68 @@ 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);
   }
+
+  public static <R> HoodieData<HoodieRecord<R>> 
getTaggedRecordsFromPartitionLocations(

Review Comment:
   synced up f2f. this is ok



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java:
##########
@@ -175,4 +186,72 @@ 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);
   }
+
+  public static <R> HoodieData<HoodieRecord<R>> 
getTaggedRecordsFromPartitionLocations(
+      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();
+    });
+  }
+
+  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());

Review Comment:
   we may not require to fix anything as such. but wanted to see what will be 
outcome. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java:
##########
@@ -175,4 +186,72 @@ 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);
   }
+
+  public static <R> HoodieData<HoodieRecord<R>> 
getTaggedRecordsFromPartitionLocations(

Review Comment:
   may be we can rename this method to 
   getExistingRecordLocations()



##########
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> {
+
+  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().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);
+        String key = record.getRecordKey();
+        if (deltaRecordMap.containsKey(key)) {

Review Comment:
   I see this is pretty much what our MOR snapshot is doing. is there a way to 
re-use code? 
   what incase we make some fixes to MOR snapshot, but missed to fix this code? 
   



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