[HUDI-2595] Fixing metadata table updates such that only regular writes from data table can trigger table services in metadata table (#3900)
This commit is contained in:
committed by
GitHub
parent
7aaf47e716
commit
6d109c6de5
@@ -258,10 +258,10 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void preCommit(String instantTime, HoodieCommitMetadata metadata) {
|
||||
protected void preCommit(HoodieInstant inflightInstant, HoodieCommitMetadata metadata) {
|
||||
this.metadataWriterOption.ifPresent(w -> {
|
||||
w.initTableMetadata(); // refresh the timeline
|
||||
w.update(metadata, instantTime);
|
||||
w.update(metadata, inflightInstant.getTimestamp(), getHoodieTable().isTableServiceAction(inflightInstant.getAction()));
|
||||
});
|
||||
}
|
||||
|
||||
@@ -406,7 +406,7 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
this.txnManager.beginTransaction(Option.of(hoodieInstant), Option.empty());
|
||||
// Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a
|
||||
// single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition.
|
||||
table.getMetadataWriter().ifPresent(w -> w.update(commitMetadata, hoodieInstant.getTimestamp()));
|
||||
table.getMetadataWriter().ifPresent(w -> w.update(commitMetadata, hoodieInstant.getTimestamp(), table.isTableServiceAction(hoodieInstant.getAction())));
|
||||
} finally {
|
||||
this.txnManager.endTransaction();
|
||||
}
|
||||
|
||||
@@ -90,7 +90,7 @@ public class FlinkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void commit(List<HoodieRecord> records, String partitionName, String instantTime) {
|
||||
protected void commit(List<HoodieRecord> records, String partitionName, String instantTime, boolean canTriggerTableService) {
|
||||
ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled");
|
||||
List<HoodieRecord> recordList = prepRecords(records, partitionName, 1);
|
||||
|
||||
@@ -125,8 +125,10 @@ public class FlinkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
|
||||
|
||||
// reload timeline
|
||||
metadataMetaClient.reloadActiveTimeline();
|
||||
compactIfNecessary(writeClient, instantTime);
|
||||
doClean(writeClient, instantTime);
|
||||
if (canTriggerTableService) {
|
||||
compactIfNecessary(writeClient, instantTime);
|
||||
doClean(writeClient, instantTime);
|
||||
}
|
||||
}
|
||||
|
||||
// Update total size of the metadata and count of base/log files
|
||||
|
||||
@@ -35,6 +35,7 @@ import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
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.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
@@ -84,6 +85,11 @@ public class HoodieFlinkCopyOnWriteTable<T extends HoodieRecordPayload>
|
||||
super(config, context, metaClient);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isTableServiceAction(String actionType) {
|
||||
return !actionType.equals(HoodieTimeline.COMMIT_ACTION);
|
||||
}
|
||||
|
||||
/**
|
||||
* Upsert a batch of new records into Hoodie table at the supplied instantTime.
|
||||
*
|
||||
|
||||
@@ -27,6 +27,7 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
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.ValidationUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
@@ -54,6 +55,11 @@ public class HoodieFlinkMergeOnReadTable<T extends HoodieRecordPayload>
|
||||
super(config, context, metaClient);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isTableServiceAction(String actionType) {
|
||||
return !actionType.equals(HoodieTimeline.DELTA_COMMIT_ACTION);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<List<WriteStatus>> upsert(
|
||||
HoodieEngineContext context,
|
||||
|
||||
@@ -147,7 +147,7 @@ public abstract class BaseFlinkCommitActionExecutor<T extends HoodieRecordPayloa
|
||||
HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(),
|
||||
extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType());
|
||||
|
||||
writeTableMetadata(metadata);
|
||||
writeTableMetadata(metadata, actionType);
|
||||
|
||||
activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime),
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
|
||||
Reference in New Issue
Block a user