1
0

Ensure Cleaner and Archiver do not delete file-slices and workload marked for compaction

This commit is contained in:
Balaji Varadarajan
2018-05-31 14:16:19 -07:00
committed by vinoth chandar
parent 0a0451a765
commit 9b78523d62
10 changed files with 666 additions and 76 deletions

View File

@@ -16,7 +16,9 @@
package com.uber.hoodie.io;
import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import com.google.common.collect.Sets;
@@ -30,7 +32,9 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import java.io.File;
@@ -133,11 +137,47 @@ public class TestHoodieCommitArchiveLog {
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 4).build())
.forTable("test-trip-table").build();
HoodieTestUtils.init(hadoopConf, basePath);
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "100"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "100"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "101"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "101"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "102"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "102"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "103"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "103"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "104"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "104"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "104", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "105"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "105"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath);
@@ -172,6 +212,37 @@ public class TestHoodieCommitArchiveLog {
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList()));
// Check compaction instants
List<HoodieInstant> instants =
HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
new Path(metaClient.getMetaAuxiliaryPath()),
HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE);
assertEquals("Should delete all compaction instants < 104", 4, instants.size());
assertFalse("Requested Compaction must be absent for 100", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "100")));
assertFalse("Inflight Compaction must be absent for 100", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "100")));
assertFalse("Requested Compaction must be absent for 101", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "101")));
assertFalse("Inflight Compaction must be absent for 101", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "101")));
assertFalse("Requested Compaction must be absent for 102", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "102")));
assertFalse("Inflight Compaction must be absent for 102", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "102")));
assertFalse("Requested Compaction must be absent for 103", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "103")));
assertFalse("Inflight Compaction must be absent for 103", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "103")));
assertTrue("Requested Compaction must be present for 104", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "104")));
assertTrue("Inflight Compaction must be present for 104", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "104")));
assertTrue("Requested Compaction must be present for 105", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "105")));
assertTrue("Inflight Compaction must be present for 105", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "105")));
//read the file
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(dfs,
new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")),
@@ -210,9 +281,33 @@ public class TestHoodieCommitArchiveLog {
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "100"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "100"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "101"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "101"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "102"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "102"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "103"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "103"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf());
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
@@ -221,6 +316,28 @@ public class TestHoodieCommitArchiveLog {
assertTrue(result);
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4, timeline.countInstants());
List<HoodieInstant> instants =
HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
new Path(metaClient.getMetaAuxiliaryPath()),
HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE);
assertEquals("Should not delete any aux compaction files when maxCommitsToKeep is 5", 8, instants.size());
assertTrue("Requested Compaction must be present for 100", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "100")));
assertTrue("Inflight Compaction must be present for 100", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "100")));
assertTrue("Requested Compaction must be present for 101", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "101")));
assertTrue("Inflight Compaction must be present for 101", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "101")));
assertTrue("Requested Compaction must be present for 102", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "102")));
assertTrue("Inflight Compaction must be present for 102", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "102")));
assertTrue("Requested Compaction must be present for 103", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "103")));
assertTrue("Inflight Compaction must be present for 103", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "103")));
}
@Test
@@ -281,6 +398,53 @@ public class TestHoodieCommitArchiveLog {
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103")));
}
@Test
public void testArchiveCommitCompactionNoHole() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf());
HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "101", dfs.getConf());
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "101"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf());
HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "104", dfs.getConf());
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "104"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "106", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "107", dfs.getConf());
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline();
assertEquals("Loaded 6 commits and the count should match", 8, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired(jsc);
assertTrue(result);
timeline = metaClient.getActiveTimeline().reload().getCommitsAndCompactionTimeline();
assertFalse("Instants before oldest pending compaction can be removed",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "100")));
assertEquals(
"Since we have a pending compaction at 101, we should never archive any commit "
+ "after 101 (we only " + "archive 100)", 7, timeline.countInstants());
assertTrue("Requested Compaction must still be present",
timeline.containsInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "102")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "105")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "106")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "107")));
}
private void verifyInflightInstants(HoodieTableMetaClient metaClient, int expectedTotalInstants) {
HoodieTimeline timeline = metaClient.getActiveTimeline().reload()
.getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION)).filterInflights();