1
0

[HUDI-4275] Refactor rollback inflight instant for clustering/compaction to reuse some code (#5894)

This commit is contained in:
huberylee
2022-06-20 14:29:21 +08:00
committed by GitHub
parent c5c4cfec91
commit d4f0326b4b
7 changed files with 51 additions and 51 deletions

View File

@@ -18,10 +18,6 @@
package org.apache.hudi.common.table.timeline;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -32,6 +28,11 @@ import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -347,16 +348,15 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
}
/**
* Revert compaction State from inflight to requested.
* Revert instant state from inflight to requested.
*
* @param inflightInstant Inflight Instant
* @return requested instant
*/
public HoodieInstant revertCompactionInflightToRequested(HoodieInstant inflightInstant) {
ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION));
public HoodieInstant revertInstantFromInflightToRequested(HoodieInstant inflightInstant) {
ValidationUtils.checkArgument(inflightInstant.isInflight());
HoodieInstant requestedInstant =
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, inflightInstant.getTimestamp());
new HoodieInstant(State.REQUESTED, inflightInstant.getAction(), inflightInstant.getTimestamp());
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
// Pass empty data since it is read from the corresponding .aux/.compaction instant file
transitionState(inflightInstant, requestedInstant, Option.empty());
@@ -514,26 +514,6 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
return commitInstant;
}
/**
* Revert replace requested State from inflight to requested.
*
* @param inflightInstant Inflight Instant
* @return requested instant
*/
public HoodieInstant revertReplaceCommitInflightToRequested(HoodieInstant inflightInstant) {
ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION));
ValidationUtils.checkArgument(inflightInstant.isInflight());
HoodieInstant requestedInstant =
new HoodieInstant(State.REQUESTED, REPLACE_COMMIT_ACTION, inflightInstant.getTimestamp());
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
// Pass empty data since it is read from the corresponding .aux/.compaction instant file
transitionState(inflightInstant, requestedInstant, Option.empty());
} else {
deleteInflight(inflightInstant);
}
return requestedInstant;
}
private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option<byte[]> data) {
transitionState(fromInstant, toInstant, data, false);
}

View File

@@ -336,7 +336,7 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
timeline = timeline.reload();
assertFalse(timeline.containsInstant(compaction));
assertTrue(timeline.containsInstant(inflight));
compaction = timeline.revertCompactionInflightToRequested(inflight);
compaction = timeline.revertInstantFromInflightToRequested(inflight);
timeline = timeline.reload();
assertTrue(timeline.containsInstant(compaction));
assertFalse(timeline.containsInstant(inflight));