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.
*/

View File

@@ -18,10 +18,15 @@
package org.apache.hudi.common.model;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.testutils.MockHoodieTimeline;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -47,4 +52,25 @@ public class TestHoodieFileGroup {
assertTrue(fileGroup.getLatestFileSlice().get().getBaseInstantTime().equals("001"));
assertTrue((new HoodieFileGroup(fileGroup)).getLatestFileSlice().get().getBaseInstantTime().equals("001"));
}
@Test
public void testCommittedFileSlicesWithSavepointAndHoles() {
MockHoodieTimeline activeTimeline = new MockHoodieTimeline(Stream.of(
new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "01"),
new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "01"),
new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "03"),
new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "03"),
new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "05") // this can be DELTA_COMMIT/REPLACE_COMMIT as well
).collect(Collectors.toList()));
HoodieFileGroup fileGroup = new HoodieFileGroup("", "data", activeTimeline.filterCompletedAndCompactionInstants());
for (int i = 0; i < 7; i++) {
HoodieBaseFile baseFile = new HoodieBaseFile("data_1_0" + i);
fileGroup.addBaseFile(baseFile);
}
List<FileSlice> allFileSlices = fileGroup.getAllFileSlices().collect(Collectors.toList());
assertEquals(6, allFileSlices.size());
assertTrue(!allFileSlices.stream().anyMatch(s -> s.getBaseInstantTime().equals("06")));
assertEquals(7, fileGroup.getAllFileSlicesIncludingInflight().count());
assertTrue(fileGroup.getLatestFileSlice().get().getBaseInstantTime().equals("05"));
}
}

View File

@@ -262,6 +262,57 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
assertEquals(instant7.getTimestamp(), timeline.getContiguousCompletedWriteTimeline().lastInstant().get().getTimestamp());
}
@Test
public void testTimelineWithSavepointAndHoles() {
timeline = new MockHoodieTimeline(Stream.of(
new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "01"),
new HoodieInstant(State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "01"),
new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "03"),
new HoodieInstant(State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "03"),
new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "05") // this can be DELTA_COMMIT/REPLACE_COMMIT as well
).collect(Collectors.toList()));
assertTrue(timeline.isBeforeTimelineStarts("00"));
assertTrue(timeline.isBeforeTimelineStarts("01"));
assertTrue(timeline.isBeforeTimelineStarts("02"));
assertTrue(timeline.isBeforeTimelineStarts("03"));
assertTrue(timeline.isBeforeTimelineStarts("04"));
assertFalse(timeline.isBeforeTimelineStarts("05"));
assertFalse(timeline.isBeforeTimelineStarts("06"));
// with an inflight savepoint in between
timeline = new MockHoodieTimeline(Stream.of(
new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "01"),
new HoodieInstant(State.INFLIGHT, HoodieTimeline.SAVEPOINT_ACTION, "01"),
new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "03"),
new HoodieInstant(State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "03"),
new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "05")
).collect(Collectors.toList()));
assertTrue(timeline.isBeforeTimelineStarts("00"));
assertTrue(timeline.isBeforeTimelineStarts("01"));
assertTrue(timeline.isBeforeTimelineStarts("02"));
assertTrue(timeline.isBeforeTimelineStarts("03"));
assertTrue(timeline.isBeforeTimelineStarts("04"));
assertFalse(timeline.isBeforeTimelineStarts("05"));
assertFalse(timeline.isBeforeTimelineStarts("06"));
// with a pending replacecommit after savepoints
timeline = new MockHoodieTimeline(Stream.of(
new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "01"),
new HoodieInstant(State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "01"),
new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "03"),
new HoodieInstant(State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "03"),
new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "05"),
new HoodieInstant(State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, "06")
).collect(Collectors.toList()));
assertTrue(timeline.isBeforeTimelineStarts("00"));
assertTrue(timeline.isBeforeTimelineStarts("01"));
assertTrue(timeline.isBeforeTimelineStarts("02"));
assertTrue(timeline.isBeforeTimelineStarts("03"));
assertTrue(timeline.isBeforeTimelineStarts("04"));
assertFalse(timeline.isBeforeTimelineStarts("05"));
assertFalse(timeline.isBeforeTimelineStarts("06"));
}
@Test
public void testTimelineGetOperations() {
List<HoodieInstant> allInstants = getAllInstants();

View File

@@ -26,6 +26,7 @@ import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
import org.apache.hudi.avro.model.HoodieRollbackPlan;
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFileFormat;
@@ -35,6 +36,7 @@ import org.apache.hudi.common.model.IOType;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.table.view.TableFileSystemView;
import org.apache.hudi.common.util.Option;
@@ -156,6 +158,10 @@ public class FileCreateUtils {
}
}
public static void createSavepointCommit(String basePath, String instantTime, HoodieSavepointMetadata savepointMetadata) throws IOException {
createMetaFile(basePath, instantTime, HoodieTimeline.SAVEPOINT_EXTENSION, TimelineMetadataUtils.serializeSavepointMetadata(savepointMetadata).get());
}
public static void createCommit(String basePath, String instantTime, FileSystem fs) throws IOException {
createMetaFile(basePath, instantTime, HoodieTimeline.COMMIT_EXTENSION, fs);
}
@@ -285,6 +291,10 @@ public class FileCreateUtils {
createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_COMPACTION_EXTENSION);
}
public static void createInflightSavepoint(String basePath, String instantTime) throws IOException {
createAuxiliaryMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_SAVEPOINT_EXTENSION);
}
public static void createPartitionMetaFile(String basePath, String partitionPath) throws IOException {
Path parentPath = Paths.get(basePath, partitionPath);
Files.createDirectories(parentPath);
@@ -439,4 +449,9 @@ public class FileCreateUtils {
public static void deleteDeltaCommit(String basePath, String instantTime, FileSystem fs) throws IOException {
deleteMetaFile(basePath, instantTime, HoodieTimeline.DELTA_COMMIT_EXTENSION, fs);
}
public static void deleteSavepointCommit(String basePath, String instantTime, FileSystem fs) throws IOException {
deleteMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_SAVEPOINT_EXTENSION, fs);
deleteMetaFile(basePath, instantTime, HoodieTimeline.SAVEPOINT_EXTENSION, fs);
}
}

