yihua commented on code in PR #13830:
URL: https://github.com/apache/hudi/pull/13830#discussion_r2320516403
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java:
##########
@@ -636,7 +638,7 @@ public static <R> HoodieData<HoodieRecord<R>>
tagGlobalLocationBackToRecords(
return Pair.of(incomingRecord, Option.empty());
}
});
- return shouldUpdatePartitionPath
+ return shouldUpdatePartitionPath || isMoRTable
Review Comment:
So could the `COMMIT_TIME_ORDERING` also be excluded from using
`mergeForPartitionUpdatesIfNeeded`? Reading existing records from storage and
merge with the incoming records adds additional overhead so see if it is
possible to exclude cases that do not need the merging.
Also wondering if there is easy way to check there is non-empty
`Option<HoodieRecordGlobalLocation>` in `incomingRecordsAndLocations`; then
trigger `mergeForPartitionUpdatesIfNeeded` if so.
In the future, once we store ordering value in RLI, for
`EVENT_TIME_ORDERING`, the `mergeForPartitionUpdatesIfNeeded` can be skipped
and comparison of the ordering value of incoming deletes and that of RLI entry
would determine if the RLI entry should be deleted.
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionedRecordLevelIndex.scala:
##########
@@ -444,6 +447,38 @@ class TestPartitionedRecordLevelIndex extends
RecordLevelIndexTestBase {
metadata.readRecordIndexLocationsWithKeys(HoodieListData.eager(recordKeys),
dataTablePartition)
.collectAsList().asScala.map(p => p.getKey -> p.getValue).toMap
}
+
+ @ParameterizedTest
+ @EnumSource(classOf[HoodieTableType])
+ def testRLIForDeletesWithHoodieIsDeletedColumn(tableType: HoodieTableType):
Unit = {
+ val hudiOpts = commonOpts + (DataSourceWriteOptions.TABLE_TYPE.key ->
tableType.name()) +
+ (HoodieIndexConfig.INDEX_TYPE.key -> "RECORD_INDEX") +
+ (HoodieIndexConfig.RECORD_INDEX_UPDATE_PARTITION_PATH_ENABLE.key ->
"false")
+ val insertDf = doWriteAndValidateDataAndRecordIndex(hudiOpts,
+ operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
+ saveMode = SaveMode.Overwrite)
+ insertDf.cache()
+
+ val instantTime = InProcessTimeGenerator.createNewInstantTime()
+ // Issue four deletes, one with the original partition, one with an
updated partition,
+ // and two with an older ordering value that should be ignored
+ val deletedRecords = dataGen.generateUniqueDeleteRecords(instantTime, 1)
+ val inputRecords = new util.ArrayList[HoodieRecord[_]](deletedRecords)
+ val lowerOrderingValue = 1L
+ inputRecords.addAll(dataGen.generateUniqueDeleteRecords(instantTime, 1,
lowerOrderingValue))
+ val deleteBatch = recordsToStrings(inputRecords).asScala
+ val deleteDf =
spark.read.json(spark.sparkContext.parallelize(deleteBatch.toSeq, 1))
+ deleteDf.cache()
+ val recordKeyToDelete =
deleteDf.collectAsList().get(0).getAs("_row_key").asInstanceOf[String]
+ deleteDf.write.format("org.apache.hudi")
Review Comment:
```suggestion
deleteDf.write.format("hudi")
```
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionedRecordLevelIndex.scala:
##########
@@ -444,6 +447,38 @@ class TestPartitionedRecordLevelIndex extends
RecordLevelIndexTestBase {
metadata.readRecordIndexLocationsWithKeys(HoodieListData.eager(recordKeys),
dataTablePartition)
.collectAsList().asScala.map(p => p.getKey -> p.getValue).toMap
}
+
+ @ParameterizedTest
+ @EnumSource(classOf[HoodieTableType])
+ def testRLIForDeletesWithHoodieIsDeletedColumn(tableType: HoodieTableType):
Unit = {
Review Comment:
Should we also add test variant(s) on the merge mode of
`COMMIT_TIME_ORDERING`?
--
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]