[HUDI-3713] Guarding archival for multi-writer (#5138)
This commit is contained in:
committed by
GitHub
parent
f6ff95f97c
commit
4569734d60
@@ -214,7 +214,8 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
try {
|
||||
preCommit(inflightInstant, metadata);
|
||||
commit(table, commitActionType, instantTime, metadata, stats);
|
||||
postCommit(table, metadata, instantTime, extraMetadata);
|
||||
// already within lock, and so no lock requried for archival
|
||||
postCommit(table, metadata, instantTime, extraMetadata, false);
|
||||
LOG.info("Committed " + instantTime);
|
||||
releaseResources();
|
||||
} catch (IOException e) {
|
||||
@@ -474,14 +475,16 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
* @param metadata Commit Metadata corresponding to committed instant
|
||||
* @param instantTime Instant Time
|
||||
* @param extraMetadata Additional Metadata passed by user
|
||||
* @param acquireLockForArchival true if lock has to be acquired for archival. false otherwise.
|
||||
*/
|
||||
protected void postCommit(HoodieTable table, HoodieCommitMetadata metadata, String instantTime, Option<Map<String, String>> extraMetadata) {
|
||||
protected void postCommit(HoodieTable table, HoodieCommitMetadata metadata, String instantTime, Option<Map<String, String>> extraMetadata,
|
||||
boolean acquireLockForArchival) {
|
||||
try {
|
||||
// Delete the marker directory for the instant.
|
||||
WriteMarkersFactory.get(config.getMarkersType(), table, instantTime)
|
||||
.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
|
||||
autoCleanOnCommit();
|
||||
autoArchiveOnCommit(table);
|
||||
autoArchiveOnCommit(table, acquireLockForArchival);
|
||||
} finally {
|
||||
this.heartbeatClient.stop(instantTime);
|
||||
}
|
||||
@@ -565,7 +568,7 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
}
|
||||
}
|
||||
|
||||
protected void autoArchiveOnCommit(HoodieTable table) {
|
||||
protected void autoArchiveOnCommit(HoodieTable table, boolean acquireLockForArchival) {
|
||||
if (!config.isAutoArchive()) {
|
||||
return;
|
||||
}
|
||||
@@ -576,7 +579,7 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
LOG.info("Async archiver has finished");
|
||||
} else {
|
||||
LOG.info("Start to archive synchronously.");
|
||||
archive(table);
|
||||
archive(table, acquireLockForArchival);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -845,15 +848,16 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
* Trigger archival for the table. This ensures that the number of commits do not explode
|
||||
* and keep increasing unbounded over time.
|
||||
* @param table table to commit on.
|
||||
* @param acquireLockForArchival true if lock has to be acquired for archival. false otherwise.
|
||||
*/
|
||||
protected void archive(HoodieTable table) {
|
||||
protected void archive(HoodieTable table, boolean acquireLockForArchival) {
|
||||
if (!tableServicesEnabled(config)) {
|
||||
return;
|
||||
}
|
||||
try {
|
||||
// We cannot have unbounded commit files. Archive commits if we have to archive
|
||||
HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(config, table);
|
||||
archiver.archiveIfRequired(context);
|
||||
archiver.archiveIfRequired(context, acquireLockForArchival);
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException("Failed to archive", ioe);
|
||||
}
|
||||
@@ -866,7 +870,7 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
public void archive() {
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable table = createTable(config, hadoopConf);
|
||||
archive(table);
|
||||
archive(table, true);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -21,6 +21,7 @@ package org.apache.hudi.client;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
|
||||
import org.apache.hudi.avro.model.HoodieMergeArchiveFilePlan;
|
||||
import org.apache.hudi.client.transaction.TransactionManager;
|
||||
import org.apache.hudi.client.utils.MetadataConversionUtils;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
@@ -71,6 +72,7 @@ import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
@@ -96,6 +98,7 @@ public class HoodieTimelineArchiver<T extends HoodieAvroPayload, I, K, O> {
|
||||
private final int minInstantsToKeep;
|
||||
private final HoodieTable<T, I, K, O> table;
|
||||
private final HoodieTableMetaClient metaClient;
|
||||
private final TransactionManager txnManager;
|
||||
|
||||
public HoodieTimelineArchiver(HoodieWriteConfig config, HoodieTable<T, I, K, O> table) {
|
||||
this.config = config;
|
||||
@@ -104,6 +107,7 @@ public class HoodieTimelineArchiver<T extends HoodieAvroPayload, I, K, O> {
|
||||
this.archiveFilePath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath());
|
||||
this.maxInstantsToKeep = config.getMaxCommitsToKeep();
|
||||
this.minInstantsToKeep = config.getMinCommitsToKeep();
|
||||
this.txnManager = new TransactionManager(config, table.getMetaClient().getFs());
|
||||
}
|
||||
|
||||
private Writer openWriter() {
|
||||
@@ -143,11 +147,18 @@ public class HoodieTimelineArchiver<T extends HoodieAvroPayload, I, K, O> {
|
||||
}
|
||||
}
|
||||
|
||||
public boolean archiveIfRequired(HoodieEngineContext context) throws IOException {
|
||||
return archiveIfRequired(context, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if commits need to be archived. If yes, archive commits.
|
||||
*/
|
||||
public boolean archiveIfRequired(HoodieEngineContext context) throws IOException {
|
||||
public boolean archiveIfRequired(HoodieEngineContext context, boolean acquireLock) throws IOException {
|
||||
try {
|
||||
if (acquireLock) {
|
||||
txnManager.beginTransaction();
|
||||
}
|
||||
List<HoodieInstant> instantsToArchive = getInstantsToArchive().collect(Collectors.toList());
|
||||
verifyLastMergeArchiveFilesIfNecessary(context);
|
||||
boolean success = true;
|
||||
@@ -167,6 +178,9 @@ public class HoodieTimelineArchiver<T extends HoodieAvroPayload, I, K, O> {
|
||||
return success;
|
||||
} finally {
|
||||
close();
|
||||
if (acquireLock) {
|
||||
txnManager.endTransaction();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -485,9 +499,16 @@ public class HoodieTimelineArchiver<T extends HoodieAvroPayload, I, K, O> {
|
||||
}
|
||||
}
|
||||
|
||||
return instants.flatMap(hoodieInstant ->
|
||||
groupByTsAction.get(Pair.of(hoodieInstant.getTimestamp(),
|
||||
HoodieInstant.getComparableAction(hoodieInstant.getAction()))).stream());
|
||||
return instants.flatMap(hoodieInstant -> {
|
||||
List<HoodieInstant> instantsToStream = groupByTsAction.get(Pair.of(hoodieInstant.getTimestamp(),
|
||||
HoodieInstant.getComparableAction(hoodieInstant.getAction())));
|
||||
if (instantsToStream != null) {
|
||||
return instantsToStream.stream();
|
||||
} else {
|
||||
// if a concurrent writer archived the instant
|
||||
return Collections.EMPTY_LIST.stream();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants, HoodieEngineContext context) throws IOException {
|
||||
|
||||
Reference in New Issue
Block a user