1
0

Nicer handling of timeline archival for Cloud storage

- When append() is not supported, rollover to new file always (instead of failing)
 - Provide way to configure archive log folder (avoids small files inside .hoodie)
 - Datasets written via Spark datasource archive to .hoodie/archived
 - HoodieClientExample will now retain only 2,3 commits to exercise archival path during dev cycles
 - Few tweaks to code structure around CommitArchiveLog
This commit is contained in:
vinothchandar
2018-01-03 04:32:21 -08:00
committed by vinoth chandar
parent 0cd186c899
commit cf7f7aabb9
12 changed files with 121 additions and 56 deletions

View File

@@ -63,7 +63,8 @@ public class TestHoodieCommitArchiveLog {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").build();
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg,
new HoodieTableMetaClient(fs.getConf(), cfg.getBasePath(), true));
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
}
@@ -82,9 +83,9 @@ public class TestHoodieCommitArchiveLog {
HoodieTestDataGenerator.createCommitFile(basePath, "104");
HoodieTestDataGenerator.createCommitFile(basePath, "105");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), basePath);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath);
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
@@ -96,18 +97,19 @@ public class TestHoodieCommitArchiveLog {
HoodieTestUtils.createCleanFiles(basePath, "105");
//reload the timeline and get all the commmits before archive
timeline = metadata.getActiveTimeline().reload().getAllCommitsTimeline()
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline()
.filterCompletedInstants();
List<HoodieInstant> originalCommits = timeline.getInstants().collect(Collectors.toList());
assertEquals("Loaded 6 commits and the count should match", 12, timeline.countInstants());
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg,
new HoodieTableMetaClient(fs.getConf(), basePath, true));
assertTrue(archiveLog.archiveIfRequired());
//reload the timeline and remove the remaining commits
timeline = metadata.getActiveTimeline().reload().getAllCommitsTimeline()
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline()
.filterCompletedInstants();
originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList()));
@@ -147,22 +149,20 @@ public class TestHoodieCommitArchiveLog {
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
HoodieTestDataGenerator.createCommitFile(basePath, "100");
HoodieTestDataGenerator.createCommitFile(basePath, "101");
HoodieTestDataGenerator.createCommitFile(basePath, "102");
HoodieTestDataGenerator.createCommitFile(basePath, "103");
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline()
.filterCompletedInstants();
assertEquals("Loaded 4 commits and the count should match", 4, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
timeline =
metadata.getActiveTimeline().reload().getCommitsTimeline()
.filterCompletedInstants();
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline()
.filterCompletedInstants();
assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4,
timeline.countInstants());
}
@@ -173,8 +173,8 @@ public class TestHoodieCommitArchiveLog {
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
HoodieTestDataGenerator.createCommitFile(basePath, "100");
HoodieTestDataGenerator.createCommitFile(basePath, "101");
HoodieTestDataGenerator.createCommitFile(basePath, "102");
@@ -182,14 +182,13 @@ public class TestHoodieCommitArchiveLog {
HoodieTestDataGenerator.createCommitFile(basePath, "104");
HoodieTestDataGenerator.createCommitFile(basePath, "105");
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline()
.filterCompletedInstants();
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
timeline =
metadata.getActiveTimeline().reload().getCommitsTimeline()
.filterCompletedInstants();
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline()
.filterCompletedInstants();
assertTrue("Archived commits should always be safe",
timeline.containsOrBeforeTimelineStarts("100"));
assertTrue("Archived commits should always be safe",
@@ -206,8 +205,8 @@ public class TestHoodieCommitArchiveLog {
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
HoodieTestDataGenerator.createCommitFile(basePath, "100");
HoodieTestDataGenerator.createCommitFile(basePath, "101");
HoodieTestDataGenerator.createSavepointFile(basePath, "101");
@@ -216,14 +215,13 @@ public class TestHoodieCommitArchiveLog {
HoodieTestDataGenerator.createCommitFile(basePath, "104");
HoodieTestDataGenerator.createCommitFile(basePath, "105");
HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline()
.filterCompletedInstants();
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired();
assertTrue(result);
timeline =
metadata.getActiveTimeline().reload().getCommitsTimeline()
.filterCompletedInstants();
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline()
.filterCompletedInstants();
assertEquals(
"Since we have a savepoint at 101, we should never archive any commit after 101 (we only archive 100)",
5, timeline.countInstants());