1
0

[HUDI-3355] Issue with out of order commits in the timeline when ingestion writers using SparkAllowUpdateStrategy (#4962)

This commit is contained in:
xiarixiaoyao
2022-03-31 06:54:25 +08:00
committed by GitHub
parent 9ff6a48f60
commit 2d73c8ae86
7 changed files with 217 additions and 28 deletions

View File

@@ -87,9 +87,10 @@ import java.nio.charset.StandardCharsets;
import java.text.ParseException;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Set;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -124,6 +125,7 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
protected transient AsyncArchiveService asyncArchiveService;
protected final TransactionManager txnManager;
protected Option<Pair<HoodieInstant, Map<String, String>>> lastCompletedTxnAndMetadata = Option.empty();
protected Set<String> pendingInflightAndRequestedInstants;
/**
* Create a write client, with new hudi index.
@@ -440,6 +442,8 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
HoodieTableMetaClient metaClient) {
setOperationType(writeOperationType);
this.lastCompletedTxnAndMetadata = TransactionUtils.getLastCompletedTxnInstantAndMetadata(metaClient);
this.pendingInflightAndRequestedInstants = TransactionUtils.getInflightAndRequestedInstants(metaClient);
this.pendingInflightAndRequestedInstants.remove(instantTime);
if (null == this.asyncCleanerService) {
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this);
} else {

View File

@@ -26,6 +26,7 @@ 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.CollectionUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
@@ -36,7 +37,9 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.Set;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;
public class TransactionUtils {
@@ -51,26 +54,8 @@ public class TransactionUtils {
* @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 {
return resolveWriteConflictIfAny(table, currentTxnOwnerInstant, thisCommitMetadata, config, lastCompletedTxnOwnerInstant, false);
}
/**
* Resolve any write conflicts when committing data.
* @param pendingInstants
*
* @param table
* @param currentTxnOwnerInstant
* @param thisCommitMetadata
* @param config
* @param lastCompletedTxnOwnerInstant
* @return
* @throws HoodieWriteConflictException
*/
@@ -80,11 +65,16 @@ public class TransactionUtils {
final Option<HoodieCommitMetadata> thisCommitMetadata,
final HoodieWriteConfig config,
Option<HoodieInstant> lastCompletedTxnOwnerInstant,
boolean reloadActiveTimeline) throws HoodieWriteConflictException {
boolean reloadActiveTimeline,
Set<String> pendingInstants) throws HoodieWriteConflictException {
if (config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) {
// deal with pendingInstants
Stream<HoodieInstant> completedInstantsDuringCurrentWriteOperation = getCompletedInstantsDuringCurrentWriteOperation(table.getMetaClient(), pendingInstants);
ConflictResolutionStrategy resolutionStrategy = config.getWriteConflictResolutionStrategy();
Stream<HoodieInstant> instantStream = resolutionStrategy.getCandidateInstants(reloadActiveTimeline
? table.getMetaClient().reloadActiveTimeline() : table.getActiveTimeline(), currentTxnOwnerInstant.get(), lastCompletedTxnOwnerInstant);
Stream<HoodieInstant> instantStream = Stream.concat(resolutionStrategy.getCandidateInstants(reloadActiveTimeline
? table.getMetaClient().reloadActiveTimeline() : table.getActiveTimeline(), currentTxnOwnerInstant.get(), lastCompletedTxnOwnerInstant),
completedInstantsDuringCurrentWriteOperation);
final ConcurrentOperation thisOperation = new ConcurrentOperation(currentTxnOwnerInstant.get(), thisCommitMetadata.orElse(new HoodieCommitMetadata()));
instantStream.forEach(instant -> {
try {
@@ -137,4 +127,35 @@ public class TransactionUtils {
throw new HoodieIOException("Unable to read metadata for instant " + hoodieInstantOption.get(), io);
}
}
}
/**
* Get InflightAndRequest instants.
*
* @param metaClient
* @return
*/
public static Set<String> getInflightAndRequestedInstants(HoodieTableMetaClient metaClient) {
// collect InflightAndRequest instants for deltaCommit/commit/compaction/clustering
Set<String> timelineActions = CollectionUtils
.createImmutableSet(HoodieTimeline.REPLACE_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMMIT_ACTION);
return metaClient
.getActiveTimeline()
.getTimelineOfActions(timelineActions)
.filterInflightsAndRequested()
.getInstants()
.map(HoodieInstant::getTimestamp)
.collect(Collectors.toSet());
}
public static Stream<HoodieInstant> getCompletedInstantsDuringCurrentWriteOperation(HoodieTableMetaClient metaClient, Set<String> pendingInstants) {
// deal with pendingInstants
// some pending instants maybe finished during current write operation,
// we should check the conflict of those pending operation
return metaClient
.reloadActiveTimeline()
.getCommitsTimeline()
.filterCompletedInstants()
.getInstants()
.filter(f -> pendingInstants.contains(f.getTimestamp()));
}
}

View File

@@ -80,6 +80,7 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload, I,
protected final TaskContextSupplier taskContextSupplier;
protected final TransactionManager txnManager;
protected Option<Pair<HoodieInstant, Map<String, String>>> lastCompletedTxn;
protected Set<String> pendingInflightAndRequestedInstants;
public BaseCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config,
HoodieTable<T, I, K, O> table, String instantTime, WriteOperationType operationType,
@@ -91,6 +92,8 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload, I,
// TODO : Remove this once we refactor and move out autoCommit method from here, since the TxnManager is held in {@link BaseHoodieWriteClient}.
this.txnManager = new TransactionManager(config, table.getMetaClient().getFs());
this.lastCompletedTxn = TransactionUtils.getLastCompletedTxnInstantAndMetadata(table.getMetaClient());
this.pendingInflightAndRequestedInstants = TransactionUtils.getInflightAndRequestedInstants(table.getMetaClient());
this.pendingInflightAndRequestedInstants.remove(instantTime);
if (table.getStorageLayout().doesNotSupport(operationType)) {
throw new UnsupportedOperationException("Executor " + this.getClass().getSimpleName()
+ " is not compatible with table layout " + table.getStorageLayout().getClass().getSimpleName());
@@ -184,7 +187,7 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload, I,
setCommitMetadata(result);
// reload active timeline so as to get all updates after current transaction have started. hence setting last arg to true.
TransactionUtils.resolveWriteConflictIfAny(table, this.txnManager.getCurrentTransactionOwner(),
result.getCommitMetadata(), config, this.txnManager.getLastCompletedTransactionOwner(), true);
result.getCommitMetadata(), config, this.txnManager.getLastCompletedTransactionOwner(), true, pendingInflightAndRequestedInstants);
commit(extraMetadata, result);
} finally {
this.txnManager.endTransaction(inflightInstant);

View File

@@ -23,6 +23,7 @@ import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hudi.avro.model.HoodieClusteringGroup;
@@ -31,6 +32,7 @@ import org.apache.hudi.avro.model.HoodieCompactionOperation;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
import org.apache.hudi.avro.model.HoodieSliceInfo;
import org.apache.hudi.client.utils.TransactionUtils;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.HoodieWriteStat;
@@ -40,6 +42,7 @@ 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.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.testutils.FileCreateUtils;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.testutils.HoodieTestTable;
@@ -318,16 +321,20 @@ public class TestSimpleConcurrentFileWritesConflictResolutionStrategy extends Ho
.withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2);
}
private HoodieCommitMetadata createCommitMetadata(String instantTime) {
private HoodieCommitMetadata createCommitMetadata(String instantTime, String writeFileName) {
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
commitMetadata.addMetadata("test", "test");
HoodieWriteStat writeStat = new HoodieWriteStat();
writeStat.setFileId("file-1");
writeStat.setFileId(writeFileName);
commitMetadata.addWriteStat(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, writeStat);
commitMetadata.setOperationType(WriteOperationType.INSERT);
return commitMetadata;
}
private HoodieCommitMetadata createCommitMetadata(String instantTime) {
return createCommitMetadata(instantTime, "file-1");
}
private void createInflightCommit(String instantTime) throws Exception {
String fileId1 = "file-" + instantTime + "-1";
String fileId2 = "file-" + instantTime + "-2";
@@ -417,4 +424,147 @@ public class TestSimpleConcurrentFileWritesConflictResolutionStrategy extends Ho
.withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2);
}
// try to simulate HUDI-3355
@Test
public void testConcurrentWritesWithPendingInstants() throws Exception {
// step1: create a pending replace/commit/compact instant: C1,C11,C12
String newInstantTimeC1 = HoodieActiveTimeline.createNewInstantTime();
createPendingReplace(newInstantTimeC1, WriteOperationType.CLUSTER);
String newCompactionInstantTimeC11 = HoodieActiveTimeline.createNewInstantTime();
createPendingCompaction(newCompactionInstantTimeC11);
String newCommitInstantTimeC12 = HoodieActiveTimeline.createNewInstantTime();
createInflightCommit(newCommitInstantTimeC12);
// step2: create a complete commit which has no conflict with C1,C11,C12, named it as C2
createCommit(HoodieActiveTimeline.createNewInstantTime());
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
// consider commits before this are all successful
Option<HoodieInstant> lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant();
// step3: write 1 starts, which has conflict with C1,C11,C12, named it as C3
String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime();
createInflightCommit(currentWriterInstant);
// step4: create a requested commit, which has conflict with C3, named it as C4
String commitC4 = HoodieActiveTimeline.createNewInstantTime();
createRequestedCommit(commitC4);
// get PendingCommit during write 1 operation
metaClient.reloadActiveTimeline();
Set<String> pendingInstant = TransactionUtils.getInflightAndRequestedInstants(metaClient);
pendingInstant.remove(currentWriterInstant);
// step5: finished pending cluster/compaction/commit operation
createCompleteReplace(newInstantTimeC1, WriteOperationType.CLUSTER);
createCompleteCompaction(newCompactionInstantTimeC11);
createCompleteCommit(newCommitInstantTimeC12);
createCompleteCommit(commitC4);
// step6: do check
Option<HoodieInstant> currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant));
SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy();
// make sure c3 has conflict with C1,C11,C12,C4;
HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant, "file-2");
timeline.reload();
List<HoodieInstant> completedInstantsDuringCurrentWriteOperation = TransactionUtils
.getCompletedInstantsDuringCurrentWriteOperation(metaClient, pendingInstant).collect(Collectors.toList());
// C1,C11,C12,C4 should be included
Assertions.assertTrue(completedInstantsDuringCurrentWriteOperation.size() == 4);
ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata);
// check C3 has conflict with C1,C11,C12,C4
for (HoodieInstant instant : completedInstantsDuringCurrentWriteOperation) {
ConcurrentOperation thatCommitOperation = new ConcurrentOperation(instant, metaClient);
Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation));
try {
strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation);
} catch (HoodieWriteConflictException e) {
// expected
}
}
}
private void createPendingReplace(String instantTime, WriteOperationType writeOperationType) throws Exception {
String fileId1 = "file-1";
String fileId2 = "file-2";
// create replace instant to mark fileId2 as deleted
HoodieRequestedReplaceMetadata requestedReplaceMetadata = new HoodieRequestedReplaceMetadata();
requestedReplaceMetadata.setOperationType(WriteOperationType.CLUSTER.name());
HoodieClusteringPlan clusteringPlan = new HoodieClusteringPlan();
HoodieClusteringGroup clusteringGroup = new HoodieClusteringGroup();
HoodieSliceInfo sliceInfo = new HoodieSliceInfo();
sliceInfo.setFileId(fileId2);
sliceInfo.setPartitionPath(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH);
clusteringGroup.setSlices(Arrays.asList(sliceInfo));
clusteringPlan.setInputGroups(Arrays.asList(clusteringGroup));
requestedReplaceMetadata.setClusteringPlan(clusteringPlan);
requestedReplaceMetadata.setVersion(TimelineLayoutVersion.CURR_VERSION);
HoodieTestTable.of(metaClient)
.addPendingReplace(instantTime, Option.of(requestedReplaceMetadata), Option.empty())
.withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2);
}
private void createCompleteReplace(String instantTime, WriteOperationType writeOperationType) throws Exception {
String fileId1 = "file-1";
String fileId2 = "file-2";
// create replace instant to mark fileId2 as deleted
HoodieReplaceCommitMetadata replaceMetadata = new HoodieReplaceCommitMetadata();
Map<String, List<String>> partitionFileIds = new HashMap<>();
partitionFileIds.put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, Arrays.asList(fileId2));
replaceMetadata.setPartitionToReplaceFileIds(partitionFileIds);
HoodieWriteStat writeStat = new HoodieWriteStat();
writeStat.setFileId("file-2");
replaceMetadata.addWriteStat(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, writeStat);
replaceMetadata.setOperationType(writeOperationType);
FileCreateUtils.createReplaceCommit(metaClient.getBasePath(), instantTime, replaceMetadata);
}
private void createPendingCompaction(String instantTime) throws Exception {
String fileId1 = "file-2";
HoodieCompactionPlan compactionPlan = new HoodieCompactionPlan();
compactionPlan.setVersion(TimelineLayoutVersion.CURR_VERSION);
HoodieCompactionOperation operation = new HoodieCompactionOperation();
operation.setFileId(fileId1);
operation.setPartitionPath(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH);
operation.setDataFilePath("/file-2");
operation.setDeltaFilePaths(Arrays.asList("/file-2"));
compactionPlan.setOperations(Arrays.asList(operation));
HoodieTestTable.of(metaClient)
.addRequestedCompaction(instantTime, compactionPlan);
FileCreateUtils.createPendingInflightCompaction(metaClient.getBasePath(), instantTime);
}
private void createCompleteCompaction(String instantTime) throws Exception {
String fileId1 = "file-1";
String fileId2 = "file-2";
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
commitMetadata.addMetadata("test", "test");
commitMetadata.setOperationType(WriteOperationType.COMPACT);
commitMetadata.setCompacted(true);
HoodieWriteStat writeStat = new HoodieWriteStat();
writeStat.setFileId("file-2");
commitMetadata.addWriteStat(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, writeStat);
HoodieTestTable.of(metaClient)
.addCommit(instantTime, Option.of(commitMetadata))
.withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2);
}
private void createRequestedCommit(String instantTime) throws Exception {
HoodieTestTable.of(metaClient)
.addInflightCommit(instantTime);
}
private void createCompleteCommit(String instantTime) throws Exception {
String fileId1 = "file-1";
String fileId2 = "file-2";
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
commitMetadata.addMetadata("test", "test");
HoodieWriteStat writeStat = new HoodieWriteStat();
writeStat.setFileId("file-2");
commitMetadata.addWriteStat(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, writeStat);
commitMetadata.setOperationType(WriteOperationType.INSERT);
HoodieTestTable.of(metaClient)
.addCommit(instantTime, Option.of(commitMetadata))
.withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2);
}
}

