danny0405 commented on code in PR #9209:
URL: https://github.com/apache/hudi/pull/9209#discussion_r1288167338


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ArchivedTimelineWriter.java:
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.
+ */
+
+package org.apache.hudi.client.utils;
+
+import org.apache.hudi.avro.model.HoodieArchivedInstant;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieArchivedManifest;
+import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.VisibleForTesting;
+import org.apache.hudi.common.util.collection.ClosableIterator;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieCommitException;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.io.storage.HoodieAvroParquetReader;
+import org.apache.hudi.io.storage.HoodieFileReaderFactory;
+import org.apache.hudi.io.storage.HoodieFileWriter;
+import org.apache.hudi.io.storage.HoodieFileWriterFactory;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.marker.WriteMarkers;
+import org.apache.hudi.table.marker.WriteMarkersFactory;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * An archived timeline writer which organizes the files as an LSM tree.
+ */
+public class ArchivedTimelineWriter {
+  private static final Logger LOG = 
LoggerFactory.getLogger(ArchivedTimelineWriter.class);
+
+  private final HoodieWriteConfig config;
+  private final HoodieTable<?, ?, ?, ?> table;
+  private final HoodieTableMetaClient metaClient;
+
+  private HoodieWriteConfig writeConfig;
+
+  private ArchivedTimelineWriter(HoodieWriteConfig config, HoodieTable<?, ?, 
?, ?> table) {
+    this.config = config;
+    this.table = table;
+    this.metaClient = table.getMetaClient();
+  }
+
+  public static ArchivedTimelineWriter getInstance(HoodieWriteConfig config, 
HoodieTable<?, ?, ?, ?> table) {
+    return new ArchivedTimelineWriter(config, table);
+  }
+
+  public void write(HoodieEngineContext context, List<ActiveInstant> instants) 
throws HoodieCommitException {
+    Path filePath = new Path(metaClient.getArchivePath(),
+        newFileName(instants.get(0).getInstantTime(), 
instants.get(instants.size() - 1).getInstantTime(), 
HoodieArchivedTimeline.FILE_LAYER_ZERO));
+    try (HoodieFileWriter writer = openWriter(filePath)) {
+      Schema wrapperSchema = HoodieArchivedInstant.getClassSchema();
+      LOG.info("Archiving schema " + wrapperSchema.toString());
+      for (ActiveInstant triple : instants) {
+        try {
+          deleteAnyLeftOverMarkers(context, triple);
+          // in local FS and HDFS, there could be empty completed instants due 
to crash.
+          final HoodieArchivedInstant metaEntry = 
MetadataConversionUtils.createArchivedInstant(triple, metaClient);
+          writer.write(metaEntry.getInstantTime(), new 
HoodieAvroIndexedRecord(metaEntry), wrapperSchema);
+        } catch (Exception e) {
+          LOG.error("Failed to archive instant: " + triple.getInstantTime(), 
e);
+          if (this.config.isFailOnTimelineArchivingEnabled()) {
+            throw e;
+          }
+        }
+      }
+      updateManifest(filePath.getName());
+    } catch (Exception e) {
+      throw new HoodieCommitException("Failed to archive commits", e);
+    }
+  }
+
+  public void updateManifest(String fileToAdd) throws IOException {
+    // 1. read the latest manifest version file;
+    // 2. read the latest manifest file for valid files;
+    // 3. add this new file to the existing file list from step2.
+    int latestVersion = 
HoodieArchivedTimeline.latestSnapshotVersion(metaClient);
+    HoodieArchivedManifest latestManifest = 
HoodieArchivedTimeline.latestSnapshotManifest(metaClient, latestVersion);
+    HoodieArchivedManifest newManifest = latestManifest.copy();
+    newManifest.addFile(getFileEntry(fileToAdd));
+    createManifestFile(newManifest, latestVersion);
+  }
+
+  public void updateManifest(List<String> filesToRemove, String fileToAdd) 
throws IOException {
+    // 1. read the latest manifest version file;
+    // 2. read the latest manifest file for valid files;
+    // 3. remove files to the existing file list from step2;
+    // 4. add this new file to the existing file list from step2.
+    int latestVersion = 
HoodieArchivedTimeline.latestSnapshotVersion(metaClient);
+    HoodieArchivedManifest latestManifest = 
HoodieArchivedTimeline.latestSnapshotManifest(metaClient, latestVersion);
+    HoodieArchivedManifest newManifest = latestManifest.copy(filesToRemove);
+    newManifest.addFile(getFileEntry(fileToAdd));
+    createManifestFile(newManifest, latestVersion);
+  }
+
+  private void createManifestFile(HoodieArchivedManifest manifest, int 
currentVersion) throws IOException {
+    byte[] content = manifest.toJsonString().getBytes(StandardCharsets.UTF_8);
+    // version starts from 1 and increases monotonically
+    int newVersion = currentVersion < 0 ? 1 : currentVersion + 1;
+    // create manifest file
+    final Path manifestFilePath = 
HoodieArchivedTimeline.getManifestFilePath(metaClient, newVersion);
+    metaClient.getFs().createImmutableFileInPath(manifestFilePath, 
Option.of(content));
+    // update version file
+    updateVersionFile(newVersion);
+  }
+
+  private void updateVersionFile(int newVersion) throws IOException {
+    byte[] content = 
(String.valueOf(newVersion)).getBytes(StandardCharsets.UTF_8);
+    final Path versionFilePath = 
HoodieArchivedTimeline.getVersionFilePath(metaClient);
+    metaClient.getFs().delete(versionFilePath, false);
+    metaClient.getFs().createImmutableFileInPath(versionFilePath, 
Option.of(content));
+  }
+
+  /**
+   * Compacts the small archive files.
+   *
+   * <p>The parquet naming convention is:
+   *
+   * <pre>${min_instant}_${max_instant}_${level}.parquet</pre>
+   *
+   * <p>The 'min_instant' and 'max_instant' represent the instant time range 
of the parquet file.
+   * The 'level' represents the number of the level where the file is located, 
currently we
+   * have no limit for the number of layers.
+   *
+   * <p>These archive parquet files composite as an LSM tree layout, one 
parquet file contains
+   * a consecutive timestamp instant metadata entries. Different parquet files 
may have
+   * overlapping with the instant time ranges.
+   *
+   * <pre>
+   *   t1_t2_0.parquet, t3_t4_0.parquet, ... t5_t6_0.parquet       L0 layer
+   *                          \            /
+   *                             \     /
+   *                                |
+   *                                V
+   *                          t3_t6_1.parquet                      L1 layer
+   * </pre>
+   *
+   * <p>Compaction and cleaning: once the files number exceed a threshold(now 
constant 10) N,
+   * the oldest N files are then replaced with a compacted file in the next 
layer.
+   * A cleaning action is triggered right after the compaction.
+   *
+   * @param context HoodieEngineContext
+   */
+  @VisibleForTesting
+  public void compactAndClean(HoodieEngineContext context) throws IOException {
+    // 1. List all the latest snapshot files
+    HoodieArchivedManifest latestManifest = 
HoodieArchivedTimeline.latestSnapshotManifest(metaClient);
+    int layer = 0;
+    // 2. triggers the compaction for L0
+    Option<String> compactedFileName = doCompact(latestManifest, layer);
+    while (compactedFileName.isPresent()) {
+      // 3. once a compaction had been executed for the current layer,
+      // continues to trigger compaction for the next layer.
+      latestManifest.addFile(getFileEntry(compactedFileName.get()));
+      compactedFileName = doCompact(latestManifest, ++layer);
+    }
+
+    // cleaning
+    clean(context, layer);
+  }
+
+  private Option<String> doCompact(HoodieArchivedManifest manifest, int layer) 
throws IOException {
+    // 1. list all the files that belong to current layer
+    List<HoodieArchivedManifest.FileEntry> files = manifest.getFiles()
+        .stream().filter(file -> 
HoodieArchivedTimeline.isFileFromLayer(file.getFileName(), 
layer)).collect(Collectors.toList());
+
+    int compactionBatchSize = config.getArchiveMergeFilesBatchSize();
+
+    if (files.size() >= compactionBatchSize) {
+      // 2. sort files by min instant time (implies ascending chronological 
order)
+      files.sort(HoodieArchivedManifest.FileEntry::compareTo);
+      List<String> candidateFiles = getCandidateFiles(files, 
compactionBatchSize);
+      if (candidateFiles.size() < 2) {
+        // the file is too large to compact, returns early.
+        return Option.empty();
+      }
+      String compactedFileName = compactedFileName(candidateFiles);
+
+      // 3. compaction
+      compactArchiveFiles(candidateFiles, compactedFileName);
+      // 4. update the manifest file
+      updateManifest(candidateFiles, compactedFileName);
+      LOG.info("Finishes compaction of archive files: " + candidateFiles);
+      return Option.of(compactedFileName);
+    }
+    return Option.empty();
+  }
+
+  public void compactArchiveFiles(List<String> candidateFiles, String 
compactedFileName) {
+    LOG.info("Starting to merge small archive files.");
+    try (HoodieFileWriter writer = openWriter(new 
Path(metaClient.getArchivePath(), compactedFileName))) {

Review Comment:
   Not sure it deserves to do that, the sort merge costs much memory.



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