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

@@ -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,

View File

@@ -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<>();