1
0

[HUDI-1576] Make archiving an async service (#4795)

This commit is contained in:
Raymond Xu
2022-02-14 18:15:06 -08:00
committed by GitHub
parent 3b401d839c
commit 27bd7b538e
15 changed files with 327 additions and 127 deletions

View File

@@ -102,7 +102,8 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000");
HoodieWriteConfig writeConfig = getConfigBuilder()
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build())
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY)
.withAutoArchive(false).withAutoClean(false).build())
.withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL)
// Timeline-server-based markers are not used for multi-writer tests
.withMarkersType(MarkerType.DIRECT.name())

View File

@@ -52,7 +52,7 @@ import org.apache.hudi.metadata.HoodieTableMetadataWriter;
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.HoodieTimelineArchiveLog;
import org.apache.hudi.client.HoodieTimelineArchiver;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -286,8 +286,8 @@ public class TestHoodieMetadataBase extends HoodieClientTestHarness {
protected void archiveDataTable(HoodieWriteConfig writeConfig, HoodieTableMetaClient metaClient) throws IOException {
HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(writeConfig, table);
archiveLog.archiveIfRequired(context);
HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(writeConfig, table);
archiver.archiveIfRequired(context);
}
protected void validateMetadata(HoodieTestTable testTable) throws IOException {

View File

@@ -47,7 +47,7 @@ import org.apache.hudi.metadata.HoodieTableMetadataWriter;
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.HoodieTimelineArchiveLog;
import org.apache.hudi.client.HoodieTimelineArchiver;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hadoop.conf.Configuration;
@@ -77,9 +77,9 @@ 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 {
public class TestHoodieTimelineArchiver extends HoodieClientTestHarness {
private static final Logger LOG = LogManager.getLogger(TestHoodieTimelineArchiveLog.class);
private static final Logger LOG = LogManager.getLogger(TestHoodieTimelineArchiver.class);
private Configuration hadoopConf;
private HoodieWrapperFileSystem wrapperFs;
@@ -172,8 +172,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
.withParallelism(2, 2).forTable("test-trip-table").build();
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
boolean result = archiveLog.archiveIfRequired(context);
HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table);
boolean result = archiver.archiveIfRequired(context);
assertTrue(result);
}
@@ -224,14 +224,14 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
// 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);
HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(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();
archiver.reOpenWriter();
Path plan = new Path(metaClient.getArchivePath(), HoodieArchivedTimeline.MERGE_ARCHIVE_PLAN_NAME);
archiveLog.buildArchiveMergePlan(candidateFiles, plan, ".commits_.archive.3_1-0-1");
archiver.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()));
@@ -274,15 +274,15 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
// do a single merge small archive files
HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(writeConfig, table);
HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(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();
archiver.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();
archiver.buildArchiveMergePlan(candidateFiles, new Path(metaClient.getArchivePath(), HoodieArchivedTimeline.MERGE_ARCHIVE_PLAN_NAME), ".commits_.archive.3_1-0-1");
archiver.mergeArchiveFiles(Arrays.stream(fsStatuses).collect(Collectors.toList()));
HoodieLogFormat.Writer writer = archiver.reOpenWriter();
// check loading archived and active timeline success
HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false);
@@ -327,16 +327,16 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
// do a single merge small archive files
HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(writeConfig, table);
HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(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();
archiver.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();
archiver.buildArchiveMergePlan(candidateFiles, new Path(metaClient.getArchivePath(), HoodieArchivedTimeline.MERGE_ARCHIVE_PLAN_NAME), ".commits_.archive.3_1-0-1");
archiver.mergeArchiveFiles(Arrays.stream(fsStatuses).collect(Collectors.toList()));
archiver.reOpenWriter();
// delete only one of the small archive file to simulate delete action failed.
metaClient.getFs().delete(fsStatuses[0].getPath());
@@ -397,16 +397,16 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
}
HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(writeConfig, table);
HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(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();
archiver.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();
archiver.buildArchiveMergePlan(candidateFiles, new Path(metaClient.getArchivePath(), HoodieArchivedTimeline.MERGE_ARCHIVE_PLAN_NAME), ".commits_.archive.3_1-0-1");
archiver.mergeArchiveFiles(Arrays.stream(fsStatuses).collect(Collectors.toList()));
HoodieLogFormat.Writer writer = archiver.reOpenWriter();
String s = "Dummy Content";
// stain the current merged archive file.
@@ -470,11 +470,11 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
HoodieTestDataGenerator.createCommitFile(basePath, "104", wrapperFs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf());
HoodieTable table = HoodieSparkTable.create(cfg, context);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table);
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
assertEquals(6, timeline.countInstants(), "Loaded 6 commits and the count should match");
assertTrue(archiveLog.archiveIfRequired(context));
assertTrue(archiver.archiveIfRequired(context));
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
assertEquals(5, timeline.countInstants(),
"Since we have a savepoint at 101, we should never archive any commit after 101 (we only archive 100)");
@@ -620,8 +620,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
HoodieTestDataGenerator.createCommitFile(basePath, "5", wrapperFs.getConf());
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
boolean result = archiveLog.archiveIfRequired(context);
HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table);
boolean result = archiver.archiveIfRequired(context);
assertTrue(result);
HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline();
List<HoodieInstant> archivedInstants = Arrays.asList(instant1, instant2, instant3);
@@ -776,9 +776,9 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
}
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table);
archiveLog.archiveIfRequired(context);
archiver.archiveIfRequired(context);
Stream<HoodieInstant> currentInstants = metaClient.getActiveTimeline().reload().getInstants();
Map<Object, List<HoodieInstant>> actionInstantMap = currentInstants.collect(Collectors.groupingBy(HoodieInstant::getAction));
@@ -810,9 +810,9 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
HoodieInstant notArchivedInstant3 = createCleanMetadata("14", true);
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table);
archiveLog.archiveIfRequired(context);
archiver.archiveIfRequired(context);
List<HoodieInstant> notArchivedInstants = metaClient.getActiveTimeline().reload().getInstants().collect(Collectors.toList());
assertEquals(3, notArchivedInstants.size(), "Not archived instants should be 3");
@@ -894,8 +894,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
HoodieTimeline timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
List<HoodieInstant> originalCommits = timeline.getInstants().collect(Collectors.toList());
HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(writeConfig, table);
archiveLog.archiveIfRequired(context);
HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(writeConfig, table);
archiver.archiveIfRequired(context);
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
List<HoodieInstant> commitsAfterArchival = timeline.getInstants().collect(Collectors.toList());
return Pair.of(originalCommits, commitsAfterArchival);