rohan-uptycs commented on code in PR #8503:
URL: https://github.com/apache/hudi/pull/8503#discussion_r1182163517


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java:
##########
@@ -275,4 +280,49 @@ public Option<HoodieRecordLocation> 
getRecordLocation(HoodieKey key) {
       throw new HoodieIndexException("Failed to getBucket as hashing node has 
no file group");
     }
   }
+
+  /**
+   * Update default metadata file(00000000000000.hashing_meta) with the latest 
committed metadata file so that default file will be in sync
+   * with latest commit.
+   *
+   * @param table
+   */
+  public void updateArchivalDependentIndexMetadata(HoodieTable 
table,List<HoodieInstant> hoodieArchivalInstants) {
+    Map<String, Boolean> partitionVisiteddMap = new HashMap<>();
+    // Update metadata for replace commit which are going to get archived.
+    Stream<HoodieInstant> hoodieListOfReplacedInstants = 
hoodieArchivalInstants.stream().filter(instane -> 
instane.getAction().equals(REPLACE_COMMIT_ACTION));
+    hoodieListOfReplacedInstants.forEach(instant -> {
+      Option<Pair<HoodieInstant, HoodieClusteringPlan>> instantPlanPair =
+          ClusteringUtils.getClusteringPlan(table.getMetaClient(), 
HoodieTimeline.getReplaceCommitRequestedInstant(instant.getTimestamp()));
+      if (instantPlanPair.isPresent()) {
+        HoodieClusteringPlan plan = instantPlanPair.get().getRight();
+        List<Map<String, String>> partitionMapList = 
plan.getInputGroups().stream().map(HoodieClusteringGroup::getExtraMetadata).collect(Collectors.toList());
+        partitionMapList.stream().forEach(partitionMap -> {
+          String partition = 
partitionMap.get(SparkConsistentBucketClusteringPlanStrategy.METADATA_PARTITION_KEY);
+          if (!partitionVisiteddMap.containsKey(partition)) {
+            Option<HoodieConsistentHashingMetadata> 
hoodieConsistentHashingMetadataOption = loadMetadata(table, partition);
+            if (hoodieConsistentHashingMetadataOption.isPresent()) {

Review Comment:
   let me give few examples which can cause inconsistency
   Example 1
   lets say we are updating metadat before clustering commit
   1 - written the data files on storage
   2 - update metatda
   3 - commited clustering operation on hudi timeline by creating replace commit
   
   now let's say **2nd operation is done but 3rd operation fails** in this case 
**metadata got synced but clustering is failed. all the subsequent write 
operation will read from failed clustering metadata synced file.** 
   
   Example 2
   lets say we are updating metadat after clustering commit
   1 -  written the data files on storage
   2 - commited clustering operation on hudi timeline by creating replace commit
   3 - update metadata
   
   now let's say 3rd operation of updating metadata fails , then latest 
metadata commited file will not be in sync default metadata file 
(0000000*.meta),and there is no scheduled mechanism to bring it in sync state. 
hence once archival archives latest metadata file related replace commit from 
timeline, all the writer will start reading from default metadata file 
(0000000*.meta) and this will cause data duplication.



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