1
0

[HUDI-3884] Support archival beyond savepoint commits (#5837)

Co-authored-by: sivabalan <n.siva.b@gmail.com>
This commit is contained in:
Sagar Sumit
2022-07-26 00:12:29 +05:30
committed by GitHub
parent eee6a02f77
commit 6e7ac45735
14 changed files with 364 additions and 60 deletions

View File

@@ -28,6 +28,10 @@ import java.util.List;
import java.util.TreeMap;
import java.util.stream.Stream;
import static org.apache.hudi.common.table.timeline.HoodieTimeline.LESSER_THAN;
import static org.apache.hudi.common.table.timeline.HoodieTimeline.LESSER_THAN_OR_EQUALS;
import static org.apache.hudi.common.table.timeline.HoodieTimeline.compareTimestamps;
/**
* A set of data/base files + set of log files, that make up an unit for all operations.
*/
@@ -118,21 +122,22 @@ public class HoodieFileGroup implements Serializable {
* some log files, that are based off a commit or delta commit.
*/
private boolean isFileSliceCommitted(FileSlice slice) {
String maxCommitTime = lastInstant.get().getTimestamp();
return timeline.containsOrBeforeTimelineStarts(slice.getBaseInstantTime())
&& HoodieTimeline.compareTimestamps(slice.getBaseInstantTime(), HoodieTimeline.LESSER_THAN_OR_EQUALS, maxCommitTime);
if (!compareTimestamps(slice.getBaseInstantTime(), LESSER_THAN_OR_EQUALS, lastInstant.get().getTimestamp())) {
return false;
}
return timeline.containsOrBeforeTimelineStarts(slice.getBaseInstantTime());
}
/**
* Get all the the file slices including in-flight ones as seen in underlying file-system.
* Get all the file slices including in-flight ones as seen in underlying file system.
*/
public Stream<FileSlice> getAllFileSlicesIncludingInflight() {
return fileSlices.values().stream();
}
/**
* Get latest file slices including in-flight ones.
* Get the latest file slices including inflight ones.
*/
public Option<FileSlice> getLatestFileSlicesIncludingInflight() {
return Option.fromJavaOptional(getAllFileSlicesIncludingInflight().findFirst());
@@ -169,8 +174,7 @@ public class HoodieFileGroup implements Serializable {
* Obtain the latest file slice, upto a instantTime i.e <= maxInstantTime.
*/
public Option<FileSlice> getLatestFileSliceBeforeOrOn(String maxInstantTime) {
return Option.fromJavaOptional(getAllFileSlices().filter(slice -> HoodieTimeline
.compareTimestamps(slice.getBaseInstantTime(), HoodieTimeline.LESSER_THAN_OR_EQUALS, maxInstantTime)).findFirst());
return Option.fromJavaOptional(getAllFileSlices().filter(slice -> compareTimestamps(slice.getBaseInstantTime(), LESSER_THAN_OR_EQUALS, maxInstantTime)).findFirst());
}
/**
@@ -181,7 +185,7 @@ public class HoodieFileGroup implements Serializable {
*/
public Option<FileSlice> getLatestFileSliceBefore(String maxInstantTime) {
return Option.fromJavaOptional(getAllFileSlices().filter(
slice -> HoodieTimeline.compareTimestamps(slice.getBaseInstantTime(), HoodieTimeline.LESSER_THAN, maxInstantTime))
slice -> compareTimestamps(slice.getBaseInstantTime(), LESSER_THAN, maxInstantTime))
.findFirst());
}

View File

@@ -35,6 +35,7 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
import static java.util.Collections.reverse;
import static org.apache.hudi.common.table.timeline.HoodieTimeline.compareTimestamps;
/**
* HoodieDefaultTimeline is a default implementation of the HoodieTimeline. It provides methods to inspect a
@@ -118,7 +119,7 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
Option<HoodieInstant> earliestPending = getWriteTimeline().filterInflightsAndRequested().firstInstant();
if (earliestPending.isPresent()) {
return getWriteTimeline().filterCompletedInstants()
.filter(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), LESSER_THAN, earliestPending.get().getTimestamp()));
.filter(instant -> compareTimestamps(instant.getTimestamp(), LESSER_THAN, earliestPending.get().getTimestamp()));
}
return getWriteTimeline().filterCompletedInstants();
}
@@ -156,34 +157,34 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
@Override
public HoodieDefaultTimeline findInstantsAfter(String instantTime, int numCommits) {
return new HoodieDefaultTimeline(instants.stream()
.filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), GREATER_THAN, instantTime)).limit(numCommits),
.filter(s -> compareTimestamps(s.getTimestamp(), GREATER_THAN, instantTime)).limit(numCommits),
details);
}
@Override
public HoodieTimeline findInstantsAfter(String instantTime) {
return new HoodieDefaultTimeline(instants.stream()
.filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), GREATER_THAN, instantTime)), details);
.filter(s -> compareTimestamps(s.getTimestamp(), GREATER_THAN, instantTime)), details);
}
@Override
public HoodieDefaultTimeline findInstantsAfterOrEquals(String commitTime, int numCommits) {
return new HoodieDefaultTimeline(instants.stream()
.filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), GREATER_THAN_OR_EQUALS, commitTime))
.filter(s -> compareTimestamps(s.getTimestamp(), GREATER_THAN_OR_EQUALS, commitTime))
.limit(numCommits), details);
}
@Override
public HoodieDefaultTimeline findInstantsBefore(String instantTime) {
return new HoodieDefaultTimeline(instants.stream()
.filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), LESSER_THAN, instantTime)),
.filter(s -> compareTimestamps(s.getTimestamp(), LESSER_THAN, instantTime)),
details);
}
@Override
public HoodieDefaultTimeline findInstantsBeforeOrEquals(String instantTime) {
return new HoodieDefaultTimeline(instants.stream()
.filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), LESSER_THAN_OR_EQUALS, instantTime)),
.filter(s -> compareTimestamps(s.getTimestamp(), LESSER_THAN_OR_EQUALS, instantTime)),
details);
}
@@ -362,11 +363,28 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
@Override
public boolean isBeforeTimelineStarts(String instant) {
Option<HoodieInstant> firstCommit = firstInstant();
return firstCommit.isPresent()
&& HoodieTimeline.compareTimestamps(instant, LESSER_THAN, firstCommit.get().getTimestamp());
Option<HoodieInstant> firstNonSavepointCommit = getFirstNonSavepointCommit();
return firstNonSavepointCommit.isPresent()
&& compareTimestamps(instant, LESSER_THAN, firstNonSavepointCommit.get().getTimestamp());
}
public Option<HoodieInstant> getFirstNonSavepointCommit() {
Option<HoodieInstant> firstCommit = firstInstant();
Set<String> savepointTimestamps = instants.stream()
.filter(entry -> entry.getAction().equals(HoodieTimeline.SAVEPOINT_ACTION))
.map(HoodieInstant::getTimestamp)
.collect(Collectors.toSet());
Option<HoodieInstant> firstNonSavepointCommit = firstCommit;
if (!savepointTimestamps.isEmpty()) {
// There are chances that there could be holes in the timeline due to archival and savepoint interplay.
// So, the first non-savepoint commit is considered as beginning of the active timeline.
firstNonSavepointCommit = Option.fromJavaOptional(instants.stream()
.filter(entry -> !savepointTimestamps.contains(entry.getTimestamp()))
.findFirst());
}
return firstNonSavepointCommit;
}
@Override
public Option<byte[]> getInstantDetails(HoodieInstant instant) {
return details.apply(instant);

View File

@@ -305,6 +305,15 @@ public interface HoodieTimeline extends Serializable {
*/
boolean isBeforeTimelineStarts(String ts);
/**
* First non-savepoint commit in the active data timeline. Examples:
* 1. An active data timeline C1, C2, C3, C4, C5 returns C1.
* 2. If archival is allowed beyond savepoint and let's say C1, C2, C4 have been archived
* while C3, C5 have been savepointed, then for the data timeline
* C3, C3_Savepoint, C5, C5_Savepoint, C6, C7 returns C6.
*/
Option<HoodieInstant> getFirstNonSavepointCommit();
/**
* Read the completed instant details.
*/