[HUDI-845] Added locking capability to allow multiple writers (#2374)
* [HUDI-845] Added locking capability to allow multiple writers 1. Added LockProvider API for pluggable lock methodologies 2. Added Resolution Strategy API to allow for pluggable conflict resolution 3. Added TableService client API to schedule table services 4. Added Transaction Manager for wrapping actions within transactions
This commit is contained in:
@@ -22,6 +22,7 @@ import com.codahale.metrics.Timer;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.avro.model.HoodieClusteringPlan;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
||||
@@ -31,15 +32,20 @@ import org.apache.hudi.callback.common.HoodieWriteCommitCallbackMessage;
|
||||
import org.apache.hudi.callback.util.HoodieCommitCallbackFactory;
|
||||
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
||||
import org.apache.hudi.client.heartbeat.HeartbeatUtils;
|
||||
import org.apache.hudi.client.transaction.TransactionManager;
|
||||
import org.apache.hudi.client.utils.TransactionUtils;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.TableServiceType;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CleanerUtils;
|
||||
import org.apache.hudi.common.util.ClusteringUtils;
|
||||
@@ -101,6 +107,8 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
private transient WriteOperationType operationType;
|
||||
private transient HoodieWriteCommitCallback commitCallback;
|
||||
protected transient AsyncCleanerService asyncCleanerService;
|
||||
protected final TransactionManager txnManager;
|
||||
protected Option<Pair<HoodieInstant, Map<String, String>>> lastCompletedTxnAndMetadata = Option.empty();
|
||||
|
||||
/**
|
||||
* Create a write client, with new hudi index.
|
||||
@@ -124,6 +132,7 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
super(context, writeConfig, timelineService);
|
||||
this.metrics = new HoodieMetrics(config, config.getTableName());
|
||||
this.index = createIndex(writeConfig);
|
||||
this.txnManager = new TransactionManager(config, fs);
|
||||
}
|
||||
|
||||
protected abstract HoodieIndex<T, I, K, O> createIndex(HoodieWriteConfig writeConfig);
|
||||
@@ -163,26 +172,28 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
|
||||
public boolean commitStats(String instantTime, List<HoodieWriteStat> stats, Option<Map<String, String>> extraMetadata,
|
||||
String commitActionType, Map<String, List<String>> partitionToReplaceFileIds) {
|
||||
LOG.info("Committing " + instantTime + " action " + commitActionType);
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable table = createTable(config, hadoopConf);
|
||||
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
HoodieCommitMetadata metadata = CommitUtils.buildMetadata(stats, partitionToReplaceFileIds, extraMetadata, operationType, config.getSchema(), commitActionType);
|
||||
// Finalize write
|
||||
finalizeWrite(table, instantTime, stats);
|
||||
HeartbeatUtils.abortIfHeartbeatExpired(instantTime, table, heartbeatClient, config);
|
||||
this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, table.getMetaClient().getCommitActionType(), instantTime)),
|
||||
lastCompletedTxnAndMetadata.isPresent() ? Option.of(lastCompletedTxnAndMetadata.get().getLeft()) : Option.empty());
|
||||
try {
|
||||
activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime),
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
preCommit(instantTime, metadata);
|
||||
commit(table, commitActionType, instantTime, metadata, stats);
|
||||
postCommit(table, metadata, instantTime, extraMetadata);
|
||||
emitCommitMetrics(instantTime, metadata, commitActionType);
|
||||
LOG.info("Committed " + instantTime);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime,
|
||||
e);
|
||||
throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, e);
|
||||
} finally {
|
||||
this.txnManager.endTransaction();
|
||||
}
|
||||
|
||||
// do this outside of lock since compaction, clustering can be time taking and we don't need a lock for the entire execution period
|
||||
runTableServicesInline(table, metadata, extraMetadata);
|
||||
emitCommitMetrics(instantTime, metadata, commitActionType);
|
||||
// callback if needed.
|
||||
if (config.writeCommitCallbackOn()) {
|
||||
if (null == commitCallback) {
|
||||
@@ -193,6 +204,16 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
return true;
|
||||
}
|
||||
|
||||
protected void commit(HoodieTable table, String commitActionType, String instantTime, HoodieCommitMetadata metadata,
|
||||
List<HoodieWriteStat> stats) throws IOException {
|
||||
LOG.info("Committing " + instantTime + " action " + commitActionType);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
// Finalize write
|
||||
finalizeWrite(table, instantTime, stats);
|
||||
activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime),
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
}
|
||||
|
||||
protected abstract HoodieTable<T, I, K, O> createTable(HoodieWriteConfig config, Configuration hadoopConf);
|
||||
|
||||
void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String actionType) {
|
||||
@@ -210,6 +231,11 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
}
|
||||
}
|
||||
|
||||
protected void preCommit(String instantTime, HoodieCommitMetadata metadata) {
|
||||
// no-op
|
||||
// TODO : Conflict resolution is not supported for Flink & Java engines
|
||||
}
|
||||
|
||||
protected void syncTableMetadata() {
|
||||
// no-op
|
||||
}
|
||||
@@ -227,6 +253,9 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
*/
|
||||
public void bootstrap(Option<Map<String, String>> extraMetadata) {
|
||||
// TODO : MULTIWRITER -> check if failed bootstrap files can be cleaned later
|
||||
if (config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) {
|
||||
throw new HoodieException("Cannot bootstrap the table in multi-writer mode");
|
||||
}
|
||||
HoodieTable<T, I, K, O> table = getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS);
|
||||
rollbackFailedBootstrap();
|
||||
table.bootstrap(context, extraMetadata);
|
||||
@@ -359,10 +388,20 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
* Common method containing steps to be performed before write (upsert/insert/...
|
||||
* @param instantTime
|
||||
* @param writeOperationType
|
||||
* @param metaClient
|
||||
*/
|
||||
protected void preWrite(String instantTime, WriteOperationType writeOperationType) {
|
||||
protected void preWrite(String instantTime, WriteOperationType writeOperationType,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
setOperationType(writeOperationType);
|
||||
syncTableMetadata();
|
||||
this.lastCompletedTxnAndMetadata = TransactionUtils.getLastCompletedTxnInstantAndMetadata(metaClient);
|
||||
this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, metaClient.getCommitActionType(), instantTime)), lastCompletedTxnAndMetadata
|
||||
.isPresent()
|
||||
? Option.of(lastCompletedTxnAndMetadata.get().getLeft()) : Option.empty());
|
||||
try {
|
||||
syncTableMetadata();
|
||||
} finally {
|
||||
this.txnManager.endTransaction();
|
||||
}
|
||||
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this);
|
||||
}
|
||||
|
||||
@@ -385,28 +424,8 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
*/
|
||||
protected void postCommit(HoodieTable<T, I, K, O> table, HoodieCommitMetadata metadata, String instantTime, Option<Map<String, String>> extraMetadata) {
|
||||
try {
|
||||
|
||||
// Delete the marker directory for the instant.
|
||||
new MarkerFiles(table, instantTime).quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
|
||||
|
||||
// Do an inline compaction if enabled
|
||||
if (config.isInlineCompaction()) {
|
||||
runAnyPendingCompactions(table);
|
||||
metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true");
|
||||
inlineCompact(extraMetadata);
|
||||
} else {
|
||||
metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "false");
|
||||
}
|
||||
|
||||
// Do an inline clustering if enabled
|
||||
if (config.isInlineClustering()) {
|
||||
runAnyPendingClustering(table);
|
||||
metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "true");
|
||||
inlineCluster(extraMetadata);
|
||||
} else {
|
||||
metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "false");
|
||||
}
|
||||
|
||||
// We cannot have unbounded commit files. Archive commits if we have to archive
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table);
|
||||
archiveLog.archiveIfRequired(context);
|
||||
@@ -419,6 +438,28 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
}
|
||||
}
|
||||
|
||||
protected void runTableServicesInline(HoodieTable<T, I, K, O> table, HoodieCommitMetadata metadata, Option<Map<String, String>> extraMetadata) {
|
||||
if (config.inlineTableServices()) {
|
||||
// Do an inline compaction if enabled
|
||||
if (config.inlineCompactionEnabled()) {
|
||||
runAnyPendingCompactions(table);
|
||||
metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true");
|
||||
inlineCompact(extraMetadata);
|
||||
} else {
|
||||
metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "false");
|
||||
}
|
||||
|
||||
// Do an inline clustering if enabled
|
||||
if (config.inlineClusteringEnabled()) {
|
||||
runAnyPendingClustering(table);
|
||||
metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "true");
|
||||
inlineCluster(extraMetadata);
|
||||
} else {
|
||||
metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "false");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected void runAnyPendingCompactions(HoodieTable<T, I, K, O> table) {
|
||||
table.getActiveTimeline().getWriteTimeline().filterPendingCompactionTimeline().getInstants()
|
||||
.forEach(instant -> {
|
||||
@@ -587,6 +628,20 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
* cleaned)
|
||||
*/
|
||||
public HoodieCleanMetadata clean(String cleanInstantTime) throws HoodieIOException {
|
||||
return clean(cleanInstantTime, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clean up any stale/old files/data lying around (either on file storage or index storage) based on the
|
||||
* configurations and CleaningPolicy used. (typically files that no longer can be used by a running query can be
|
||||
* cleaned). This API provides the flexibility to schedule clean instant asynchronously via
|
||||
* {@link AbstractHoodieWriteClient#scheduleTableService(String, Option, TableServiceType)} and disable inline scheduling
|
||||
* of clean.
|
||||
*/
|
||||
public HoodieCleanMetadata clean(String cleanInstantTime, boolean scheduleInline) throws HoodieIOException {
|
||||
if (scheduleInline) {
|
||||
scheduleCleaningAtInstant(cleanInstantTime, Option.empty());
|
||||
}
|
||||
LOG.info("Cleaner started");
|
||||
final Timer.Context timerContext = metrics.getCleanCtx();
|
||||
LOG.info("Cleaned failed attempts if any");
|
||||
@@ -663,7 +718,6 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
|
||||
/**
|
||||
* Schedules a new compaction instant.
|
||||
*
|
||||
* @param extraMetadata Extra Metadata to be stored
|
||||
*/
|
||||
public Option<String> scheduleCompaction(Option<Map<String, String>> extraMetadata) throws HoodieIOException {
|
||||
@@ -673,15 +727,11 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
|
||||
/**
|
||||
* Schedules a new compaction instant with passed-in instant time.
|
||||
*
|
||||
* @param instantTime Compaction Instant Time
|
||||
* @param extraMetadata Extra Metadata to be stored
|
||||
*/
|
||||
public boolean scheduleCompactionAtInstant(String instantTime, Option<Map<String, String>> extraMetadata) throws HoodieIOException {
|
||||
LOG.info("Scheduling compaction at instant time :" + instantTime);
|
||||
Option<HoodieCompactionPlan> plan = createTable(config, hadoopConf)
|
||||
.scheduleCompaction(context, instantTime, extraMetadata);
|
||||
return plan.isPresent();
|
||||
return scheduleTableService(instantTime, extraMetadata, TableServiceType.COMPACT).isPresent();
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -723,14 +773,14 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
|
||||
/**
|
||||
* Get inflight time line exclude compaction and clustering.
|
||||
* @param table
|
||||
* @param metaClient
|
||||
* @return
|
||||
*/
|
||||
private HoodieTimeline getInflightTimelineExcludeCompactionAndClustering(HoodieTable<T, I, K, O> table) {
|
||||
HoodieTimeline inflightTimelineWithReplaceCommit = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
|
||||
private HoodieTimeline getInflightTimelineExcludeCompactionAndClustering(HoodieTableMetaClient metaClient) {
|
||||
HoodieTimeline inflightTimelineWithReplaceCommit = metaClient.getCommitsTimeline().filterPendingExcludingCompaction();
|
||||
HoodieTimeline inflightTimelineExcludeClusteringCommit = inflightTimelineWithReplaceCommit.filter(instant -> {
|
||||
if (instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) {
|
||||
Option<Pair<HoodieInstant, HoodieClusteringPlan>> instantPlan = ClusteringUtils.getClusteringPlan(table.getMetaClient(), instant);
|
||||
Option<Pair<HoodieInstant, HoodieClusteringPlan>> instantPlan = ClusteringUtils.getClusteringPlan(metaClient, instant);
|
||||
return !instantPlan.isPresent();
|
||||
} else {
|
||||
return true;
|
||||
@@ -744,7 +794,12 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
*/
|
||||
public Boolean rollbackFailedWrites() {
|
||||
HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
|
||||
List<String> instantsToRollback = getInstantsToRollback(table);
|
||||
List<String> instantsToRollback = getInstantsToRollback(table.getMetaClient(), config.getFailedWritesCleanPolicy());
|
||||
rollbackFailedWrites(instantsToRollback);
|
||||
return true;
|
||||
}
|
||||
|
||||
protected void rollbackFailedWrites(List<String> instantsToRollback) {
|
||||
for (String instant : instantsToRollback) {
|
||||
if (HoodieTimeline.compareTimestamps(instant, HoodieTimeline.LESSER_THAN_OR_EQUALS,
|
||||
HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) {
|
||||
@@ -761,15 +816,14 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
} catch (IOException io) {
|
||||
LOG.error("Unable to delete heartbeat files", io);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private List<String> getInstantsToRollback(HoodieTable<T, I, K, O> table) {
|
||||
Stream<HoodieInstant> inflightInstantsStream = getInflightTimelineExcludeCompactionAndClustering(table)
|
||||
protected List<String> getInstantsToRollback(HoodieTableMetaClient metaClient, HoodieFailedWritesCleaningPolicy cleaningPolicy) {
|
||||
Stream<HoodieInstant> inflightInstantsStream = getInflightTimelineExcludeCompactionAndClustering(metaClient)
|
||||
.getReverseOrderedInstants();
|
||||
if (config.getFailedWritesCleanPolicy().isEager()) {
|
||||
if (cleaningPolicy.isEager()) {
|
||||
return inflightInstantsStream.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
} else if (config.getFailedWritesCleanPolicy().isLazy()) {
|
||||
} else if (cleaningPolicy.isLazy()) {
|
||||
return inflightInstantsStream.filter(instant -> {
|
||||
try {
|
||||
return heartbeatClient.isHeartbeatExpired(instant.getTimestamp());
|
||||
@@ -777,7 +831,7 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
throw new HoodieException("Failed to check heartbeat for instant " + instant, io);
|
||||
}
|
||||
}).map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
} else if (config.getFailedWritesCleanPolicy().isNever()) {
|
||||
} else if (cleaningPolicy.isNever()) {
|
||||
return Collections.EMPTY_LIST;
|
||||
} else {
|
||||
throw new IllegalArgumentException("Invalid Failed Writes Cleaning Policy " + config.getFailedWritesCleanPolicy());
|
||||
@@ -797,16 +851,15 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
*/
|
||||
protected Option<String> inlineCompact(Option<Map<String, String>> extraMetadata) {
|
||||
Option<String> compactionInstantTimeOpt = scheduleCompaction(extraMetadata);
|
||||
compactionInstantTimeOpt.ifPresent(compactionInstantTime -> {
|
||||
compactionInstantTimeOpt.ifPresent(compactInstantTime -> {
|
||||
// inline compaction should auto commit as the user is never given control
|
||||
compact(compactionInstantTime, true);
|
||||
compact(compactInstantTime, true);
|
||||
});
|
||||
return compactionInstantTimeOpt;
|
||||
}
|
||||
|
||||
/**
|
||||
* Schedules a new clustering instant.
|
||||
*
|
||||
* @param extraMetadata Extra Metadata to be stored
|
||||
*/
|
||||
public Option<String> scheduleClustering(Option<Map<String, String>> extraMetadata) throws HoodieIOException {
|
||||
@@ -816,25 +869,93 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
|
||||
/**
|
||||
* Schedules a new clustering instant with passed-in instant time.
|
||||
*
|
||||
* @param instantTime clustering Instant Time
|
||||
* @param extraMetadata Extra Metadata to be stored
|
||||
*/
|
||||
public boolean scheduleClusteringAtInstant(String instantTime, Option<Map<String, String>> extraMetadata) throws HoodieIOException {
|
||||
LOG.info("Scheduling clustering at instant time :" + instantTime);
|
||||
Option<HoodieClusteringPlan> plan = createTable(config, hadoopConf)
|
||||
.scheduleClustering(context, instantTime, extraMetadata);
|
||||
return plan.isPresent();
|
||||
return scheduleTableService(instantTime, extraMetadata, TableServiceType.CLUSTER).isPresent();
|
||||
}
|
||||
|
||||
/**
|
||||
* Schedules a new cleaning instant.
|
||||
* @param extraMetadata Extra Metadata to be stored
|
||||
*/
|
||||
protected Option<String> scheduleCleaning(Option<Map<String, String>> extraMetadata) throws HoodieIOException {
|
||||
String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
return scheduleCleaningAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Schedules a new cleaning instant with passed-in instant time.
|
||||
* @param instantTime cleaning Instant Time
|
||||
* @param extraMetadata Extra Metadata to be stored
|
||||
*/
|
||||
protected boolean scheduleCleaningAtInstant(String instantTime, Option<Map<String, String>> extraMetadata) throws HoodieIOException {
|
||||
return scheduleTableService(instantTime, extraMetadata, TableServiceType.CLEAN).isPresent();
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensures clustering instant is in expected state and performs clustering for the plan stored in metadata.
|
||||
*
|
||||
* @param clusteringInstant Clustering Instant Time
|
||||
* @return Collection of Write Status
|
||||
*/
|
||||
public abstract HoodieWriteMetadata<O> cluster(String clusteringInstant, boolean shouldComplete);
|
||||
|
||||
/**
|
||||
* Schedule table services such as clustering, compaction & cleaning.
|
||||
*
|
||||
* @param extraMetadata Metadata to pass onto the scheduled service instant
|
||||
* @param tableServiceType Type of table service to schedule
|
||||
* @return
|
||||
*/
|
||||
public Option<String> scheduleTableService(Option<Map<String, String>> extraMetadata, TableServiceType tableServiceType) {
|
||||
String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
return scheduleTableService(instantTime, extraMetadata, tableServiceType);
|
||||
}
|
||||
|
||||
/**
|
||||
* Schedule table services such as clustering, compaction & cleaning.
|
||||
*
|
||||
* @param extraMetadata Metadata to pass onto the scheduled service instant
|
||||
* @param tableServiceType Type of table service to schedule
|
||||
* @return
|
||||
*/
|
||||
public Option<String> scheduleTableService(String instantTime, Option<Map<String, String>> extraMetadata,
|
||||
TableServiceType tableServiceType) {
|
||||
// A lock is required to guard against race conditions between an on-going writer and scheduling a table service.
|
||||
try {
|
||||
this.txnManager.beginTransaction(Option.of(new HoodieInstant(HoodieInstant.State.REQUESTED,
|
||||
tableServiceType.getAction(), instantTime)), Option.empty());
|
||||
LOG.info("Scheduling table service " + tableServiceType);
|
||||
return scheduleTableServiceInternal(instantTime, extraMetadata, tableServiceType);
|
||||
} finally {
|
||||
this.txnManager.endTransaction();
|
||||
}
|
||||
}
|
||||
|
||||
private Option<String> scheduleTableServiceInternal(String instantTime, Option<Map<String, String>> extraMetadata,
|
||||
TableServiceType tableServiceType) {
|
||||
switch (tableServiceType) {
|
||||
case CLUSTER:
|
||||
LOG.info("Scheduling clustering at instant time :" + instantTime);
|
||||
Option<HoodieClusteringPlan> clusteringPlan = createTable(config, hadoopConf)
|
||||
.scheduleClustering(context, instantTime, extraMetadata);
|
||||
return clusteringPlan.isPresent() ? Option.of(instantTime) : Option.empty();
|
||||
case COMPACT:
|
||||
LOG.info("Scheduling compaction at instant time :" + instantTime);
|
||||
Option<HoodieCompactionPlan> compactionPlan = createTable(config, hadoopConf)
|
||||
.scheduleCompaction(context, instantTime, extraMetadata);
|
||||
return compactionPlan.isPresent() ? Option.of(instantTime) : Option.empty();
|
||||
case CLEAN:
|
||||
LOG.info("Scheduling cleaning at instant time :" + instantTime);
|
||||
Option<HoodieCleanerPlan> cleanerPlan = createTable(config, hadoopConf)
|
||||
.scheduleCleaning(context, instantTime, extraMetadata);
|
||||
return cleanerPlan.isPresent() ? Option.of(instantTime) : Option.empty();
|
||||
default:
|
||||
throw new IllegalArgumentException("Invalid TableService " + tableServiceType);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Executes a clustering plan on a table, serially before or after an insert/upsert action.
|
||||
*/
|
||||
@@ -923,12 +1044,12 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
// release AsyncCleanerService
|
||||
AsyncCleanerService.forceShutdown(asyncCleanerService);
|
||||
asyncCleanerService = null;
|
||||
|
||||
// Stop timeline-server if running
|
||||
super.close();
|
||||
// Calling this here releases any resources used by your index, so make sure to finish any related operations
|
||||
// before this point
|
||||
this.index.close();
|
||||
this.heartbeatClient.stop();
|
||||
this.txnManager.close();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,153 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client.transaction;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
|
||||
import org.apache.hudi.client.utils.MetadataConversionUtils;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieMetadataWrapper;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.CommitUtils;
|
||||
import java.util.Collections;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION;
|
||||
import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION;
|
||||
import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION;
|
||||
import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION;
|
||||
|
||||
/**
|
||||
* This class is used to hold all information used to identify how to resolve conflicts between instants.
|
||||
* Since we interchange payload types between AVRO specific records and POJO's, this object serves as
|
||||
* a common payload to manage these conversions.
|
||||
*/
|
||||
public class ConcurrentOperation {
|
||||
|
||||
private WriteOperationType operationType;
|
||||
private final HoodieMetadataWrapper metadataWrapper;
|
||||
private final Option<HoodieCommitMetadata> commitMetadataOption;
|
||||
private final String actionState;
|
||||
private final String actionType;
|
||||
private final String instantTime;
|
||||
private Set<String> mutatedFileIds = Collections.EMPTY_SET;
|
||||
|
||||
public ConcurrentOperation(HoodieInstant instant, HoodieTableMetaClient metaClient) throws IOException {
|
||||
this.metadataWrapper = new HoodieMetadataWrapper(MetadataConversionUtils.createMetaWrapper(instant, metaClient));
|
||||
this.commitMetadataOption = Option.empty();
|
||||
this.actionState = instant.getState().name();
|
||||
this.actionType = instant.getAction();
|
||||
this.instantTime = instant.getTimestamp();
|
||||
init(instant);
|
||||
}
|
||||
|
||||
public ConcurrentOperation(HoodieInstant instant, HoodieCommitMetadata commitMetadata) {
|
||||
this.commitMetadataOption = Option.of(commitMetadata);
|
||||
this.metadataWrapper = new HoodieMetadataWrapper(commitMetadata);
|
||||
this.actionState = instant.getState().name();
|
||||
this.actionType = instant.getAction();
|
||||
this.instantTime = instant.getTimestamp();
|
||||
init(instant);
|
||||
}
|
||||
|
||||
public String getInstantActionState() {
|
||||
return actionState;
|
||||
}
|
||||
|
||||
public String getInstantActionType() {
|
||||
return actionType;
|
||||
}
|
||||
|
||||
public String getInstantTimestamp() {
|
||||
return instantTime;
|
||||
}
|
||||
|
||||
public WriteOperationType getOperationType() {
|
||||
return operationType;
|
||||
}
|
||||
|
||||
public Set<String> getMutatedFileIds() {
|
||||
return mutatedFileIds;
|
||||
}
|
||||
|
||||
public Option<HoodieCommitMetadata> getCommitMetadataOption() {
|
||||
return commitMetadataOption;
|
||||
}
|
||||
|
||||
private void init(HoodieInstant instant) {
|
||||
if (this.metadataWrapper.isAvroMetadata()) {
|
||||
switch (getInstantActionType()) {
|
||||
case COMPACTION_ACTION:
|
||||
this.operationType = WriteOperationType.COMPACT;
|
||||
this.mutatedFileIds = this.metadataWrapper.getMetadataFromTimeline().getHoodieCompactionPlan().getOperations()
|
||||
.stream()
|
||||
.map(op -> op.getFileId())
|
||||
.collect(Collectors.toSet());
|
||||
break;
|
||||
case COMMIT_ACTION:
|
||||
case DELTA_COMMIT_ACTION:
|
||||
this.mutatedFileIds = CommitUtils.getFileIdWithoutSuffixAndRelativePathsFromSpecificRecord(this.metadataWrapper.getMetadataFromTimeline().getHoodieCommitMetadata()
|
||||
.getPartitionToWriteStats()).keySet();
|
||||
this.operationType = WriteOperationType.fromValue(this.metadataWrapper.getMetadataFromTimeline().getHoodieCommitMetadata().getOperationType());
|
||||
break;
|
||||
case REPLACE_COMMIT_ACTION:
|
||||
if (instant.isCompleted()) {
|
||||
this.mutatedFileIds = CommitUtils.getFileIdWithoutSuffixAndRelativePathsFromSpecificRecord(
|
||||
this.metadataWrapper.getMetadataFromTimeline().getHoodieReplaceCommitMetadata().getPartitionToWriteStats()).keySet();
|
||||
this.operationType = WriteOperationType.fromValue(this.metadataWrapper.getMetadataFromTimeline().getHoodieReplaceCommitMetadata().getOperationType());
|
||||
} else {
|
||||
HoodieRequestedReplaceMetadata requestedReplaceMetadata = this.metadataWrapper.getMetadataFromTimeline().getHoodieRequestedReplaceMetadata();
|
||||
this.mutatedFileIds = requestedReplaceMetadata
|
||||
.getClusteringPlan().getInputGroups()
|
||||
.stream()
|
||||
.flatMap(ig -> ig.getSlices().stream())
|
||||
.map(file -> file.getFileId())
|
||||
.collect(Collectors.toSet());
|
||||
this.operationType = WriteOperationType.CLUSTER;
|
||||
}
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Unsupported Action Type " + getInstantActionType());
|
||||
}
|
||||
} else {
|
||||
switch (getInstantActionType()) {
|
||||
case COMMIT_ACTION:
|
||||
case DELTA_COMMIT_ACTION:
|
||||
this.mutatedFileIds = CommitUtils.getFileIdWithoutSuffixAndRelativePaths(this.metadataWrapper.getCommitMetadata().getPartitionToWriteStats()).keySet();
|
||||
this.operationType = this.metadataWrapper.getCommitMetadata().getOperationType();
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Unsupported Action Type " + getInstantActionType());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "{"
|
||||
+ "actionType=" + this.getInstantActionType()
|
||||
+ ", instantTime=" + this.getInstantTimestamp()
|
||||
+ ", actionState=" + this.getInstantActionState()
|
||||
+ '\'' + '}';
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,64 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client.transaction;
|
||||
|
||||
import org.apache.hudi.ApiMaturityLevel;
|
||||
import org.apache.hudi.PublicAPIMethod;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieWriteConflictException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Strategy interface for conflict resolution with multiple writers.
|
||||
* Users can provide pluggable implementations for different kinds of strategies to resolve conflicts when multiple
|
||||
* writers are mutating the hoodie table.
|
||||
*/
|
||||
public interface ConflictResolutionStrategy {
|
||||
|
||||
/**
|
||||
* Stream of instants to check conflicts against.
|
||||
* @return
|
||||
*/
|
||||
Stream<HoodieInstant> getCandidateInstants(HoodieActiveTimeline activeTimeline, HoodieInstant currentInstant, Option<HoodieInstant> lastSuccessfulInstant);
|
||||
|
||||
/**
|
||||
* Implementations of this method will determine whether a conflict exists between 2 commits.
|
||||
* @param thisOperation
|
||||
* @param otherOperation
|
||||
* @return
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
boolean hasConflict(ConcurrentOperation thisOperation, ConcurrentOperation otherOperation);
|
||||
|
||||
/**
|
||||
* Implementations of this method will determine how to resolve a conflict between 2 commits.
|
||||
* @param thisOperation
|
||||
* @param otherOperation
|
||||
* @return
|
||||
*/
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
Option<HoodieCommitMetadata> resolveConflict(HoodieTable table,
|
||||
ConcurrentOperation thisOperation, ConcurrentOperation otherOperation) throws HoodieWriteConflictException;
|
||||
|
||||
}
|
||||
@@ -0,0 +1,105 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client.transaction;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieWriteConflictException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.util.ConcurrentModificationException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION;
|
||||
import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION;
|
||||
|
||||
/**
|
||||
* This class is a basic implementation of a conflict resolution strategy for concurrent writes {@link ConflictResolutionStrategy}.
|
||||
*/
|
||||
public class SimpleConcurrentFileWritesConflictResolutionStrategy
|
||||
implements ConflictResolutionStrategy {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(SimpleConcurrentFileWritesConflictResolutionStrategy.class);
|
||||
|
||||
@Override
|
||||
public Stream<HoodieInstant> getCandidateInstants(HoodieActiveTimeline activeTimeline, HoodieInstant currentInstant,
|
||||
Option<HoodieInstant> lastSuccessfulInstant) {
|
||||
|
||||
// To find which instants are conflicting, we apply the following logic
|
||||
// 1. Get completed instants timeline only for commits that have happened since the last successful write.
|
||||
// 2. Get any scheduled or completed compaction or clustering operations that have started and/or finished
|
||||
// after the current instant. We need to check for write conflicts since they may have mutated the same files
|
||||
// that are being newly created by the current write.
|
||||
Stream<HoodieInstant> completedCommitsInstantStream = activeTimeline
|
||||
.getCommitsTimeline()
|
||||
.filterCompletedInstants()
|
||||
.findInstantsAfter(lastSuccessfulInstant.isPresent() ? lastSuccessfulInstant.get().getTimestamp() : HoodieTimeline.INIT_INSTANT_TS)
|
||||
.getInstants();
|
||||
|
||||
Stream<HoodieInstant> compactionAndClusteringPendingTimeline = activeTimeline
|
||||
.getTimelineOfActions(CollectionUtils.createSet(REPLACE_COMMIT_ACTION, COMPACTION_ACTION))
|
||||
.findInstantsAfter(currentInstant.getTimestamp())
|
||||
.filterInflightsAndRequested()
|
||||
.getInstants();
|
||||
return Stream.concat(completedCommitsInstantStream, compactionAndClusteringPendingTimeline);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasConflict(ConcurrentOperation thisOperation, ConcurrentOperation otherOperation) {
|
||||
// TODO : UUID's can clash even for insert/insert, handle that case.
|
||||
Set<String> fileIdsSetForFirstInstant = thisOperation.getMutatedFileIds();
|
||||
Set<String> fileIdsSetForSecondInstant = otherOperation.getMutatedFileIds();
|
||||
Set<String> intersection = new HashSet<>(fileIdsSetForFirstInstant);
|
||||
intersection.retainAll(fileIdsSetForSecondInstant);
|
||||
if (!intersection.isEmpty()) {
|
||||
LOG.info("Found conflicting writes between first operation = " + thisOperation
|
||||
+ ", second operation = " + otherOperation + " , intersecting file ids " + intersection);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<HoodieCommitMetadata> resolveConflict(HoodieTable table,
|
||||
ConcurrentOperation thisOperation, ConcurrentOperation otherOperation) {
|
||||
// A completed COMPACTION action eventually shows up as a COMMIT action on the timeline.
|
||||
// We need to ensure we handle this during conflict resolution and not treat the commit from a
|
||||
// compaction operation as a regular commit. Regular commits & deltacommits are candidates for conflict.
|
||||
// Since the REPLACE action with CLUSTER operation does not support concurrent updates, we have
|
||||
// to consider it as conflict if we see overlapping file ids. Once concurrent updates are
|
||||
// supported for CLUSTER (https://issues.apache.org/jira/browse/HUDI-1042),
|
||||
// add that to the below check so that concurrent updates do not conflict.
|
||||
if (otherOperation.getOperationType() == WriteOperationType.COMPACT
|
||||
&& HoodieTimeline.compareTimestamps(otherOperation.getInstantTimestamp(), HoodieTimeline.LESSER_THAN, thisOperation.getInstantTimestamp())) {
|
||||
return thisOperation.getCommitMetadataOption();
|
||||
}
|
||||
// just abort the current write if conflicts are found
|
||||
throw new HoodieWriteConflictException(new ConcurrentModificationException("Cannot resolve conflicts for overlapping writes"));
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,94 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client.transaction;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.client.transaction.lock.LockManager;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* This class allows clients to start and end transactions. Anything done between a start and end transaction is
|
||||
* guaranteed to be atomic.
|
||||
*/
|
||||
public class TransactionManager implements Serializable {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(TransactionManager.class);
|
||||
|
||||
private final LockManager lockManager;
|
||||
private Option<HoodieInstant> currentTxnOwnerInstant;
|
||||
private Option<HoodieInstant> lastCompletedTxnOwnerInstant;
|
||||
private boolean supportsOptimisticConcurrency;
|
||||
|
||||
public TransactionManager(HoodieWriteConfig config, FileSystem fs) {
|
||||
this.lockManager = new LockManager(config, fs);
|
||||
this.supportsOptimisticConcurrency = config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl();
|
||||
}
|
||||
|
||||
public synchronized void beginTransaction() {
|
||||
if (supportsOptimisticConcurrency) {
|
||||
LOG.info("Transaction starting without a transaction owner");
|
||||
lockManager.lock();
|
||||
LOG.info("Transaction started");
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized void beginTransaction(Option<HoodieInstant> currentTxnOwnerInstant, Option<HoodieInstant> lastCompletedTxnOwnerInstant) {
|
||||
if (supportsOptimisticConcurrency) {
|
||||
this.lastCompletedTxnOwnerInstant = lastCompletedTxnOwnerInstant;
|
||||
lockManager.setLatestCompletedWriteInstant(lastCompletedTxnOwnerInstant);
|
||||
LOG.info("Latest completed transaction instant " + lastCompletedTxnOwnerInstant);
|
||||
this.currentTxnOwnerInstant = currentTxnOwnerInstant;
|
||||
LOG.info("Transaction starting with transaction owner " + currentTxnOwnerInstant);
|
||||
lockManager.lock();
|
||||
LOG.info("Transaction started");
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized void endTransaction() {
|
||||
if (supportsOptimisticConcurrency) {
|
||||
LOG.info("Transaction ending with transaction owner " + currentTxnOwnerInstant);
|
||||
lockManager.unlock();
|
||||
LOG.info("Transaction ended");
|
||||
this.lastCompletedTxnOwnerInstant = Option.empty();
|
||||
lockManager.resetLatestCompletedWriteInstant();
|
||||
}
|
||||
}
|
||||
|
||||
public void close() {
|
||||
if (supportsOptimisticConcurrency) {
|
||||
lockManager.close();
|
||||
LOG.info("Transaction manager closed");
|
||||
}
|
||||
}
|
||||
|
||||
public Option<HoodieInstant> getLastCompletedTransactionOwner() {
|
||||
return lastCompletedTxnOwnerInstant;
|
||||
}
|
||||
|
||||
public Option<HoodieInstant> getCurrentTransactionOwner() {
|
||||
return currentTxnOwnerInstant;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,135 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client.transaction.lock;
|
||||
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.common.config.LockConfiguration;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.lock.LockProvider;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieLockException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* This class wraps implementations of {@link LockProvider} and provides an easy way to manage the lifecycle of a lock.
|
||||
*/
|
||||
public class LockManager implements Serializable, AutoCloseable {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(LockManager.class);
|
||||
private final HoodieWriteConfig writeConfig;
|
||||
private final LockConfiguration lockConfiguration;
|
||||
private final SerializableConfiguration hadoopConf;
|
||||
private volatile LockProvider lockProvider;
|
||||
// Holds the latest completed write instant to know which ones to check conflict against
|
||||
private final AtomicReference<Option<HoodieInstant>> latestCompletedWriteInstant;
|
||||
|
||||
public LockManager(HoodieWriteConfig writeConfig, FileSystem fs) {
|
||||
this.latestCompletedWriteInstant = new AtomicReference<>(Option.empty());
|
||||
this.writeConfig = writeConfig;
|
||||
this.hadoopConf = new SerializableConfiguration(fs.getConf());
|
||||
this.lockConfiguration = new LockConfiguration(writeConfig.getProps());
|
||||
}
|
||||
|
||||
public void lock() {
|
||||
if (writeConfig.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) {
|
||||
LockProvider lockProvider = getLockProvider();
|
||||
int retryCount = 0;
|
||||
boolean acquired = false;
|
||||
int retries = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP);
|
||||
long waitTimeInMs = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP);
|
||||
while (retryCount <= retries) {
|
||||
try {
|
||||
acquired = lockProvider.tryLock(writeConfig.getLockAcquireWaitTimeoutInMs(), TimeUnit.MILLISECONDS);
|
||||
if (acquired) {
|
||||
break;
|
||||
}
|
||||
LOG.info("Retrying to acquire lock...");
|
||||
Thread.sleep(waitTimeInMs);
|
||||
retryCount++;
|
||||
} catch (InterruptedException e) {
|
||||
if (retryCount >= retries) {
|
||||
throw new HoodieLockException("Unable to acquire lock, lock object ", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (!acquired) {
|
||||
throw new HoodieLockException("Unable to acquire lock, lock object " + lockProvider.getLock());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void unlock() {
|
||||
if (writeConfig.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) {
|
||||
getLockProvider().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized LockProvider getLockProvider() {
|
||||
// Perform lazy initialization of lock provider only if needed
|
||||
if (lockProvider == null) {
|
||||
LOG.info("Lock Provider " + writeConfig.getLockProviderClass());
|
||||
lockProvider = (LockProvider) ReflectionUtils.loadClass(writeConfig.getLockProviderClass(),
|
||||
lockConfiguration, hadoopConf.get());
|
||||
}
|
||||
return lockProvider;
|
||||
}
|
||||
|
||||
public void setLatestCompletedWriteInstant(Option<HoodieInstant> instant) {
|
||||
this.latestCompletedWriteInstant.set(instant);
|
||||
}
|
||||
|
||||
public void compareAndSetLatestCompletedWriteInstant(Option<HoodieInstant> expected, Option<HoodieInstant> newValue) {
|
||||
this.latestCompletedWriteInstant.compareAndSet(expected, newValue);
|
||||
}
|
||||
|
||||
public AtomicReference<Option<HoodieInstant>> getLatestCompletedWriteInstant() {
|
||||
return latestCompletedWriteInstant;
|
||||
}
|
||||
|
||||
public void resetLatestCompletedWriteInstant() {
|
||||
this.latestCompletedWriteInstant.set(Option.empty());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
closeQuietly();
|
||||
}
|
||||
|
||||
private void closeQuietly() {
|
||||
try {
|
||||
if (lockProvider != null) {
|
||||
lockProvider.close();
|
||||
LOG.info("Released connection created for acquiring lock");
|
||||
lockProvider = null;
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.error("Unable to close and release connection created for acquiring lock", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,168 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client.transaction.lock;
|
||||
|
||||
import org.apache.curator.framework.CuratorFramework;
|
||||
import org.apache.curator.framework.CuratorFrameworkFactory;
|
||||
import org.apache.curator.framework.imps.CuratorFrameworkState;
|
||||
import org.apache.curator.framework.recipes.locks.InterProcessMutex;
|
||||
import org.apache.curator.retry.BoundedExponentialBackoffRetry;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.common.config.LockConfiguration;
|
||||
import org.apache.hudi.common.lock.LockProvider;
|
||||
import org.apache.hudi.common.lock.LockState;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.exception.HoodieLockException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import javax.annotation.concurrent.NotThreadSafe;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_CONNECTION_TIMEOUT_MS;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_SESSION_TIMEOUT_MS;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP;
|
||||
|
||||
/**
|
||||
* A zookeeper based lock. This {@link LockProvider} implementation allows to lock table operations
|
||||
* using zookeeper. Users need to have a Zookeeper cluster deployed to be able to use this lock.
|
||||
*/
|
||||
@NotThreadSafe
|
||||
public class ZookeeperBasedLockProvider implements LockProvider<InterProcessMutex> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(ZookeeperBasedLockProvider.class);
|
||||
|
||||
private final CuratorFramework curatorFrameworkClient;
|
||||
private volatile InterProcessMutex lock = null;
|
||||
protected LockConfiguration lockConfiguration;
|
||||
|
||||
public ZookeeperBasedLockProvider(final LockConfiguration lockConfiguration, final Configuration conf) {
|
||||
checkRequiredProps(lockConfiguration);
|
||||
this.lockConfiguration = lockConfiguration;
|
||||
this.curatorFrameworkClient = CuratorFrameworkFactory.builder()
|
||||
.connectString(lockConfiguration.getConfig().getString(ZK_CONNECT_URL_PROP))
|
||||
.retryPolicy(new BoundedExponentialBackoffRetry(lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP),
|
||||
5000, lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP)))
|
||||
.sessionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_SESSION_TIMEOUT_MS_PROP, DEFAULT_ZK_SESSION_TIMEOUT_MS))
|
||||
.connectionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_CONNECTION_TIMEOUT_MS_PROP, DEFAULT_ZK_CONNECTION_TIMEOUT_MS))
|
||||
.build();
|
||||
this.curatorFrameworkClient.start();
|
||||
}
|
||||
|
||||
// Only used for testing
|
||||
public ZookeeperBasedLockProvider(
|
||||
final LockConfiguration lockConfiguration, final CuratorFramework curatorFrameworkClient) {
|
||||
checkRequiredProps(lockConfiguration);
|
||||
this.lockConfiguration = lockConfiguration;
|
||||
this.curatorFrameworkClient = curatorFrameworkClient;
|
||||
synchronized (this.curatorFrameworkClient) {
|
||||
if (this.curatorFrameworkClient.getState() != CuratorFrameworkState.STARTED) {
|
||||
this.curatorFrameworkClient.start();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean tryLock(long time, TimeUnit unit) {
|
||||
LOG.info(generateLogStatement(LockState.ACQUIRING, generateLogSuffixString()));
|
||||
try {
|
||||
acquireLock(time, unit);
|
||||
LOG.info(generateLogStatement(LockState.ACQUIRED, generateLogSuffixString()));
|
||||
} catch (HoodieLockException e) {
|
||||
throw e;
|
||||
} catch (Exception e) {
|
||||
throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_ACQUIRE, generateLogSuffixString()), e);
|
||||
}
|
||||
return lock != null && lock.isAcquiredInThisProcess();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void unlock() {
|
||||
try {
|
||||
LOG.info(generateLogStatement(LockState.RELEASING, generateLogSuffixString()));
|
||||
if (lock == null || !lock.isAcquiredInThisProcess()) {
|
||||
return;
|
||||
}
|
||||
lock.release();
|
||||
lock = null;
|
||||
LOG.info(generateLogStatement(LockState.RELEASED, generateLogSuffixString()));
|
||||
} catch (Exception e) {
|
||||
throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_RELEASE, generateLogSuffixString()), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
try {
|
||||
if (lock != null) {
|
||||
lock.release();
|
||||
lock = null;
|
||||
}
|
||||
this.curatorFrameworkClient.close();
|
||||
} catch (Exception e) {
|
||||
LOG.error(generateLogStatement(LockState.FAILED_TO_RELEASE, generateLogSuffixString()));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public InterProcessMutex getLock() {
|
||||
return this.lock;
|
||||
}
|
||||
|
||||
private void acquireLock(long time, TimeUnit unit) throws Exception {
|
||||
ValidationUtils.checkArgument(this.lock == null, generateLogStatement(LockState.ALREADY_ACQUIRED, generateLogSuffixString()));
|
||||
InterProcessMutex newLock = new InterProcessMutex(
|
||||
this.curatorFrameworkClient, lockConfiguration.getConfig().getString(ZK_BASE_PATH_PROP) + "/"
|
||||
+ this.lockConfiguration.getConfig().getString(ZK_LOCK_KEY_PROP));
|
||||
boolean acquired = newLock.acquire(time, unit);
|
||||
if (!acquired) {
|
||||
throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_ACQUIRE, generateLogSuffixString()));
|
||||
}
|
||||
if (newLock.isAcquiredInThisProcess()) {
|
||||
lock = newLock;
|
||||
} else {
|
||||
throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_ACQUIRE, generateLogSuffixString()));
|
||||
}
|
||||
}
|
||||
|
||||
private void checkRequiredProps(final LockConfiguration config) {
|
||||
ValidationUtils.checkArgument(config.getConfig().getString(ZK_CONNECT_URL_PROP) != null);
|
||||
ValidationUtils.checkArgument(config.getConfig().getString(ZK_BASE_PATH_PROP) != null);
|
||||
ValidationUtils.checkArgument(config.getConfig().getString(ZK_SESSION_TIMEOUT_MS_PROP) != null);
|
||||
ValidationUtils.checkArgument(config.getConfig().getString(ZK_CONNECTION_TIMEOUT_MS_PROP) != null);
|
||||
ValidationUtils.checkArgument(config.getConfig().getString(ZK_LOCK_KEY_PROP) != null);
|
||||
}
|
||||
|
||||
private String generateLogSuffixString() {
|
||||
String zkBasePath = this.lockConfiguration.getConfig().getString(ZK_BASE_PATH_PROP);
|
||||
String lockKey = this.lockConfiguration.getConfig().getString(ZK_LOCK_KEY_PROP);
|
||||
return StringUtils.join("ZkBasePath = ", zkBasePath, ", lock key = ", lockKey);
|
||||
}
|
||||
|
||||
protected String generateLogStatement(LockState state, String suffix) {
|
||||
return StringUtils.join(state.name(), " lock at", suffix);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,132 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client.utils;
|
||||
|
||||
import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import java.io.IOException;
|
||||
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
|
||||
import org.apache.hudi.client.ReplaceArchivalHelper;
|
||||
import org.apache.hudi.common.model.ActionType;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRollingStatMetadata;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
||||
import org.apache.hudi.common.util.CleanerUtils;
|
||||
import org.apache.hudi.common.util.ClusteringUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
|
||||
/**
|
||||
* Helper class to convert between different action related payloads and {@link HoodieArchivedMetaEntry}.
|
||||
*/
|
||||
public class MetadataConversionUtils {
|
||||
|
||||
public static HoodieArchivedMetaEntry createMetaWrapper(HoodieInstant hoodieInstant, HoodieTableMetaClient metaClient) throws IOException {
|
||||
HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry();
|
||||
archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp());
|
||||
archivedMetaWrapper.setActionState(hoodieInstant.getState().name());
|
||||
switch (hoodieInstant.getAction()) {
|
||||
case HoodieTimeline.CLEAN_ACTION: {
|
||||
if (hoodieInstant.isCompleted()) {
|
||||
archivedMetaWrapper.setHoodieCleanMetadata(CleanerUtils.getCleanerMetadata(metaClient, hoodieInstant));
|
||||
} else {
|
||||
archivedMetaWrapper.setHoodieCleanerPlan(CleanerUtils.getCleanerPlan(metaClient, hoodieInstant));
|
||||
}
|
||||
archivedMetaWrapper.setActionType(ActionType.clean.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION: {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class);
|
||||
archivedMetaWrapper.setHoodieCommitMetadata(convertCommitMetadata(commitMetadata));
|
||||
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.REPLACE_COMMIT_ACTION: {
|
||||
if (hoodieInstant.isCompleted()) {
|
||||
HoodieReplaceCommitMetadata replaceCommitMetadata = HoodieReplaceCommitMetadata
|
||||
.fromBytes(metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieReplaceCommitMetadata.class);
|
||||
archivedMetaWrapper.setHoodieReplaceCommitMetadata(ReplaceArchivalHelper.convertReplaceCommitMetadata(replaceCommitMetadata));
|
||||
} else {
|
||||
HoodieRequestedReplaceMetadata requestedReplaceMetadata =
|
||||
ClusteringUtils.getRequestedReplaceMetadata(metaClient, hoodieInstant).get();
|
||||
archivedMetaWrapper.setHoodieRequestedReplaceMetadata(requestedReplaceMetadata);
|
||||
}
|
||||
archivedMetaWrapper.setActionType(ActionType.replacecommit.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.ROLLBACK_ACTION: {
|
||||
archivedMetaWrapper.setHoodieRollbackMetadata(TimelineMetadataUtils.deserializeAvroMetadata(
|
||||
metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieRollbackMetadata.class));
|
||||
archivedMetaWrapper.setActionType(ActionType.rollback.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.SAVEPOINT_ACTION: {
|
||||
archivedMetaWrapper.setHoodieSavePointMetadata(TimelineMetadataUtils.deserializeAvroMetadata(
|
||||
metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieSavepointMetadata.class));
|
||||
archivedMetaWrapper.setActionType(ActionType.savepoint.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.COMPACTION_ACTION: {
|
||||
HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, hoodieInstant.getTimestamp());
|
||||
archivedMetaWrapper.setHoodieCompactionPlan(plan);
|
||||
archivedMetaWrapper.setActionType(ActionType.compaction.name());
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
throw new UnsupportedOperationException("Action not fully supported yet");
|
||||
}
|
||||
}
|
||||
return archivedMetaWrapper;
|
||||
}
|
||||
|
||||
public static HoodieArchivedMetaEntry createMetaWrapper(HoodieInstant hoodieInstant,
|
||||
HoodieCommitMetadata hoodieCommitMetadata) {
|
||||
HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry();
|
||||
archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp());
|
||||
archivedMetaWrapper.setActionState(hoodieInstant.getState().name());
|
||||
archivedMetaWrapper.setHoodieCommitMetadata(convertCommitMetadata(hoodieCommitMetadata));
|
||||
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
||||
return archivedMetaWrapper;
|
||||
}
|
||||
|
||||
public static org.apache.hudi.avro.model.HoodieCommitMetadata convertCommitMetadata(
|
||||
HoodieCommitMetadata hoodieCommitMetadata) {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
// Need this to ignore other public get() methods
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
org.apache.hudi.avro.model.HoodieCommitMetadata avroMetaData =
|
||||
mapper.convertValue(hoodieCommitMetadata, org.apache.hudi.avro.model.HoodieCommitMetadata.class);
|
||||
if (hoodieCommitMetadata.getCompacted()) {
|
||||
avroMetaData.setOperationType(WriteOperationType.COMPACT.name());
|
||||
}
|
||||
// Do not archive Rolling Stats, cannot set to null since AVRO will throw null pointer
|
||||
avroMetaData.getExtraMetadata().put(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, "");
|
||||
return avroMetaData;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,128 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client.utils;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import org.apache.hudi.client.transaction.ConflictResolutionStrategy;
|
||||
import org.apache.hudi.client.transaction.ConcurrentOperation;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieWriteConflictException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import java.io.IOException;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class TransactionUtils {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(TransactionUtils.class);
|
||||
|
||||
/**
|
||||
* Resolve any write conflicts when committing data.
|
||||
* @param table
|
||||
* @param currentTxnOwnerInstant
|
||||
* @param thisCommitMetadata
|
||||
* @param config
|
||||
* @param lastCompletedTxnOwnerInstant
|
||||
* @return
|
||||
* @throws HoodieWriteConflictException
|
||||
*/
|
||||
public static Option<HoodieCommitMetadata> resolveWriteConflictIfAny(final HoodieTable table, final Option<HoodieInstant> currentTxnOwnerInstant,
|
||||
final Option<HoodieCommitMetadata> thisCommitMetadata, final HoodieWriteConfig config, Option<HoodieInstant> lastCompletedTxnOwnerInstant) throws HoodieWriteConflictException {
|
||||
if (config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) {
|
||||
ConflictResolutionStrategy resolutionStrategy = config.getWriteConflictResolutionStrategy();
|
||||
Stream<HoodieInstant> instantStream = resolutionStrategy.getCandidateInstants(table.getActiveTimeline(), currentTxnOwnerInstant.get(), lastCompletedTxnOwnerInstant);
|
||||
final ConcurrentOperation thisOperation = new ConcurrentOperation(currentTxnOwnerInstant.get(), thisCommitMetadata.get());
|
||||
instantStream.forEach(instant -> {
|
||||
try {
|
||||
ConcurrentOperation otherOperation = new ConcurrentOperation(instant, table.getMetaClient());
|
||||
if (resolutionStrategy.hasConflict(thisOperation, otherOperation)) {
|
||||
LOG.info("Conflict encountered between current instant = " + thisOperation + " and instant = "
|
||||
+ otherOperation + ", attempting to resolve it...");
|
||||
resolutionStrategy.resolveConflict(table, thisOperation, otherOperation);
|
||||
}
|
||||
} catch (IOException io) {
|
||||
throw new HoodieWriteConflictException("Unable to resolve conflict, if present", io);
|
||||
}
|
||||
});
|
||||
LOG.info("Successfully resolved conflicts, if any");
|
||||
// carry over necessary metadata from latest commit metadata
|
||||
overrideWithLatestCommitMetadata(table.getMetaClient(), thisOperation.getCommitMetadataOption(), currentTxnOwnerInstant, Arrays.asList(config.getWriteMetaKeyPrefixes().split(",")));
|
||||
return thisOperation.getCommitMetadataOption();
|
||||
}
|
||||
return thisCommitMetadata;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the last completed transaction hoodie instant and {@link HoodieCommitMetadata#getExtraMetadata()}.
|
||||
* @param metaClient
|
||||
* @return
|
||||
*/
|
||||
public static Option<Pair<HoodieInstant, Map<String, String>>> getLastCompletedTxnInstantAndMetadata(
|
||||
HoodieTableMetaClient metaClient) {
|
||||
Option<HoodieInstant> hoodieInstantOption = metaClient.getActiveTimeline().getCommitsTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
try {
|
||||
if (hoodieInstantOption.isPresent()) {
|
||||
switch (hoodieInstantOption.get().getAction()) {
|
||||
case HoodieTimeline.REPLACE_COMMIT_ACTION:
|
||||
HoodieReplaceCommitMetadata replaceCommitMetadata = HoodieReplaceCommitMetadata
|
||||
.fromBytes(metaClient.getActiveTimeline().getInstantDetails(hoodieInstantOption.get()).get(), HoodieReplaceCommitMetadata.class);
|
||||
return Option.of(Pair.of(hoodieInstantOption.get(), replaceCommitMetadata.getExtraMetadata()));
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(metaClient.getActiveTimeline().getInstantDetails(hoodieInstantOption.get()).get(), HoodieCommitMetadata.class);
|
||||
return Option.of(Pair.of(hoodieInstantOption.get(), commitMetadata.getExtraMetadata()));
|
||||
default:
|
||||
throw new IllegalArgumentException("Unknown instant action" + hoodieInstantOption.get().getAction());
|
||||
}
|
||||
} else {
|
||||
return Option.empty();
|
||||
}
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("Unable to read metadata for instant " + hoodieInstantOption.get(), io);
|
||||
}
|
||||
}
|
||||
|
||||
// override the current metadata with the metadata from the latest instant for the specified key prefixes
|
||||
private static void overrideWithLatestCommitMetadata(HoodieTableMetaClient metaClient, Option<HoodieCommitMetadata> thisMetadata,
|
||||
Option<HoodieInstant> thisInstant, List<String> keyPrefixes) {
|
||||
if (keyPrefixes.size() == 1 && keyPrefixes.get(0).length() < 1) {
|
||||
return;
|
||||
}
|
||||
Option<Pair<HoodieInstant, Map<String, String>>> lastInstant = getLastCompletedTxnInstantAndMetadata(metaClient);
|
||||
if (lastInstant.isPresent() && thisMetadata.isPresent()) {
|
||||
Stream<String> keys = thisMetadata.get().getExtraMetadata().keySet().stream();
|
||||
keyPrefixes.stream().forEach(keyPrefix -> keys
|
||||
.filter(key -> key.startsWith(keyPrefix))
|
||||
.forEach(key -> thisMetadata.get().getExtraMetadata().put(key, lastInstant.get().getRight().get(key))));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -43,7 +43,6 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
public static final String CLEANER_POLICY_PROP = "hoodie.cleaner.policy";
|
||||
public static final String AUTO_CLEAN_PROP = "hoodie.clean.automatic";
|
||||
public static final String ASYNC_CLEAN_PROP = "hoodie.clean.async";
|
||||
|
||||
// Turn on inline compaction - after fw delta commits a inline compaction will be run
|
||||
public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
|
||||
// Run a compaction every N delta commits
|
||||
|
||||
@@ -0,0 +1,191 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
import org.apache.hudi.client.transaction.SimpleConcurrentFileWritesConflictResolutionStrategy;
|
||||
import org.apache.hudi.client.transaction.ConflictResolutionStrategy;
|
||||
import org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider;
|
||||
import org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
import org.apache.hudi.common.lock.LockProvider;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ACQUIRE_LOCK_WAIT_TIMEOUT_MS;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_CLIENT_NUM_RETRIES;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_NUM_RETRIES;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_CONNECTION_TIMEOUT_MS;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_SESSION_TIMEOUT_MS;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.HIVE_TABLE_NAME_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_PREFIX;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_PORT_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP;
|
||||
|
||||
|
||||
/**
|
||||
* Hoodie Configs for Locks.
|
||||
*/
|
||||
public class HoodieLockConfig extends DefaultHoodieConfig {
|
||||
|
||||
// Pluggable type of lock provider
|
||||
public static final String LOCK_PROVIDER_CLASS_PROP = LOCK_PREFIX + "provider";
|
||||
public static final String DEFAULT_LOCK_PROVIDER_CLASS = ZookeeperBasedLockProvider.class.getName();
|
||||
// Pluggable strategies to use when resolving conflicts
|
||||
public static final String WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP =
|
||||
LOCK_PREFIX + "conflict.resolution.strategy";
|
||||
public static final String DEFAULT_WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS =
|
||||
SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName();
|
||||
|
||||
private HoodieLockConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static HoodieLockConfig.Builder newBuilder() {
|
||||
return new HoodieLockConfig.Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
|
||||
public HoodieLockConfig.Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withLockProvider(Class<? extends LockProvider> lockProvider) {
|
||||
props.setProperty(LOCK_PROVIDER_CLASS_PROP, lockProvider.getName());
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withHiveDatabaseName(String databaseName) {
|
||||
props.setProperty(HIVE_DATABASE_NAME_PROP, databaseName);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withHiveTableName(String tableName) {
|
||||
props.setProperty(HIVE_TABLE_NAME_PROP, tableName);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withZkQuorum(String zkQuorum) {
|
||||
props.setProperty(ZK_CONNECT_URL_PROP, zkQuorum);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withZkBasePath(String zkBasePath) {
|
||||
props.setProperty(ZK_BASE_PATH_PROP, zkBasePath);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withZkPort(String zkPort) {
|
||||
props.setProperty(ZK_PORT_PROP, zkPort);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withZkLockKey(String zkLockKey) {
|
||||
props.setProperty(ZK_LOCK_KEY_PROP, zkLockKey);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withZkConnectionTimeoutInMs(Long connectionTimeoutInMs) {
|
||||
props.setProperty(ZK_CONNECTION_TIMEOUT_MS_PROP, String.valueOf(connectionTimeoutInMs));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withZkSessionTimeoutInMs(Long sessionTimeoutInMs) {
|
||||
props.setProperty(ZK_SESSION_TIMEOUT_MS_PROP, String.valueOf(sessionTimeoutInMs));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withNumRetries(int numRetries) {
|
||||
props.setProperty(LOCK_ACQUIRE_NUM_RETRIES_PROP, String.valueOf(numRetries));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withRetryWaitTimeInMillis(Long retryWaitTimeInMillis) {
|
||||
props.setProperty(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP, String.valueOf(retryWaitTimeInMillis));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withClientNumRetries(int clientNumRetries) {
|
||||
props.setProperty(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP, String.valueOf(clientNumRetries));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withClientRetryWaitTimeInMillis(Long clientRetryWaitTimeInMillis) {
|
||||
props.setProperty(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP, String.valueOf(clientRetryWaitTimeInMillis));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withLockWaitTimeInMillis(Long waitTimeInMillis) {
|
||||
props.setProperty(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP, String.valueOf(waitTimeInMillis));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withConflictResolutionStrategy(ConflictResolutionStrategy conflictResolutionStrategy) {
|
||||
props.setProperty(WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP, conflictResolutionStrategy.getClass().getName());
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig build() {
|
||||
HoodieLockConfig config = new HoodieLockConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOCK_PROVIDER_CLASS_PROP),
|
||||
LOCK_PROVIDER_CLASS_PROP, DEFAULT_LOCK_PROVIDER_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP),
|
||||
WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP, DEFAULT_WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_NUM_RETRIES_PROP),
|
||||
LOCK_ACQUIRE_NUM_RETRIES_PROP, DEFAULT_LOCK_ACQUIRE_NUM_RETRIES);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP),
|
||||
LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP, DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP),
|
||||
LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP, DEFAULT_LOCK_ACQUIRE_CLIENT_NUM_RETRIES);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP),
|
||||
LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP, DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS);
|
||||
setDefaultOnCondition(props, !props.containsKey(ZK_CONNECTION_TIMEOUT_MS_PROP),
|
||||
ZK_CONNECTION_TIMEOUT_MS_PROP, String.valueOf(DEFAULT_ZK_CONNECTION_TIMEOUT_MS));
|
||||
setDefaultOnCondition(props, !props.containsKey(ZK_SESSION_TIMEOUT_MS_PROP),
|
||||
ZK_SESSION_TIMEOUT_MS_PROP, String.valueOf(DEFAULT_ZK_SESSION_TIMEOUT_MS));
|
||||
setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP),
|
||||
LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP, String.valueOf(DEFAULT_ACQUIRE_LOCK_WAIT_TIMEOUT_MS));
|
||||
return config;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -18,18 +18,23 @@
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.bootstrap.BootstrapMode;
|
||||
import org.apache.hudi.client.transaction.ConflictResolutionStrategy;
|
||||
import org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.config.LockConfiguration;
|
||||
import org.apache.hudi.common.engine.EngineType;
|
||||
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
|
||||
import org.apache.hudi.common.model.WriteConcurrencyMode;
|
||||
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.keygen.SimpleAvroKeyGenerator;
|
||||
@@ -37,12 +42,9 @@ import org.apache.hudi.metrics.MetricsReporterType;
|
||||
import org.apache.hudi.metrics.datadog.DatadogHttpClient.ApiSite;
|
||||
import org.apache.hudi.table.action.compact.CompactionTriggerStrategy;
|
||||
import org.apache.hudi.table.action.compact.strategy.CompactionStrategy;
|
||||
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
@@ -55,6 +57,8 @@ import java.util.Properties;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.HIVE_TABLE_NAME_PROP;
|
||||
|
||||
/**
|
||||
* Class storing configs for the HoodieWriteClient.
|
||||
@@ -148,6 +152,15 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP = "hoodie.client.heartbeat.tolerable.misses";
|
||||
public static final Integer DEFAULT_CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES = 2;
|
||||
// Enable different concurrency support
|
||||
public static final String WRITE_CONCURRENCY_MODE_PROP =
|
||||
"hoodie.write.concurrency.mode";
|
||||
public static final String DEFAULT_WRITE_CONCURRENCY_MODE = WriteConcurrencyMode.SINGLE_WRITER.name();
|
||||
|
||||
// Comma separated metadata key prefixes to override from latest commit during overlapping commits via multi writing
|
||||
public static final String WRITE_META_KEY_PREFIXES_PROP =
|
||||
"hoodie.write.meta.key.prefixes";
|
||||
public static final String DEFAULT_WRITE_META_KEY_PREFIXES = "";
|
||||
|
||||
/**
|
||||
* HUDI-858 : There are users who had been directly using RDD APIs and have relied on a behavior in 0.4.x to allow
|
||||
@@ -427,7 +440,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.CLEANER_INCREMENTAL_MODE));
|
||||
}
|
||||
|
||||
public boolean isInlineCompaction() {
|
||||
public boolean inlineCompactionEnabled() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_PROP));
|
||||
}
|
||||
|
||||
@@ -459,7 +472,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP));
|
||||
}
|
||||
|
||||
public boolean isInlineClustering() {
|
||||
public boolean inlineClusteringEnabled() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieClusteringConfig.INLINE_CLUSTERING_PROP));
|
||||
}
|
||||
|
||||
@@ -469,7 +482,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
|
||||
public boolean isClusteringEnabled() {
|
||||
// TODO: future support async clustering
|
||||
return isInlineClustering() || isAsyncClusteringEnabled();
|
||||
return inlineClusteringEnabled() || isAsyncClusteringEnabled();
|
||||
}
|
||||
|
||||
public int getInlineClusterMaxCommits() {
|
||||
@@ -939,10 +952,6 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
return Long.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP));
|
||||
}
|
||||
|
||||
public int getMetadataCleanerCommitsRetained() {
|
||||
return Integer.parseInt(props.getProperty(HoodieMetadataConfig.CLEANER_COMMITS_RETAINED_PROP));
|
||||
}
|
||||
|
||||
public Long getHoodieClientHeartbeatIntervalInMs() {
|
||||
return Long.valueOf(props.getProperty(CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP));
|
||||
}
|
||||
@@ -982,6 +991,47 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
return Integer.parseInt(props.getProperty(HoodieMetadataConfig.MIN_COMMITS_TO_KEEP_PROP));
|
||||
}
|
||||
|
||||
public int getMetadataCleanerCommitsRetained() {
|
||||
return Integer.parseInt(props.getProperty(HoodieMetadataConfig.CLEANER_COMMITS_RETAINED_PROP));
|
||||
}
|
||||
|
||||
/**
|
||||
* Hoodie Client Lock Configs.
|
||||
* @return
|
||||
*/
|
||||
|
||||
public String getLockProviderClass() {
|
||||
return props.getProperty(HoodieLockConfig.LOCK_PROVIDER_CLASS_PROP);
|
||||
}
|
||||
|
||||
public String getLockHiveDatabaseName() {
|
||||
return props.getProperty(HIVE_DATABASE_NAME_PROP);
|
||||
}
|
||||
|
||||
public String getLockHiveTableName() {
|
||||
return props.getProperty(HIVE_TABLE_NAME_PROP);
|
||||
}
|
||||
|
||||
public ConflictResolutionStrategy getWriteConflictResolutionStrategy() {
|
||||
return ReflectionUtils.loadClass(props.getProperty(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP));
|
||||
}
|
||||
|
||||
public Long getLockAcquireWaitTimeoutInMs() {
|
||||
return Long.valueOf(props.getProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP));
|
||||
}
|
||||
|
||||
public WriteConcurrencyMode getWriteConcurrencyMode() {
|
||||
return WriteConcurrencyMode.fromValue(props.getProperty(WRITE_CONCURRENCY_MODE_PROP));
|
||||
}
|
||||
|
||||
public Boolean inlineTableServices() {
|
||||
return inlineClusteringEnabled() || inlineCompactionEnabled() || isAutoClean();
|
||||
}
|
||||
|
||||
public String getWriteMetaKeyPrefixes() {
|
||||
return props.getProperty(WRITE_META_KEY_PREFIXES_PROP);
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
protected final Properties props = new Properties();
|
||||
@@ -998,6 +1048,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
private boolean isCallbackConfigSet = false;
|
||||
private boolean isPayloadConfigSet = false;
|
||||
private boolean isMetadataConfigSet = false;
|
||||
private boolean isLockConfigSet = false;
|
||||
|
||||
public Builder withEngineType(EngineType engineType) {
|
||||
this.engineType = engineType;
|
||||
@@ -1141,6 +1192,12 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withLockConfig(HoodieLockConfig lockConfig) {
|
||||
props.putAll(lockConfig.getProps());
|
||||
isLockConfigSet = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMetricsConfig(HoodieMetricsConfig metricsConfig) {
|
||||
props.putAll(metricsConfig.getProps());
|
||||
isMetricsConfigSet = true;
|
||||
@@ -1254,6 +1311,16 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withWriteConcurrencyMode(WriteConcurrencyMode concurrencyMode) {
|
||||
props.setProperty(WRITE_CONCURRENCY_MODE_PROP, concurrencyMode.value());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withWriteMetaKeyPrefixes(String writeMetaKeyPrefixes) {
|
||||
props.setProperty(WRITE_META_KEY_PREFIXES_PROP, writeMetaKeyPrefixes);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withProperties(Properties properties) {
|
||||
this.props.putAll(properties);
|
||||
return this;
|
||||
@@ -1314,7 +1381,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP, String.valueOf(DEFAULT_CLIENT_HEARTBEAT_INTERVAL_IN_MS));
|
||||
setDefaultOnCondition(props, !props.containsKey(CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP),
|
||||
CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP, String.valueOf(DEFAULT_CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES));
|
||||
|
||||
setDefaultOnCondition(props, !props.containsKey(WRITE_CONCURRENCY_MODE_PROP),
|
||||
WRITE_CONCURRENCY_MODE_PROP, DEFAULT_WRITE_CONCURRENCY_MODE);
|
||||
setDefaultOnCondition(props, !props.containsKey(WRITE_META_KEY_PREFIXES_PROP),
|
||||
WRITE_META_KEY_PREFIXES_PROP, DEFAULT_WRITE_META_KEY_PREFIXES);
|
||||
// Make sure the props is propagated
|
||||
setDefaultOnCondition(props, !isIndexConfigSet, HoodieIndexConfig.newBuilder().withEngineType(engineType).fromProperties(props).build());
|
||||
setDefaultOnCondition(props, !isStorageConfigSet, HoodieStorageConfig.newBuilder().fromProperties(props).build());
|
||||
@@ -1336,6 +1406,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
HoodiePayloadConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, !isMetadataConfigSet,
|
||||
HoodieMetadataConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, !isLockConfigSet,
|
||||
HoodieLockConfig.newBuilder().fromProperties(props).build());
|
||||
|
||||
setDefaultOnCondition(props, !props.containsKey(EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION),
|
||||
EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION, DEFAULT_EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION);
|
||||
@@ -1349,6 +1421,11 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
// Ensure Layout Version is good
|
||||
new TimelineLayoutVersion(Integer.parseInt(layoutVersion));
|
||||
Objects.requireNonNull(props.getProperty(BASE_PATH_PROP));
|
||||
if (props.getProperty(WRITE_CONCURRENCY_MODE_PROP)
|
||||
.equalsIgnoreCase(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.name())) {
|
||||
ValidationUtils.checkArgument(props.getProperty(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY_PROP)
|
||||
!= HoodieFailedWritesCleaningPolicy.EAGER.name());
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieWriteConfig build() {
|
||||
|
||||
@@ -36,6 +36,7 @@ import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.WriteConcurrencyMode;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
@@ -89,7 +90,8 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
||||
protected SerializableConfiguration hadoopConf;
|
||||
protected final transient HoodieEngineContext engineContext;
|
||||
|
||||
protected HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig, HoodieEngineContext engineContext) {
|
||||
protected HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig,
|
||||
HoodieEngineContext engineContext) {
|
||||
this.datasetWriteConfig = writeConfig;
|
||||
this.engineContext = engineContext;
|
||||
this.hadoopConf = new SerializableConfiguration(hadoopConf);
|
||||
@@ -101,7 +103,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
||||
|
||||
// Inline compaction and auto clean is required as we dont expose this table outside
|
||||
ValidationUtils.checkArgument(!this.metadataWriteConfig.isAutoClean(), "Cleaning is controlled internally for Metadata table.");
|
||||
ValidationUtils.checkArgument(!this.metadataWriteConfig.isInlineCompaction(), "Compaction is controlled internally for metadata table.");
|
||||
ValidationUtils.checkArgument(!this.metadataWriteConfig.inlineCompactionEnabled(), "Compaction is controlled internally for metadata table.");
|
||||
// Metadata Table cannot have metadata listing turned on. (infinite loop, much?)
|
||||
ValidationUtils.checkArgument(this.metadataWriteConfig.shouldAutoCommit(), "Auto commit is required for Metadata Table");
|
||||
ValidationUtils.checkArgument(!this.metadataWriteConfig.useFileListingMetadata(), "File listing cannot be used for Metadata Table");
|
||||
@@ -143,6 +145,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
||||
.withMaxConsistencyCheckIntervalMs(writeConfig.getConsistencyGuardConfig().getMaxConsistencyCheckIntervalMs())
|
||||
.withMaxConsistencyChecks(writeConfig.getConsistencyGuardConfig().getMaxConsistencyChecks())
|
||||
.build())
|
||||
.withWriteConcurrencyMode(WriteConcurrencyMode.SINGLE_WRITER)
|
||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
|
||||
.withAutoCommit(true)
|
||||
.withAvroSchemaValidate(true)
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.avro.model.HoodieClusteringPlan;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
||||
@@ -393,6 +394,19 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
||||
*/
|
||||
public abstract void rollbackBootstrap(HoodieEngineContext context, String instantTime);
|
||||
|
||||
|
||||
/**
|
||||
* Schedule cleaning for the instant time.
|
||||
*
|
||||
* @param context HoodieEngineContext
|
||||
* @param instantTime Instant Time for scheduling cleaning
|
||||
* @param extraMetadata additional metadata to write into plan
|
||||
* @return HoodieCleanerPlan, if there is anything to clean.
|
||||
*/
|
||||
public abstract Option<HoodieCleanerPlan> scheduleCleaning(HoodieEngineContext context,
|
||||
String instantTime,
|
||||
Option<Map<String, String>> extraMetadata);
|
||||
|
||||
/**
|
||||
* Executes a new clean action.
|
||||
*
|
||||
|
||||
@@ -18,24 +18,17 @@
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
|
||||
import org.apache.hudi.client.ReplaceArchivalHelper;
|
||||
import org.apache.hudi.client.utils.MetadataConversionUtils;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.ActionType;
|
||||
import org.apache.hudi.common.model.HoodieArchivedLogFile;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRollingStatMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
|
||||
@@ -46,12 +39,9 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.TableFileSystemView;
|
||||
import org.apache.hudi.common.util.CleanerUtils;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
@@ -59,7 +49,6 @@ import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadata;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -378,73 +367,6 @@ public class HoodieTimelineArchiveLog<T extends HoodieAvroPayload, I, K, O> {
|
||||
|
||||
private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline, HoodieInstant hoodieInstant)
|
||||
throws IOException {
|
||||
HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry();
|
||||
archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp());
|
||||
archivedMetaWrapper.setActionState(hoodieInstant.getState().name());
|
||||
switch (hoodieInstant.getAction()) {
|
||||
case HoodieTimeline.CLEAN_ACTION: {
|
||||
if (hoodieInstant.isCompleted()) {
|
||||
archivedMetaWrapper.setHoodieCleanMetadata(CleanerUtils.getCleanerMetadata(metaClient, hoodieInstant));
|
||||
} else {
|
||||
archivedMetaWrapper.setHoodieCleanerPlan(CleanerUtils.getCleanerPlan(metaClient, hoodieInstant));
|
||||
}
|
||||
archivedMetaWrapper.setActionType(ActionType.clean.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION: {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class);
|
||||
archivedMetaWrapper.setHoodieCommitMetadata(convertCommitMetadata(commitMetadata));
|
||||
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.REPLACE_COMMIT_ACTION: {
|
||||
if (hoodieInstant.isRequested()) {
|
||||
archivedMetaWrapper.setHoodieRequestedReplaceMetadata(
|
||||
TimelineMetadataUtils.deserializeRequestedReplaceMetadata(commitTimeline.getInstantDetails(hoodieInstant).get()));
|
||||
} else if (hoodieInstant.isCompleted()) {
|
||||
HoodieReplaceCommitMetadata replaceCommitMetadata = HoodieReplaceCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieReplaceCommitMetadata.class);
|
||||
archivedMetaWrapper.setHoodieReplaceCommitMetadata(ReplaceArchivalHelper.convertReplaceCommitMetadata(replaceCommitMetadata));
|
||||
}
|
||||
archivedMetaWrapper.setActionType(ActionType.replacecommit.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.ROLLBACK_ACTION: {
|
||||
archivedMetaWrapper.setHoodieRollbackMetadata(TimelineMetadataUtils.deserializeAvroMetadata(
|
||||
commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieRollbackMetadata.class));
|
||||
archivedMetaWrapper.setActionType(ActionType.rollback.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.SAVEPOINT_ACTION: {
|
||||
archivedMetaWrapper.setHoodieSavePointMetadata(TimelineMetadataUtils.deserializeAvroMetadata(
|
||||
commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieSavepointMetadata.class));
|
||||
archivedMetaWrapper.setActionType(ActionType.savepoint.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.COMPACTION_ACTION: {
|
||||
HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, hoodieInstant.getTimestamp());
|
||||
archivedMetaWrapper.setHoodieCompactionPlan(plan);
|
||||
archivedMetaWrapper.setActionType(ActionType.compaction.name());
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
throw new UnsupportedOperationException("Action not fully supported yet");
|
||||
}
|
||||
}
|
||||
return archivedMetaWrapper;
|
||||
}
|
||||
|
||||
public static org.apache.hudi.avro.model.HoodieCommitMetadata convertCommitMetadata(
|
||||
HoodieCommitMetadata hoodieCommitMetadata) {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
// Need this to ignore other public get() methods
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
org.apache.hudi.avro.model.HoodieCommitMetadata avroMetaData =
|
||||
mapper.convertValue(hoodieCommitMetadata, org.apache.hudi.avro.model.HoodieCommitMetadata.class);
|
||||
// Do not archive Rolling Stats, cannot set to null since AVRO will throw null pointer
|
||||
avroMetaData.getExtraMetadata().put(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, "");
|
||||
return avroMetaData;
|
||||
return MetadataConversionUtils.createMetaWrapper(hoodieInstant, metaClient);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -20,23 +20,17 @@ package org.apache.hudi.table.action.clean;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.model.HoodieActionInstant;
|
||||
import org.apache.hudi.avro.model.HoodieCleanFileInfo;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
||||
import org.apache.hudi.common.util.CleanerUtils;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.common.util.HoodieTimer;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
@@ -46,8 +40,8 @@ import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public abstract class BaseCleanActionExecutor<T extends HoodieRecordPayload, I, K, O> extends BaseActionExecutor<T, I, K, O, HoodieCleanMetadata> {
|
||||
@@ -59,42 +53,6 @@ public abstract class BaseCleanActionExecutor<T extends HoodieRecordPayload, I,
|
||||
super(context, config, table, instantTime);
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates List of files to be cleaned.
|
||||
*
|
||||
* @param context HoodieEngineContext
|
||||
* @return Cleaner Plan
|
||||
*/
|
||||
HoodieCleanerPlan requestClean(HoodieEngineContext context) {
|
||||
try {
|
||||
CleanPlanner<T, I, K, O> planner = new CleanPlanner<>(context, table, config);
|
||||
Option<HoodieInstant> earliestInstant = planner.getEarliestCommitToRetain();
|
||||
List<String> partitionsToClean = planner.getPartitionPathsToClean(earliestInstant);
|
||||
|
||||
if (partitionsToClean.isEmpty()) {
|
||||
LOG.info("Nothing to clean here.");
|
||||
return HoodieCleanerPlan.newBuilder().setPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name()).build();
|
||||
}
|
||||
LOG.info("Total Partitions to clean : " + partitionsToClean.size() + ", with policy " + config.getCleanerPolicy());
|
||||
int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism());
|
||||
LOG.info("Using cleanerParallelism: " + cleanerParallelism);
|
||||
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Generates list of file slices to be cleaned");
|
||||
|
||||
Map<String, List<HoodieCleanFileInfo>> cleanOps = context
|
||||
.map(partitionsToClean, partitionPathToClean -> Pair.of(partitionPathToClean, planner.getDeletePaths(partitionPathToClean)), cleanerParallelism)
|
||||
.stream()
|
||||
.collect(Collectors.toMap(Pair::getKey, y -> CleanerUtils.convertToHoodieCleanFileInfoList(y.getValue())));
|
||||
|
||||
return new HoodieCleanerPlan(earliestInstant
|
||||
.map(x -> new HoodieActionInstant(x.getTimestamp(), x.getAction(), x.getState().name())).orElse(null),
|
||||
config.getCleanerPolicy().name(), CollectionUtils.createImmutableMap(),
|
||||
CleanPlanner.LATEST_CLEAN_PLAN_VERSION, cleanOps);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to schedule clean operation", e);
|
||||
}
|
||||
}
|
||||
|
||||
protected static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr) throws IOException {
|
||||
Path deletePath = new Path(deletePathStr);
|
||||
LOG.debug("Working on delete path :" + deletePath);
|
||||
@@ -118,40 +76,13 @@ public abstract class BaseCleanActionExecutor<T extends HoodieRecordPayload, I,
|
||||
*/
|
||||
abstract List<HoodieCleanStat> clean(HoodieEngineContext context, HoodieCleanerPlan cleanerPlan);
|
||||
|
||||
/**
|
||||
* Creates a Cleaner plan if there are files to be cleaned and stores them in instant file.
|
||||
* Cleaner Plan contains absolute file paths.
|
||||
*
|
||||
* @param startCleanTime Cleaner Instant Time
|
||||
* @return Cleaner Plan if generated
|
||||
*/
|
||||
Option<HoodieCleanerPlan> requestClean(String startCleanTime) {
|
||||
final HoodieCleanerPlan cleanerPlan = requestClean(context);
|
||||
if ((cleanerPlan.getFilePathsToBeDeletedPerPartition() != null)
|
||||
&& !cleanerPlan.getFilePathsToBeDeletedPerPartition().isEmpty()
|
||||
&& cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().mapToInt(List::size).sum() > 0) {
|
||||
// Only create cleaner plan which does some work
|
||||
final HoodieInstant cleanInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, startCleanTime);
|
||||
// Save to both aux and timeline folder
|
||||
try {
|
||||
table.getActiveTimeline().saveToCleanRequested(cleanInstant, TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan));
|
||||
LOG.info("Requesting Cleaning with instant time " + cleanInstant);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Got exception when saving cleaner requested file", e);
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
return Option.of(cleanerPlan);
|
||||
}
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Executes the Cleaner plan stored in the instant metadata.
|
||||
*/
|
||||
void runPendingClean(HoodieTable<T, I, K, O> table, HoodieInstant cleanInstant) {
|
||||
HoodieCleanMetadata runPendingClean(HoodieTable<T, I, K, O> table, HoodieInstant cleanInstant) {
|
||||
try {
|
||||
HoodieCleanerPlan cleanerPlan = CleanerUtils.getCleanerPlan(table.getMetaClient(), cleanInstant);
|
||||
runClean(table, cleanInstant, cleanerPlan);
|
||||
return runClean(table, cleanInstant, cleanerPlan);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
@@ -195,6 +126,7 @@ public abstract class BaseCleanActionExecutor<T extends HoodieRecordPayload, I,
|
||||
|
||||
@Override
|
||||
public HoodieCleanMetadata execute() {
|
||||
List<HoodieCleanMetadata> cleanMetadataList = new ArrayList<>();
|
||||
// If there are inflight(failed) or previously requested clean operation, first perform them
|
||||
List<HoodieInstant> pendingCleanInstants = table.getCleanTimeline()
|
||||
.filterInflightsAndRequested().getInstants().collect(Collectors.toList());
|
||||
@@ -202,23 +134,16 @@ public abstract class BaseCleanActionExecutor<T extends HoodieRecordPayload, I,
|
||||
pendingCleanInstants.forEach(hoodieInstant -> {
|
||||
LOG.info("Finishing previously unfinished cleaner instant=" + hoodieInstant);
|
||||
try {
|
||||
runPendingClean(table, hoodieInstant);
|
||||
cleanMetadataList.add(runPendingClean(table, hoodieInstant));
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Failed to perform previous clean operation, instant: " + hoodieInstant, e);
|
||||
}
|
||||
});
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
}
|
||||
|
||||
// Plan and execute a new clean action
|
||||
Option<HoodieCleanerPlan> cleanerPlanOpt = requestClean(instantTime);
|
||||
if (cleanerPlanOpt.isPresent()) {
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
HoodieCleanerPlan cleanerPlan = cleanerPlanOpt.get();
|
||||
if ((cleanerPlan.getFilePathsToBeDeletedPerPartition() != null) && !cleanerPlan.getFilePathsToBeDeletedPerPartition().isEmpty()) {
|
||||
return runClean(table, HoodieTimeline.getCleanRequestedInstant(instantTime), cleanerPlan);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
// return the last clean metadata for now
|
||||
// TODO (NA) : Clean only the earliest pending clean just like how we do for other table services
|
||||
// This requires the BaseCleanActionExecutor to be refactored as BaseCommitActionExecutor
|
||||
return cleanMetadataList.size() > 0 ? cleanMetadataList.get(cleanMetadataList.size() - 1) : null;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,132 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.action.clean;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieActionInstant;
|
||||
import org.apache.hudi.avro.model.HoodieCleanFileInfo;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
||||
import org.apache.hudi.common.util.CleanerUtils;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.BaseActionExecutor;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public abstract class BaseCleanPlanActionExecutor<T extends HoodieRecordPayload, I, K, O> extends BaseActionExecutor<T, I, K, O, Option<HoodieCleanerPlan>> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(CleanPlanner.class);
|
||||
|
||||
private final Option<Map<String, String>> extraMetadata;
|
||||
|
||||
public BaseCleanPlanActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable<T, I, K, O> table,
|
||||
String instantTime,
|
||||
Option<Map<String, String>> extraMetadata) {
|
||||
super(context, config, table, instantTime);
|
||||
this.extraMetadata = extraMetadata;
|
||||
}
|
||||
|
||||
protected abstract Option<HoodieCleanerPlan> createCleanerPlan();
|
||||
|
||||
/**
|
||||
* Generates List of files to be cleaned.
|
||||
*
|
||||
* @param context HoodieEngineContext
|
||||
* @return Cleaner Plan
|
||||
*/
|
||||
HoodieCleanerPlan requestClean(HoodieEngineContext context) {
|
||||
try {
|
||||
CleanPlanner<T, I, K, O> planner = new CleanPlanner<>(context, table, config);
|
||||
Option<HoodieInstant> earliestInstant = planner.getEarliestCommitToRetain();
|
||||
List<String> partitionsToClean = planner.getPartitionPathsToClean(earliestInstant);
|
||||
|
||||
if (partitionsToClean.isEmpty()) {
|
||||
LOG.info("Nothing to clean here. It is already clean");
|
||||
return HoodieCleanerPlan.newBuilder().setPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name()).build();
|
||||
}
|
||||
LOG.info("Total Partitions to clean : " + partitionsToClean.size() + ", with policy " + config.getCleanerPolicy());
|
||||
int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism());
|
||||
LOG.info("Using cleanerParallelism: " + cleanerParallelism);
|
||||
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Generates list of file slices to be cleaned");
|
||||
|
||||
Map<String, List<HoodieCleanFileInfo>> cleanOps = context
|
||||
.map(partitionsToClean, partitionPathToClean -> Pair.of(partitionPathToClean, planner.getDeletePaths(partitionPathToClean)), cleanerParallelism)
|
||||
.stream()
|
||||
.collect(Collectors.toMap(Pair::getKey, y -> CleanerUtils.convertToHoodieCleanFileInfoList(y.getValue())));
|
||||
|
||||
return new HoodieCleanerPlan(earliestInstant
|
||||
.map(x -> new HoodieActionInstant(x.getTimestamp(), x.getAction(), x.getState().name())).orElse(null),
|
||||
config.getCleanerPolicy().name(), CollectionUtils.createImmutableMap(),
|
||||
CleanPlanner.LATEST_CLEAN_PLAN_VERSION, cleanOps);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to schedule clean operation", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a Cleaner plan if there are files to be cleaned and stores them in instant file.
|
||||
* Cleaner Plan contains absolute file paths.
|
||||
*
|
||||
* @param startCleanTime Cleaner Instant Time
|
||||
* @return Cleaner Plan if generated
|
||||
*/
|
||||
protected Option<HoodieCleanerPlan> requestClean(String startCleanTime) {
|
||||
final HoodieCleanerPlan cleanerPlan = requestClean(context);
|
||||
if ((cleanerPlan.getFilePathsToBeDeletedPerPartition() != null)
|
||||
&& !cleanerPlan.getFilePathsToBeDeletedPerPartition().isEmpty()
|
||||
&& cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().mapToInt(List::size).sum() > 0) {
|
||||
// Only create cleaner plan which does some work
|
||||
final HoodieInstant cleanInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, startCleanTime);
|
||||
// Save to both aux and timeline folder
|
||||
try {
|
||||
table.getActiveTimeline().saveToCleanRequested(cleanInstant, TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan));
|
||||
LOG.info("Requesting Cleaning with instant time " + cleanInstant);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Got exception when saving cleaner requested file", e);
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
return Option.of(cleanerPlan);
|
||||
}
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<HoodieCleanerPlan> execute() {
|
||||
// Plan a new clean action
|
||||
return requestClean(instantTime);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -19,6 +19,8 @@
|
||||
package org.apache.hudi.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.transaction.TransactionManager;
|
||||
import org.apache.hudi.client.utils.TransactionUtils;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
@@ -29,7 +31,9 @@ import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
@@ -58,6 +62,8 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload, I,
|
||||
protected final Option<Map<String, String>> extraMetadata;
|
||||
protected final WriteOperationType operationType;
|
||||
protected final TaskContextSupplier taskContextSupplier;
|
||||
protected final TransactionManager txnManager;
|
||||
protected Option<Pair<HoodieInstant, Map<String, String>>> lastCompletedTxn;
|
||||
|
||||
public BaseCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config,
|
||||
HoodieTable<T, I, K, O> table, String instantTime, WriteOperationType operationType,
|
||||
@@ -66,6 +72,9 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload, I,
|
||||
this.operationType = operationType;
|
||||
this.extraMetadata = extraMetadata;
|
||||
this.taskContextSupplier = context.getTaskContextSupplier();
|
||||
// TODO : Remove this once we refactor and move out autoCommit method from here, since the TxnManager is held in {@link AbstractHoodieWriteClient}.
|
||||
this.txnManager = new TransactionManager(config, table.getMetaClient().getFs());
|
||||
this.lastCompletedTxn = TransactionUtils.getLastCompletedTxnInstantAndMetadata(table.getMetaClient());
|
||||
}
|
||||
|
||||
public abstract HoodieWriteMetadata<O> execute(I inputRecords);
|
||||
@@ -117,12 +126,24 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload, I,
|
||||
protected void commitOnAutoCommit(HoodieWriteMetadata result) {
|
||||
if (config.shouldAutoCommit()) {
|
||||
LOG.info("Auto commit enabled: Committing " + instantTime);
|
||||
commit(extraMetadata, result);
|
||||
autoCommit(extraMetadata, result);
|
||||
} else {
|
||||
LOG.info("Auto commit disabled for " + instantTime);
|
||||
}
|
||||
}
|
||||
|
||||
protected void autoCommit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata<O> result) {
|
||||
this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime)),
|
||||
lastCompletedTxn.isPresent() ? Option.of(lastCompletedTxn.get().getLeft()) : Option.empty());
|
||||
try {
|
||||
TransactionUtils.resolveWriteConflictIfAny(table, this.txnManager.getCurrentTransactionOwner(),
|
||||
result.getCommitMetadata(), config, this.txnManager.getLastCompletedTransactionOwner());
|
||||
commit(extraMetadata, result);
|
||||
} finally {
|
||||
this.txnManager.endTransaction();
|
||||
}
|
||||
}
|
||||
|
||||
protected abstract void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata<O> result);
|
||||
|
||||
/**
|
||||
@@ -140,6 +161,10 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload, I,
|
||||
}
|
||||
}
|
||||
|
||||
protected void syncTableMetadata() {
|
||||
// No Op
|
||||
}
|
||||
|
||||
/**
|
||||
* By default, return the writer schema in Write Config for storing in commit.
|
||||
*/
|
||||
|
||||
@@ -53,22 +53,24 @@ public abstract class BaseScheduleCompactionActionExecutor<T extends HoodieRecor
|
||||
|
||||
@Override
|
||||
public Option<HoodieCompactionPlan> execute() {
|
||||
// if there are inflight writes, their instantTime must not be less than that of compaction instant time
|
||||
table.getActiveTimeline().getCommitsTimeline().filterPendingExcludingCompaction().firstInstant()
|
||||
.ifPresent(earliestInflight -> ValidationUtils.checkArgument(
|
||||
HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), HoodieTimeline.GREATER_THAN, instantTime),
|
||||
"Earliest write inflight instant time must be later than compaction time. Earliest :" + earliestInflight
|
||||
+ ", Compaction scheduled at " + instantTime));
|
||||
|
||||
// Committed and pending compaction instants should have strictly lower timestamps
|
||||
List<HoodieInstant> conflictingInstants = table.getActiveTimeline()
|
||||
.getWriteTimeline().getInstants()
|
||||
.filter(instant -> HoodieTimeline.compareTimestamps(
|
||||
instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime))
|
||||
.collect(Collectors.toList());
|
||||
ValidationUtils.checkArgument(conflictingInstants.isEmpty(),
|
||||
"Following instants have timestamps >= compactionInstant (" + instantTime + ") Instants :"
|
||||
+ conflictingInstants);
|
||||
if (!config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()
|
||||
&& !config.getFailedWritesCleanPolicy().isLazy()) {
|
||||
// if there are inflight writes, their instantTime must not be less than that of compaction instant time
|
||||
table.getActiveTimeline().getCommitsTimeline().filterPendingExcludingCompaction().firstInstant()
|
||||
.ifPresent(earliestInflight -> ValidationUtils.checkArgument(
|
||||
HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), HoodieTimeline.GREATER_THAN, instantTime),
|
||||
"Earliest write inflight instant time must be later than compaction time. Earliest :" + earliestInflight
|
||||
+ ", Compaction scheduled at " + instantTime));
|
||||
// Committed and pending compaction instants should have strictly lower timestamps
|
||||
List<HoodieInstant> conflictingInstants = table.getActiveTimeline()
|
||||
.getWriteTimeline().getInstants()
|
||||
.filter(instant -> HoodieTimeline.compareTimestamps(
|
||||
instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime))
|
||||
.collect(Collectors.toList());
|
||||
ValidationUtils.checkArgument(conflictingInstants.isEmpty(),
|
||||
"Following instants have timestamps >= compactionInstant (" + instantTime + ") Instants :"
|
||||
+ conflictingInstants);
|
||||
}
|
||||
|
||||
HoodieCompactionPlan plan = scheduleCompaction();
|
||||
if (plan != null && (plan.getOperations() != null) && (!plan.getOperations().isEmpty())) {
|
||||
|
||||
@@ -78,6 +78,12 @@ public abstract class AbstractUpgradeDowngrade {
|
||||
public abstract void run(HoodieTableMetaClient metaClient, HoodieTableVersion toVersion, HoodieWriteConfig config,
|
||||
HoodieEngineContext context, String instantTime);
|
||||
|
||||
public boolean needsUpgradeOrDowngrade(HoodieTableVersion toVersion) {
|
||||
HoodieTableVersion fromVersion = metaClient.getTableConfig().getTableVersion();
|
||||
// Ensure no inflight commits & versions are same
|
||||
return toVersion.versionCode() != fromVersion.versionCode();
|
||||
}
|
||||
|
||||
protected AbstractUpgradeDowngrade(HoodieTableMetaClient metaClient, HoodieWriteConfig config, HoodieEngineContext context) {
|
||||
this.metaClient = metaClient;
|
||||
this.config = config;
|
||||
@@ -90,7 +96,7 @@ public abstract class AbstractUpgradeDowngrade {
|
||||
protected void run(HoodieTableVersion toVersion, String instantTime) throws IOException {
|
||||
// Fetch version from property file and current version
|
||||
HoodieTableVersion fromVersion = metaClient.getTableConfig().getTableVersion();
|
||||
if (toVersion.versionCode() == fromVersion.versionCode()) {
|
||||
if (!needsUpgradeOrDowngrade(toVersion)) {
|
||||
return;
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user