zhangyue19921010 commented on a change in pull request #4078:
URL: https://github.com/apache/hudi/pull/4078#discussion_r781997631



##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
##########
@@ -134,12 +161,222 @@ public boolean archiveIfRequired(HoodieEngineContext 
context) throws IOException
         LOG.info("No Instants to archive");
       }
 
+      if (config.getArchiveAutoMergeEnable() && 
!StorageSchemes.isAppendSupported(metaClient.getFs().getScheme())) {
+        mergeArchiveFilesIfNecessary(context);
+      }
       return success;
     } finally {
       close();
     }
   }
 
+  /**
+   * Here Hoodie can merge the small archive files into a new larger one.
+   * Only used for filesystem which is not supported append operation.
+   * The hole merge small archive files operation has four stages:
+   * 1. Build merge plan with merge candidates/merged file name infos.
+   * 2. Do merge.
+   * 3. Delete all the candidates.
+   * 4. Delete the merge plan.
+   * @param context HoodieEngineContext
+   * @throws IOException
+   */
+  private void mergeArchiveFilesIfNecessary(HoodieEngineContext context) 
throws IOException {
+    Path planPath = new Path(metaClient.getArchivePath(), 
mergeArchivePlanName);
+    // Flush reminded content if existed and open a new write
+    reOpenWriter();
+    // List all archive files
+    FileStatus[] fsStatuses = metaClient.getFs().globStatus(
+        new Path(metaClient.getArchivePath() + "/.commits_.archive*"));
+    // Sort files by version suffix in reverse (implies reverse chronological 
order)
+    Arrays.sort(fsStatuses, new 
HoodieArchivedTimeline.ArchiveFileVersionComparator());
+
+    int archiveFilesMergeBatch = config.getArchiveFilesMergeBatchSize();
+    long smallFileLimitBytes = config.getArchiveMergeSmallFileLimitBytes();
+
+    List<FileStatus> mergeCandidate = getMergeCandidates(smallFileLimitBytes, 
fsStatuses);
+
+    if (mergeCandidate.size() >= archiveFilesMergeBatch) {
+      List<String> candidateFiles = mergeCandidate.stream().map(fs -> 
fs.getPath().toString()).collect(Collectors.toList());
+      // before merge archive files build merge plan
+      String logFileName = computeLogFileName();
+      buildArchiveMergePlan(candidateFiles, planPath, logFileName);
+      // merge archive files
+      mergeArchiveFiles(mergeCandidate);
+      // after merge, delete the small archive files.
+      deleteFilesParallelize(metaClient, candidateFiles, context, true);
+      // finally, delete archiveMergePlan which means merge small archive 
files operatin is succeed.
+      metaClient.getFs().delete(planPath, false);
+    }
+  }
+
+  /**
+   * Find the latest 'huge archive file' index as a break point and only 
check/merge newer archive files.
+   * Because we need to keep the original order of archive files which is 
important when loading archived instants with time filter.
+   * {@link HoodieArchivedTimeline} loadInstants(TimeRangeFilter filter, 
boolean loadInstantDetails, Function<GenericRecord, Boolean> commitsFilter)
+   * @param smallFileLimitBytes small File Limit Bytes
+   * @param fsStatuses Sort by version suffix in reverse
+   * @return merge candidates
+   */
+  private List<FileStatus> getMergeCandidates(long smallFileLimitBytes, 
FileStatus[] fsStatuses) {
+    int index = 0;
+    for (; index < fsStatuses.length; index++) {
+      if (fsStatuses[index].getLen() > smallFileLimitBytes) {
+        break;
+      }
+    }
+    return Arrays.stream(fsStatuses).limit(index).collect(Collectors.toList());
+  }
+
+  /**
+   * Get final written archive file name based on storageSchemes support 
append or not.
+   */
+  private String computeLogFileName() throws IOException {
+    String logWriteToken = writer.getLogFile().getLogWriteToken();
+    HoodieLogFile hoodieLogFile = 
writer.getLogFile().rollOver(metaClient.getFs(), logWriteToken);
+    return hoodieLogFile.getFileName();
+  }
+
+  /**
+   * Check/Solve if there is any failed and unfinished merge small archive 
files operation
+   * @param context HoodieEngineContext used for parallelize to delete small 
archive files if necessary.
+   * @throws IOException
+   */
+  private void verifyLastMergeArchiveFilesIfNecessary(HoodieEngineContext 
context) throws IOException {
+    if (config.getArchiveAutoMergeEnable() && 
!StorageSchemes.isAppendSupported(metaClient.getFs().getScheme())) {
+      Path planPath = new Path(metaClient.getArchivePath(), 
mergeArchivePlanName);
+      HoodieWrapperFileSystem fs = metaClient.getFs();
+      // If plan exist, last merge small archive files was failed.
+      // we need to revert or complete last action.
+      if (fs.exists(planPath)) {
+        HoodieMergeArchiveFilePlan plan = 
TimelineMetadataUtils.deserializeAvroMetadata(readDataFromPath(planPath).get(), 
HoodieMergeArchiveFilePlan.class);

Review comment:
       Nice catch! We need to take care of parsing merge plan failed.
   Changed.




-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to