You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/01/02 16:32:23 UTC

[GitHub] [hudi] nsivabalan commented on a change in pull request #4078: [HUDI-2833] Clean up unused archive files instead of expanding indefinitely.

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
##########
@@ -106,6 +116,23 @@ private Writer openWriter() {
     }
   }
 
+  public void reOpenWriter() {
+    try {
+      if (this.writer != null) {
+        this.writer.close();

Review comment:
       can we call openwriter() here and avoid lines 124 to 129.
   ```
   if (this.writer != null) {
           this.writer.close();
           this.writer = null;
   }
   openWriter(); 
   ```

##########
File path: hudi-common/src/main/avro/HoodieMergeArchiveFilePlan.avsc
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+{
+   "namespace":"org.apache.hudi.avro.model",
+   "type":"record",
+   "name":"HoodieMergeArchiveFilePlan",
+   "fields":[

Review comment:
       Can we introduce a version number.Just incase we evolve this in future.

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
##########
@@ -134,12 +161,199 @@ public boolean archiveIfRequired(HoodieEngineContext context) throws IOException
         LOG.info("No Instants to archive");
       }
 
+      if (config.getArchiveAutoMergeEnable()) {
+        mergeArchiveFilesIfNecessary(context);
+      }
       return success;
     } finally {
       close();
     }
   }
 
+  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*"));
+    List<FileStatus> mergeCandidate = new ArrayList<>();
+    int archiveFilesCompactBatch = config.getArchiveFilesMergeBatchSize();
+    long smallFileLimitBytes = config.getArchiveMergeSmallFileLimitBytes();
+
+    for (FileStatus fs: fsStatuses) {
+      if (fs.getLen() < smallFileLimitBytes) {
+        mergeCandidate.add(fs);
+      }
+      if (mergeCandidate.size() >= archiveFilesCompactBatch) {
+        break;
+      }
+    }
+
+    if (mergeCandidate.size() >= archiveFilesCompactBatch) {
+      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);
+    }
+  }
+
+  /**
+   * Get final written archive file name based on storageSchemes support append or not.
+   */
+  private String computeLogFileName() throws IOException {
+    if (!StorageSchemes.isAppendSupported(metaClient.getFs().getScheme())) {
+      String logWriteToken = writer.getLogFile().getLogWriteToken();
+      HoodieLogFile hoodieLogFile = writer.getLogFile().rollOver(metaClient.getFs(), logWriteToken);
+      return hoodieLogFile.getFileName();
+    } else {
+      return writer.getLogFile().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()) {

Review comment:
       there is a chance that user enabled this config and later disabled it back. After disabling it, whats the expected behavior. Do we atleast clean any residues ? 

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
##########
@@ -134,12 +161,199 @@ public boolean archiveIfRequired(HoodieEngineContext context) throws IOException
         LOG.info("No Instants to archive");
       }
 
+      if (config.getArchiveAutoMergeEnable()) {
+        mergeArchiveFilesIfNecessary(context);
+      }
       return success;
     } finally {
       close();
     }
   }
 
+  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(

Review comment:
       do you think we should order these files in some order. like file sizes or mod time. Just incase there are too many files for the first time when this config is enabled, it might take few merge archives to merge all small archive files. I could not think of any correctness issue. but wanted to remind just incase. 

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
##########
@@ -134,12 +161,199 @@ public boolean archiveIfRequired(HoodieEngineContext context) throws IOException
         LOG.info("No Instants to archive");
       }
 
+      if (config.getArchiveAutoMergeEnable()) {
+        mergeArchiveFilesIfNecessary(context);
+      }
       return success;
     } finally {
       close();
     }
   }
 
+  private void mergeArchiveFilesIfNecessary(HoodieEngineContext context) throws IOException {

Review comment:
       can we add a java doc comment on what all we do in this method. 

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
##########
@@ -134,12 +161,199 @@ public boolean archiveIfRequired(HoodieEngineContext context) throws IOException
         LOG.info("No Instants to archive");
       }
 
+      if (config.getArchiveAutoMergeEnable()) {
+        mergeArchiveFilesIfNecessary(context);
+      }
       return success;
     } finally {
       close();
     }
   }
 
+  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*"));
+    List<FileStatus> mergeCandidate = new ArrayList<>();
+    int archiveFilesCompactBatch = config.getArchiveFilesMergeBatchSize();
+    long smallFileLimitBytes = config.getArchiveMergeSmallFileLimitBytes();
+
+    for (FileStatus fs: fsStatuses) {
+      if (fs.getLen() < smallFileLimitBytes) {
+        mergeCandidate.add(fs);
+      }
+      if (mergeCandidate.size() >= archiveFilesCompactBatch) {
+        break;
+      }
+    }
+
+    if (mergeCandidate.size() >= archiveFilesCompactBatch) {
+      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);

Review comment:
       in relation to my question raised in computeLogFileName(), if we don't explicitly use a new log file for the merged file, don't we delete the log file which has the merge log block as well here? log10 for eg (based on my example comment in computeLogFileName)

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
##########
@@ -134,12 +161,199 @@ public boolean archiveIfRequired(HoodieEngineContext context) throws IOException
         LOG.info("No Instants to archive");
       }
 
+      if (config.getArchiveAutoMergeEnable()) {
+        mergeArchiveFilesIfNecessary(context);
+      }
       return success;
     } finally {
       close();
     }
   }
 
+  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*"));
+    List<FileStatus> mergeCandidate = new ArrayList<>();
+    int archiveFilesCompactBatch = config.getArchiveFilesMergeBatchSize();
+    long smallFileLimitBytes = config.getArchiveMergeSmallFileLimitBytes();
+
+    for (FileStatus fs: fsStatuses) {
+      if (fs.getLen() < smallFileLimitBytes) {
+        mergeCandidate.add(fs);
+      }
+      if (mergeCandidate.size() >= archiveFilesCompactBatch) {
+        break;
+      }
+    }
+
+    if (mergeCandidate.size() >= archiveFilesCompactBatch) {
+      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);
+    }
+  }
+
+  /**
+   * Get final written archive file name based on storageSchemes support append or not.
+   */
+  private String computeLogFileName() throws IOException {
+    if (!StorageSchemes.isAppendSupported(metaClient.getFs().getScheme())) {
+      String logWriteToken = writer.getLogFile().getLogWriteToken();
+      HoodieLogFile hoodieLogFile = writer.getLogFile().rollOver(metaClient.getFs(), logWriteToken);
+      return hoodieLogFile.getFileName();
+    } else {
+      return writer.getLogFile().getFileName();

Review comment:
       what does this entire feature mean for a storage scheme which supports append. 
   For eg, 
   log1 (could have 20 log blocks pertaining to 20 archivals)
   log2 (could have 20 log blocks pertaining to 20 archivals)
   log3
   .
   .
   After may be 10 log files, threshold hits and mergeArchivekicks in. last log fie is log10. 
   
   1. Do we create a new log block in log10 which will contain all merged log blocks from log1 to log10 ? Or should we explicitly create a new log file (log11) 
   2. Do we even need to consider adding this new feature for storage schemes where append is supported. In other words, should we consider enabling this feature just for storage schemes where append is not supported. and leave it as no-op for storage schemes appends are supported. 
   




-- 
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-unsubscribe@hudi.apache.org

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