View File

@@ -471,7 +471,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
// Important to create this after the lock to ensure the latest commits show up in the timeline without need for reload
HoodieTable table = createTable(config, hadoopConf);
TransactionUtils.resolveWriteConflictIfAny(table, this.txnManager.getCurrentTransactionOwner(),
Option.of(metadata), config, txnManager.getLastCompletedTransactionOwner());
Option.of(metadata), config, txnManager.getLastCompletedTransactionOwner(), false, this.pendingInflightAndRequestedInstants);
}
@Override

View File

@@ -274,6 +274,10 @@ public class FileCreateUtils {
createAuxiliaryMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_COMPACTION_EXTENSION);
}
public static void createPendingInflightCompaction(String basePath, String instantTime) throws IOException {
createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_COMPACTION_EXTENSION);
}
public static void createPartitionMetaFile(String basePath, String partitionPath) throws IOException {
Path parentPath = Paths.get(basePath, partitionPath);
Files.createDirectories(parentPath);

View File

@@ -257,6 +257,13 @@ public class HoodieTestTable {
return this;
}
public HoodieTestTable addPendingReplace(String instantTime, Option<HoodieRequestedReplaceMetadata> requestedReplaceMetadata, Option<HoodieCommitMetadata> inflightReplaceMetadata) throws Exception {
createRequestedReplaceCommit(basePath, instantTime, requestedReplaceMetadata);
createInflightReplaceCommit(basePath, instantTime, inflightReplaceMetadata);
currentInstantTime = instantTime;
return this;
}
public HoodieTestTable addRequestedReplace(String instantTime, Option<HoodieRequestedReplaceMetadata> requestedReplaceMetadata) throws Exception {
createRequestedReplaceCommit(basePath, instantTime, requestedReplaceMetadata);
currentInstantTime = instantTime;