View File

@@ -99,6 +99,7 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightCom
import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightDeltaCommit;
import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightReplaceCommit;
import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightRollbackFile;
import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightSavepoint;
import static org.apache.hudi.common.testutils.FileCreateUtils.createMarkerFile;
import static org.apache.hudi.common.testutils.FileCreateUtils.createReplaceCommit;
import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedCleanFile;
@@ -109,6 +110,8 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedRe
import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedRollbackFile;
import static org.apache.hudi.common.testutils.FileCreateUtils.createRestoreFile;
import static org.apache.hudi.common.testutils.FileCreateUtils.createRollbackFile;
import static org.apache.hudi.common.testutils.FileCreateUtils.createSavepointCommit;
import static org.apache.hudi.common.testutils.FileCreateUtils.deleteSavepointCommit;
import static org.apache.hudi.common.testutils.FileCreateUtils.logFileName;
import static org.apache.hudi.common.util.CleanerUtils.convertCleanMetadata;
import static org.apache.hudi.common.util.CollectionUtils.createImmutableMap;
@@ -199,6 +202,12 @@ public class HoodieTestTable {
return this;
}
public HoodieTestTable addSavepointCommit(String instantTime, HoodieSavepointMetadata savepointMetadata) throws IOException {
createInflightSavepoint(basePath, instantTime);
createSavepointCommit(basePath, instantTime, savepointMetadata);
return this;
}
public HoodieCommitMetadata createCommitMetadata(WriteOperationType operationType, String commitTime,
HoodieTestTableState testTableState) {
String actionType = getCommitActionType(operationType, metaClient.getTableType());
@@ -394,7 +403,7 @@ public class HoodieTestTable {
public HoodieSavepointMetadata getSavepointMetadata(String instant, Map<String, List<String>> partitionToFilesMeta) {
HoodieSavepointMetadata savepointMetadata = new HoodieSavepointMetadata();
savepointMetadata.setSavepointedAt(Long.valueOf(instant));
savepointMetadata.setSavepointedAt(12345L);
Map<String, HoodieSavepointPartitionMetadata> partitionMetadataMap = new HashMap<>();
for (Map.Entry<String, List<String>> entry : partitionToFilesMeta.entrySet()) {
HoodieSavepointPartitionMetadata savepointPartitionMetadata = new HoodieSavepointPartitionMetadata();
@@ -404,6 +413,7 @@ public class HoodieTestTable {
}
savepointMetadata.setPartitionMetadata(partitionMetadataMap);
savepointMetadata.setSavepointedBy("test");
savepointMetadata.setComments("test_comment");
return savepointMetadata;
}
@@ -454,6 +464,17 @@ public class HoodieTestTable {
return this;
}
public HoodieTestTable addSavepoint(String instantTime, HoodieSavepointMetadata savepointMetadata) throws IOException {
createInflightSavepoint(basePath, instantTime);
createSavepointCommit(basePath, instantTime, savepointMetadata);
return this;
}
public HoodieTestTable deleteSavepoint(String instantTime) throws IOException {
deleteSavepointCommit(basePath, instantTime, fs);
return this;
}
public HoodieTestTable forCommit(String instantTime) {
currentInstantTime = instantTime;
return this;

View File

@@ -23,6 +23,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import java.util.Comparator;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -38,4 +39,9 @@ public class MockHoodieTimeline extends HoodieActiveTimeline {
inflights.map(s -> new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, s)))
.sorted(Comparator.comparing(HoodieInstant::getFileName)).collect(Collectors.toList()));
}
public MockHoodieTimeline(List<HoodieInstant> instants) {
super();
this.setInstants(instants);
}
}