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:
committed by
vinoth chandar
parent
0cd186c899
commit
cf7f7aabb9
@@ -91,7 +91,6 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
private final HoodieWriteConfig config;
|
||||
private transient final HoodieMetrics metrics;
|
||||
private transient final HoodieIndex<T> index;
|
||||
private transient final HoodieCommitArchiveLog archiveLog;
|
||||
private transient Timer.Context writeContext = null;
|
||||
|
||||
/**
|
||||
@@ -116,7 +115,6 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
this.config = clientConfig;
|
||||
this.index = HoodieIndex.createIndex(config, jsc);
|
||||
this.metrics = new HoodieMetrics(config, config.getTableName());
|
||||
this.archiveLog = new HoodieCommitArchiveLog(clientConfig, fs);
|
||||
|
||||
if (rollbackInFlight) {
|
||||
rollbackInflightCommits();
|
||||
@@ -446,6 +444,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
|
||||
// We cannot have unbounded commit files. Archive commits if we have to archive
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(config,
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true));
|
||||
archiveLog.archiveIfRequired();
|
||||
if (config.isAutoClean()) {
|
||||
// Call clean to cleanup if there is anything to cleanup after the commit,
|
||||
|
||||
@@ -40,7 +40,6 @@ import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
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.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -60,15 +59,14 @@ public class HoodieCommitArchiveLog {
|
||||
private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class);
|
||||
|
||||
private final Path archiveFilePath;
|
||||
private final FileSystem fs;
|
||||
private final HoodieTableMetaClient metaClient;
|
||||
private final HoodieWriteConfig config;
|
||||
private HoodieLogFormat.Writer writer;
|
||||
|
||||
public HoodieCommitArchiveLog(HoodieWriteConfig config, FileSystem fs) {
|
||||
this.fs = fs;
|
||||
public HoodieCommitArchiveLog(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
|
||||
this.config = config;
|
||||
this.archiveFilePath = HoodieArchivedTimeline
|
||||
.getArchiveLogPath(config.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME);
|
||||
this.metaClient = metaClient;
|
||||
this.archiveFilePath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath());
|
||||
}
|
||||
|
||||
private HoodieLogFormat.Writer openWriter() {
|
||||
@@ -78,7 +76,7 @@ public class HoodieCommitArchiveLog {
|
||||
.onParentPath(archiveFilePath.getParent())
|
||||
.withFileId(archiveFilePath.getName())
|
||||
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION)
|
||||
.withFs(fs)
|
||||
.withFs(metaClient.getFs())
|
||||
.overBaseCommit("").build();
|
||||
} else {
|
||||
return this.writer;
|
||||
@@ -125,9 +123,7 @@ public class HoodieCommitArchiveLog {
|
||||
int maxCommitsToKeep = config.getMaxCommitsToKeep();
|
||||
int minCommitsToKeep = config.getMinCommitsToKeep();
|
||||
|
||||
HoodieTable table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true),
|
||||
config);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
|
||||
|
||||
// GroupBy each action and limit each action timeline to maxCommitsToKeep
|
||||
HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline()
|
||||
@@ -165,16 +161,13 @@ public class HoodieCommitArchiveLog {
|
||||
|
||||
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) {
|
||||
log.info("Deleting instants " + archivedInstants);
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true);
|
||||
|
||||
boolean success = true;
|
||||
for (HoodieInstant archivedInstant : archivedInstants) {
|
||||
Path commitFile =
|
||||
new Path(metaClient.getMetaPath(), archivedInstant.getFileName());
|
||||
try {
|
||||
if (fs.exists(commitFile)) {
|
||||
success &= fs.delete(commitFile, false);
|
||||
if (metaClient.getFs().exists(commitFile)) {
|
||||
success &= metaClient.getFs().delete(commitFile, false);
|
||||
log.info("Archived and deleted instant file " + commitFile);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
@@ -186,13 +179,9 @@ public class HoodieCommitArchiveLog {
|
||||
}
|
||||
|
||||
public void archive(List<HoodieInstant> instants) throws HoodieCommitException {
|
||||
|
||||
try {
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true);
|
||||
HoodieTimeline commitTimeline =
|
||||
metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants();
|
||||
|
||||
Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema();
|
||||
log.info("Wrapper schema " + wrapperSchema.toString());
|
||||
List<IndexedRecord> records = new ArrayList<>();
|
||||
|
||||
@@ -24,9 +24,10 @@ import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import com.uber.hoodie.index.HoodieIndex.IndexType;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -92,7 +93,8 @@ public class HoodieClientExample {
|
||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.forTable(tableName).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
HoodieIndexConfig.newBuilder().withIndexType(IndexType.BLOOM).build())
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 3).build())
|
||||
.build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
|
||||
|
||||
@@ -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());
|
||||
|
||||
Reference in New Issue
Block a user