[HUDI-2833][Design] Merge small archive files instead of expanding indefinitely. (#4078)
Co-authored-by: yuezhang <yuezhang@freewheel.tv>
This commit is contained in:
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.io;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.client.utils.MetadataConversionUtils;
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
@@ -26,6 +27,8 @@ import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
@@ -35,9 +38,12 @@ import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
||||
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
@@ -68,6 +74,7 @@ import java.util.stream.Stream;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
|
||||
@@ -113,12 +120,41 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
|
||||
return initTestTableAndGetWriteConfig(enableMetadata, minArchivalCommits, maxArchivalCommits, maxDeltaCommitsMetadataTable, HoodieTableType.COPY_ON_WRITE);
|
||||
}
|
||||
|
||||
private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, int minArchivalCommits, int maxArchivalCommits, int maxDeltaCommitsMetadataTable,
|
||||
private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata,
|
||||
int minArchivalCommits,
|
||||
int maxArchivalCommits,
|
||||
int maxDeltaCommitsMetadataTable,
|
||||
HoodieTableType tableType) throws Exception {
|
||||
return initTestTableAndGetWriteConfig(enableMetadata, minArchivalCommits, maxArchivalCommits, maxDeltaCommitsMetadataTable, tableType, false, 10, 209715200);
|
||||
}
|
||||
|
||||
private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata,
|
||||
int minArchivalCommits,
|
||||
int maxArchivalCommits,
|
||||
int maxDeltaCommitsMetadataTable,
|
||||
boolean enableArchiveMerge,
|
||||
int archiveFilesBatch,
|
||||
long size) throws Exception {
|
||||
return initTestTableAndGetWriteConfig(enableMetadata, minArchivalCommits, maxArchivalCommits,
|
||||
maxDeltaCommitsMetadataTable, HoodieTableType.COPY_ON_WRITE, enableArchiveMerge, archiveFilesBatch, size);
|
||||
}
|
||||
|
||||
private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata,
|
||||
int minArchivalCommits,
|
||||
int maxArchivalCommits,
|
||||
int maxDeltaCommitsMetadataTable,
|
||||
HoodieTableType tableType,
|
||||
boolean enableArchiveMerge,
|
||||
int archiveFilesBatch,
|
||||
long size) throws Exception {
|
||||
init(tableType);
|
||||
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build())
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits)
|
||||
.withArchiveMergeEnable(enableArchiveMerge)
|
||||
.withArchiveMergeFilesBatchSize(archiveFilesBatch)
|
||||
.withArchiveMergeSmallFileLimit(size)
|
||||
.build())
|
||||
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||
.withRemoteServerPort(timelineServicePort).build())
|
||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata)
|
||||
@@ -174,6 +210,222 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testMergeSmallArchiveFilesRecoverFromBuildPlanFailed(boolean enableArchiveMerge) throws Exception {
|
||||
HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(false, 2, 3, 2, enableArchiveMerge, 3, 209715200);
|
||||
|
||||
// do ingestion and trigger archive actions here.
|
||||
for (int i = 1; i < 8; i++) {
|
||||
testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
|
||||
archiveAndGetCommitsList(writeConfig);
|
||||
}
|
||||
|
||||
// build a merge small archive plan with dummy content
|
||||
// this plan can not be deserialized.
|
||||
HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(writeConfig, table);
|
||||
FileStatus[] fsStatuses = metaClient.getFs().globStatus(
|
||||
new Path(metaClient.getArchivePath() + "/.commits_.archive*"));
|
||||
List<String> candidateFiles = Arrays.stream(fsStatuses).map(fs -> fs.getPath().toString()).collect(Collectors.toList());
|
||||
|
||||
archiveLog.reOpenWriter();
|
||||
Path plan = new Path(metaClient.getArchivePath(), HoodieArchivedTimeline.MERGE_ARCHIVE_PLAN_NAME);
|
||||
archiveLog.buildArchiveMergePlan(candidateFiles, plan, ".commits_.archive.3_1-0-1");
|
||||
String s = "Dummy Content";
|
||||
// stain the current merge plan file.
|
||||
FileIOUtils.createFileInPath(metaClient.getFs(), plan, Option.of(s.getBytes()));
|
||||
|
||||
// check that damaged plan file will not block archived timeline loading.
|
||||
HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false);
|
||||
HoodieArchivedTimeline archivedTimeLine = metaClient.getArchivedTimeline().reload();
|
||||
assertEquals(7 * 3, rawActiveTimeline.countInstants() + archivedTimeLine.countInstants());
|
||||
|
||||
// trigger several archive after left damaged merge small archive file plan.
|
||||
for (int i = 1; i < 10; i++) {
|
||||
testTable.doWriteOperation("1000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
|
||||
archiveAndGetCommitsList(writeConfig);
|
||||
}
|
||||
|
||||
// loading archived timeline and active timeline success
|
||||
HoodieActiveTimeline rawActiveTimeline1 = new HoodieActiveTimeline(metaClient, false);
|
||||
HoodieArchivedTimeline archivedTimeLine1 = metaClient.getArchivedTimeline().reload();
|
||||
|
||||
// check instant number
|
||||
assertEquals(16 * 3, archivedTimeLine1.countInstants() + rawActiveTimeline1.countInstants());
|
||||
|
||||
// if there are damaged archive files and damaged plan, hoodie need throw ioe while loading archived timeline.
|
||||
Path damagedFile = new Path(metaClient.getArchivePath(), ".commits_.archive.300_1-0-1");
|
||||
FileIOUtils.createFileInPath(metaClient.getFs(), damagedFile, Option.of(s.getBytes()));
|
||||
|
||||
assertThrows(HoodieException.class, () -> metaClient.getArchivedTimeline().reload());
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testMergeSmallArchiveFilesRecoverFromMergeFailed(boolean enableArchiveMerge) throws Exception {
|
||||
HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(false, 2, 3, 2, enableArchiveMerge, 3, 209715200);
|
||||
|
||||
// do ingestion and trigger archive actions here.
|
||||
for (int i = 1; i < 8; i++) {
|
||||
testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
|
||||
archiveAndGetCommitsList(writeConfig);
|
||||
}
|
||||
|
||||
// do a single merge small archive files
|
||||
HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(writeConfig, table);
|
||||
FileStatus[] fsStatuses = metaClient.getFs().globStatus(
|
||||
new Path(metaClient.getArchivePath() + "/.commits_.archive*"));
|
||||
List<String> candidateFiles = Arrays.stream(fsStatuses).map(fs -> fs.getPath().toString()).collect(Collectors.toList());
|
||||
archiveLog.reOpenWriter();
|
||||
|
||||
archiveLog.buildArchiveMergePlan(candidateFiles, new Path(metaClient.getArchivePath(), HoodieArchivedTimeline.MERGE_ARCHIVE_PLAN_NAME), ".commits_.archive.3_1-0-1");
|
||||
archiveLog.mergeArchiveFiles(Arrays.stream(fsStatuses).collect(Collectors.toList()));
|
||||
HoodieLogFormat.Writer writer = archiveLog.reOpenWriter();
|
||||
|
||||
// check loading archived and active timeline success
|
||||
HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false);
|
||||
HoodieArchivedTimeline archivedTimeLine = metaClient.getArchivedTimeline().reload();
|
||||
assertEquals(7 * 3, rawActiveTimeline.countInstants() + archivedTimeLine.reload().countInstants());
|
||||
|
||||
String s = "Dummy Content";
|
||||
// stain the current merged archive file.
|
||||
FileIOUtils.createFileInPath(metaClient.getFs(), writer.getLogFile().getPath(), Option.of(s.getBytes()));
|
||||
|
||||
// do another archive actions with merge small archive files.
|
||||
for (int i = 1; i < 10; i++) {
|
||||
testTable.doWriteOperation("1000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
|
||||
archiveAndGetCommitsList(writeConfig);
|
||||
}
|
||||
|
||||
// check result.
|
||||
// we need to load archived timeline successfully and ignore the parsing damage merged archive files exception.
|
||||
HoodieActiveTimeline rawActiveTimeline1 = new HoodieActiveTimeline(metaClient, false);
|
||||
HoodieArchivedTimeline archivedTimeLine1 = metaClient.getArchivedTimeline().reload();
|
||||
|
||||
assertEquals(16 * 3, archivedTimeLine1.countInstants() + rawActiveTimeline1.countInstants());
|
||||
|
||||
// if there are a damaged merged archive files and other common damaged archive file.
|
||||
// hoodie need throw ioe while loading archived timeline because of parsing the damaged archive file.
|
||||
Path damagedFile = new Path(metaClient.getArchivePath(), ".commits_.archive.300_1-0-1");
|
||||
FileIOUtils.createFileInPath(metaClient.getFs(), damagedFile, Option.of(s.getBytes()));
|
||||
|
||||
assertThrows(HoodieException.class, () -> metaClient.getArchivedTimeline().reload());
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testMergeSmallArchiveFilesRecoverFromDeleteFailed(boolean enableArchiveMerge) throws Exception {
|
||||
HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(false, 2, 3, 2, enableArchiveMerge, 3, 209715200);
|
||||
|
||||
// do ingestion and trigger archive actions here.
|
||||
for (int i = 1; i < 8; i++) {
|
||||
testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
|
||||
archiveAndGetCommitsList(writeConfig);
|
||||
}
|
||||
|
||||
// do a single merge small archive files
|
||||
HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(writeConfig, table);
|
||||
FileStatus[] fsStatuses = metaClient.getFs().globStatus(
|
||||
new Path(metaClient.getArchivePath() + "/.commits_.archive*"));
|
||||
List<String> candidateFiles = Arrays.stream(fsStatuses).map(fs -> fs.getPath().toString()).collect(Collectors.toList());
|
||||
|
||||
archiveLog.reOpenWriter();
|
||||
|
||||
archiveLog.buildArchiveMergePlan(candidateFiles, new Path(metaClient.getArchivePath(), HoodieArchivedTimeline.MERGE_ARCHIVE_PLAN_NAME), ".commits_.archive.3_1-0-1");
|
||||
archiveLog.mergeArchiveFiles(Arrays.stream(fsStatuses).collect(Collectors.toList()));
|
||||
archiveLog.reOpenWriter();
|
||||
|
||||
// delete only one of the small archive file to simulate delete action failed.
|
||||
metaClient.getFs().delete(fsStatuses[0].getPath());
|
||||
|
||||
// loading archived timeline and active timeline success
|
||||
HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false);
|
||||
HoodieArchivedTimeline archivedTimeLine = metaClient.getArchivedTimeline().reload();
|
||||
assertEquals(7 * 3, rawActiveTimeline.countInstants() + archivedTimeLine.countInstants());
|
||||
|
||||
// do another archive actions with merge small archive files.
|
||||
for (int i = 1; i < 10; i++) {
|
||||
testTable.doWriteOperation("1000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
|
||||
archiveAndGetCommitsList(writeConfig);
|
||||
}
|
||||
|
||||
// check result.
|
||||
HoodieActiveTimeline rawActiveTimeline1 = new HoodieActiveTimeline(metaClient, false);
|
||||
HoodieArchivedTimeline archivedTimeLine1 = metaClient.getArchivedTimeline().reload();
|
||||
|
||||
assertEquals(16 * 3, archivedTimeLine1.countInstants() + rawActiveTimeline1.countInstants());
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testLoadArchiveTimelineWithDamagedPlanFile(boolean enableArchiveMerge) throws Exception {
|
||||
HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(false, 2, 3, 2, enableArchiveMerge, 3, 209715200);
|
||||
|
||||
// do ingestion and trigger archive actions here.
|
||||
for (int i = 1; i < 8; i++) {
|
||||
testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
|
||||
archiveAndGetCommitsList(writeConfig);
|
||||
}
|
||||
|
||||
Path plan = new Path(metaClient.getArchivePath(), HoodieArchivedTimeline.MERGE_ARCHIVE_PLAN_NAME);
|
||||
String s = "Dummy Content";
|
||||
// stain the current merge plan file.
|
||||
FileIOUtils.createFileInPath(metaClient.getFs(), plan, Option.of(s.getBytes()));
|
||||
|
||||
// check that damaged plan file will not block archived timeline loading.
|
||||
HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false);
|
||||
HoodieArchivedTimeline archivedTimeLine = metaClient.getArchivedTimeline().reload();
|
||||
assertEquals(7 * 3, rawActiveTimeline.countInstants() + archivedTimeLine.countInstants());
|
||||
|
||||
// if there are damaged archive files and damaged plan, hoodie need throw ioe while loading archived timeline.
|
||||
Path damagedFile = new Path(metaClient.getArchivePath(), ".commits_.archive.300_1-0-1");
|
||||
FileIOUtils.createFileInPath(metaClient.getFs(), damagedFile, Option.of(s.getBytes()));
|
||||
|
||||
assertThrows(HoodieException.class, () -> metaClient.getArchivedTimeline().reload());
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testLoadArchiveTimelineWithUncompletedMergeArchiveFile(boolean enableArchiveMerge) throws Exception {
|
||||
HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(false, 2, 3, 2, enableArchiveMerge, 3, 209715200);
|
||||
for (int i = 1; i < 8; i++) {
|
||||
testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
|
||||
archiveAndGetCommitsList(writeConfig);
|
||||
}
|
||||
|
||||
HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(writeConfig, table);
|
||||
FileStatus[] fsStatuses = metaClient.getFs().globStatus(
|
||||
new Path(metaClient.getArchivePath() + "/.commits_.archive*"));
|
||||
List<String> candidateFiles = Arrays.stream(fsStatuses).map(fs -> fs.getPath().toString()).collect(Collectors.toList());
|
||||
|
||||
archiveLog.reOpenWriter();
|
||||
|
||||
archiveLog.buildArchiveMergePlan(candidateFiles, new Path(metaClient.getArchivePath(), HoodieArchivedTimeline.MERGE_ARCHIVE_PLAN_NAME), ".commits_.archive.3_1-0-1");
|
||||
archiveLog.mergeArchiveFiles(Arrays.stream(fsStatuses).collect(Collectors.toList()));
|
||||
HoodieLogFormat.Writer writer = archiveLog.reOpenWriter();
|
||||
|
||||
String s = "Dummy Content";
|
||||
// stain the current merged archive file.
|
||||
FileIOUtils.createFileInPath(metaClient.getFs(), writer.getLogFile().getPath(), Option.of(s.getBytes()));
|
||||
|
||||
// if there's only a damaged merged archive file, we need to ignore the exception while reading this damaged file.
|
||||
HoodieActiveTimeline rawActiveTimeline1 = new HoodieActiveTimeline(metaClient, false);
|
||||
HoodieArchivedTimeline archivedTimeLine1 = metaClient.getArchivedTimeline();
|
||||
|
||||
assertEquals(7 * 3, archivedTimeLine1.countInstants() + rawActiveTimeline1.countInstants());
|
||||
|
||||
// if there are a damaged merged archive files and other common damaged archive file.
|
||||
// hoodie need throw ioe while loading archived timeline because of parsing the damaged archive file.
|
||||
Path damagedFile = new Path(metaClient.getArchivePath(), ".commits_.archive.300_1-0-1");
|
||||
FileIOUtils.createFileInPath(metaClient.getFs(), damagedFile, Option.of(s.getBytes()));
|
||||
|
||||
assertThrows(HoodieException.class, () -> metaClient.getArchivedTimeline().reload());
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testNoArchivalUntilMaxArchiveConfigWithExtraInflightCommits(boolean enableMetadata) throws Exception {
|
||||
|
||||
Reference in New Issue
Block a user