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


##########
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:
   @danny0405 , There are consistency issues we might face if we try updating 
metadata after finishing clustering commit.
   What if underlying file system is down and updateMetadata fails to sync 
metadata, then there is no mechanism to bring it in sync with latest committed 
metadata, and archival will remove replace commit eventually and it will end up 
in an inconsistent state.
   On the other hand in archival process , it will be eventually in sync with 
committed metadata before replace commit getting archived.
   I think consistent hashing metadata has strong dependency on archival 
process, As it is dependent on active timeline replaced commit to load metadata.
   
   



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