1
0

[HUDI-1072] Introduce REPLACE top level action. Implement insert_overwrite operation on top of replace action (#2048)

This commit is contained in:
satishkotha
2020-09-29 17:04:25 -07:00
committed by GitHub
parent 32c9cad52c
commit a99e93bed5
60 changed files with 2129 additions and 380 deletions

View File

@@ -135,7 +135,7 @@ public class TestTimelineUtils extends HoodieCommonTestHarness {
String ts = i + "";
HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, ts);
activeTimeline.createNewInstant(instant);
activeTimeline.saveAsComplete(instant, Option.of(getRestoreMetadata(basePath, ts, ts, 2)));
activeTimeline.saveAsComplete(instant, Option.of(getRestoreMetadata(basePath, ts, ts, 2, HoodieTimeline.COMMIT_ACTION)));
}
metaClient.reloadActiveTimeline();
@@ -184,25 +184,23 @@ public class TestTimelineUtils extends HoodieCommonTestHarness {
assertEquals(extraMetadataValue1, extraMetadataEntries.get("1").get());
}
private byte[] getRestoreMetadata(String basePath, String partition, String commitTs, int count) throws IOException {
HoodieRestoreMetadata metadata = new HoodieRestoreMetadata();
private byte[] getRestoreMetadata(String basePath, String partition, String commitTs, int count, String actionType) throws IOException {
List<HoodieRollbackMetadata> rollbackM = new ArrayList<>();
rollbackM.add(getRollbackMetadataInstance(basePath, partition, commitTs, count));
metadata.setHoodieRestoreMetadata(CollectionUtils.createImmutableMap(commitTs, rollbackM));
List<String> rollbackInstants = new ArrayList<>();
rollbackInstants.add(commitTs);
metadata.setInstantsToRollback(rollbackInstants);
metadata.setStartRestoreTime(commitTs);
rollbackM.add(getRollbackMetadataInstance(basePath, partition, commitTs, count, actionType));
List<HoodieInstant> rollbackInstants = new ArrayList<>();
rollbackInstants.add(new HoodieInstant(false, commitTs, actionType));
HoodieRestoreMetadata metadata = TimelineMetadataUtils.convertRestoreMetadata(commitTs, 200, rollbackInstants,
CollectionUtils.createImmutableMap(commitTs, rollbackM));
return TimelineMetadataUtils.serializeRestoreMetadata(metadata).get();
}
private HoodieRollbackMetadata getRollbackMetadataInstance(String basePath, String partition, String commitTs, int count) {
private HoodieRollbackMetadata getRollbackMetadataInstance(String basePath, String partition, String commitTs, int count, String actionType) {
List<String> deletedFiles = new ArrayList<>();
for (int i = 1; i <= count; i++) {
deletedFiles.add("file-" + i);
}
List<String> rollbacks = new ArrayList<>();
rollbacks.add(commitTs);
List<HoodieInstant> rollbacks = new ArrayList<>();
rollbacks.add(new HoodieInstant(false, actionType, commitTs));
HoodieRollbackStat rollbackStat = new HoodieRollbackStat(partition, deletedFiles, Collections.emptyList(), Collections.emptyMap());
List<HoodieRollbackStat> rollbackStats = new ArrayList<>();

View File

@@ -199,10 +199,10 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
// Test that various types of getXXX operations from HoodieActiveTimeline
// return the correct set of Instant
checkTimeline.accept(timeline.getCommitsTimeline(),
CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION));
CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION));
checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
checkTimeline.accept(timeline.getCommitTimeline(), Collections.singleton(HoodieTimeline.COMMIT_ACTION));
CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION));
checkTimeline.accept(timeline.getCommitTimeline(), CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION));
checkTimeline.accept(timeline.getDeltaCommitTimeline(), Collections.singleton(HoodieTimeline.DELTA_COMMIT_ACTION));
checkTimeline.accept(timeline.getCleanerTimeline(), Collections.singleton(HoodieTimeline.CLEAN_ACTION));
checkTimeline.accept(timeline.getRollbackTimeline(), Collections.singleton(HoodieTimeline.ROLLBACK_ACTION));
@@ -210,7 +210,7 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
checkTimeline.accept(timeline.getSavePointTimeline(), Collections.singleton(HoodieTimeline.SAVEPOINT_ACTION));
checkTimeline.accept(timeline.getAllCommitsTimeline(),
CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION,
HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
// Get some random Instants
@@ -404,6 +404,27 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
.forEach(i -> assertFalse(t2.containsInstant(i)));
}
@Test
public void testReplaceActionsTimeline() {
int instantTime = 1;
List<HoodieInstant> allInstants = new ArrayList<>();
HoodieInstant instant = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, String.format("%03d", instantTime++));
allInstants.add(instant);
instant = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, String.format("%03d", instantTime++));
allInstants.add(instant);
instant = new HoodieInstant(State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, String.format("%03d", instantTime++));
allInstants.add(instant);
timeline = new HoodieActiveTimeline(metaClient);
timeline.setInstants(allInstants);
List<HoodieInstant> validReplaceInstants =
timeline.getCompletedReplaceTimeline().getInstants().collect(Collectors.toList());
assertEquals(1, validReplaceInstants.size());
assertEquals(instant.getTimestamp(), validReplaceInstants.get(0).getTimestamp());
assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, validReplaceInstants.get(0).getAction());
}
/**
* Returns an exhaustive list of all possible HoodieInstant.
* @return list of HoodieInstant

View File

@@ -18,6 +18,9 @@
package org.apache.hudi.common.table.view;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.avro.model.HoodieFSPermission;
import org.apache.hudi.avro.model.HoodieFileStatus;
@@ -31,10 +34,12 @@ import org.apache.hudi.common.model.BootstrapFileMapping;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
@@ -44,13 +49,10 @@ import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
import org.apache.hudi.common.table.view.TableFileSystemView.SliceView;
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.util.CommitUtils;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.junit.jupiter.api.BeforeEach;
@@ -61,11 +63,13 @@ import org.junit.jupiter.params.provider.MethodSource;
import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -1273,6 +1277,153 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
}
}
@Test
public void testReplaceWithTimeTravel() throws IOException {
String partitionPath1 = "2020/06/27";
new File(basePath + "/" + partitionPath1).mkdirs();
// create 2 fileId in partition1 - fileId1 is replaced later on.
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
assertFalse(roView.getLatestBaseFiles(partitionPath1)
.anyMatch(dfile -> dfile.getFileId().equals(fileId1) || dfile.getFileId().equals(fileId2)),
"No commit, should not find any data file");
// Only one commit
String commitTime1 = "1";
String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1);
String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2);
new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile();
new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile();
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1);
saveAsComplete(commitTimeline, instant1, Option.empty());
refreshFsView();
assertEquals(1, roView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId1)).count());
assertEquals(1, roView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId2)).count());
// create commit2 - fileId1 is replaced. new file groups fileId3,fileId4 are created.
String fileId3 = UUID.randomUUID().toString();
String fileId4 = UUID.randomUUID().toString();
String fileName3 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId3);
String fileName4 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId4);
new File(basePath + "/" + partitionPath1 + "/" + fileName3).createNewFile();
new File(basePath + "/" + partitionPath1 + "/" + fileName4).createNewFile();
String commitTime2 = "2";
Map<String, List<String>> partitionToReplaceFileIds = new HashMap<>();
List<String> replacedFileIds = new ArrayList<>();
replacedFileIds.add(fileId1);
partitionToReplaceFileIds.put(partitionPath1, replacedFileIds);
HoodieCommitMetadata commitMetadata =
CommitUtils.buildMetadata(Collections.emptyList(), partitionToReplaceFileIds, Option.empty(), WriteOperationType.INSERT_OVERWRITE, "", HoodieTimeline.REPLACE_COMMIT_ACTION);
commitTimeline = metaClient.getActiveTimeline();
HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.REPLACE_COMMIT_ACTION, commitTime2);
saveAsComplete(commitTimeline, instant2, Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
//make sure view doesnt include fileId1
refreshFsView();
assertEquals(0, roView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId1)).count());
assertEquals(1, roView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId2)).count());
assertEquals(1, roView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId3)).count());
assertEquals(1, roView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId4)).count());
//exclude commit 2 and make sure fileId1 shows up in view.
SyncableFileSystemView filteredView = getFileSystemView(metaClient.getActiveTimeline().findInstantsBefore("2"), false);
assertEquals(1, filteredView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId1)).count());
assertEquals(1, filteredView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId2)).count());
assertEquals(1, filteredView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId3)).count());
assertEquals(1, filteredView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId4)).count());
// ensure replacedFileGroupsBefore works with all instants
List<HoodieFileGroup> replacedOnInstant1 = fsView.getReplacedFileGroupsBeforeOrOn("1", partitionPath1).collect(Collectors.toList());
assertEquals(0, replacedOnInstant1.size());
List<HoodieFileGroup> allReplaced = fsView.getReplacedFileGroupsBeforeOrOn("2", partitionPath1).collect(Collectors.toList());
assertEquals(1, allReplaced.size());
assertEquals(fileId1, allReplaced.get(0).getFileGroupId().getFileId());
}
@Test
public void testReplaceFileIdIsExcludedInView() throws IOException {
String partitionPath1 = "2020/06/27";
String partitionPath2 = "2020/07/14";
new File(basePath + "/" + partitionPath1).mkdirs();
new File(basePath + "/" + partitionPath2).mkdirs();
// create 2 fileId in partition1 - fileId1 is replaced later on.
String fileId1 = UUID.randomUUID().toString();
String fileId2 = UUID.randomUUID().toString();
// create 2 fileId in partition2 - fileId3, fileId4 is replaced later on.
String fileId3 = UUID.randomUUID().toString();
String fileId4 = UUID.randomUUID().toString();
assertFalse(roView.getLatestBaseFiles(partitionPath1)
.anyMatch(dfile -> dfile.getFileId().equals(fileId1) || dfile.getFileId().equals(fileId2)),
"No commit, should not find any data file");
assertFalse(roView.getLatestBaseFiles(partitionPath2)
.anyMatch(dfile -> dfile.getFileId().equals(fileId3) || dfile.getFileId().equals(fileId4)),
"No commit, should not find any data file");
// Only one commit
String commitTime1 = "1";
String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1);
String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2);
String fileName3 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId3);
String fileName4 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId4);
new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile();
new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile();
new File(basePath + "/" + partitionPath2 + "/" + fileName3).createNewFile();
new File(basePath + "/" + partitionPath2 + "/" + fileName4).createNewFile();
Map<String, List<String>> partitionToReplaceFileIds = new HashMap<>();
List<String> replacedFileIdsP1 = new ArrayList<>();
replacedFileIdsP1.add(fileId1);
partitionToReplaceFileIds.put(partitionPath1, replacedFileIdsP1);
List<String> replacedFileIdsP2 = new ArrayList<>();
replacedFileIdsP2.add(fileId3);
replacedFileIdsP2.add(fileId4);
partitionToReplaceFileIds.put(partitionPath2, replacedFileIdsP2);
HoodieCommitMetadata commitMetadata =
CommitUtils.buildMetadata(Collections.emptyList(), partitionToReplaceFileIds, Option.empty(), WriteOperationType.INSERT_OVERWRITE, "", HoodieTimeline.REPLACE_COMMIT_ACTION);
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.REPLACE_COMMIT_ACTION, commitTime1);
saveAsComplete(commitTimeline, instant1, Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
refreshFsView();
assertEquals(0, roView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId1)).count());
assertEquals(fileName2, roView.getLatestBaseFiles(partitionPath1)
.filter(dfile -> dfile.getFileId().equals(fileId2)).findFirst().get().getFileName());
assertEquals(0, roView.getLatestBaseFiles(partitionPath2)
.filter(dfile -> dfile.getFileId().equals(fileId3)).count());
assertEquals(0, roView.getLatestBaseFiles(partitionPath2)
.filter(dfile -> dfile.getFileId().equals(fileId4)).count());
// ensure replacedFileGroupsBefore works with all instants
List<HoodieFileGroup> replacedOnInstant1 = fsView.getReplacedFileGroupsBeforeOrOn("0", partitionPath1).collect(Collectors.toList());
assertEquals(0, replacedOnInstant1.size());
List<HoodieFileGroup> allReplaced = fsView.getReplacedFileGroupsBeforeOrOn("2", partitionPath1).collect(Collectors.toList());
allReplaced.addAll(fsView.getReplacedFileGroupsBeforeOrOn("2", partitionPath2).collect(Collectors.toList()));
assertEquals(3, allReplaced.size());
Set<String> allReplacedFileIds = allReplaced.stream().map(fg -> fg.getFileGroupId().getFileId()).collect(Collectors.toSet());
Set<String> actualReplacedFileIds = Stream.of(fileId1, fileId3, fileId4).collect(Collectors.toSet());
assertEquals(actualReplacedFileIds, allReplacedFileIds);
}
@Override
protected HoodieTableType getTableType() {
return HoodieTableType.MERGE_ON_READ;

View File

@@ -32,6 +32,7 @@ import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -49,6 +50,7 @@ import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.junit.jupiter.api.BeforeEach;
@@ -62,6 +64,7 @@ import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -81,12 +84,13 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
private static final Logger LOG = LogManager.getLogger(TestIncrementalFSViewSync.class);
private static final int NUM_FILE_IDS_PER_PARTITION = 10;
private static String TEST_WRITE_TOKEN = "1-0-1";
private final List<String> partitions = Arrays.asList("2018/01/01", "2018/01/02", "2019/03/01");
private final List<String> fileIdsPerPartition =
IntStream.range(0, 10).mapToObj(x -> UUID.randomUUID().toString()).collect(Collectors.toList());
IntStream.range(0, NUM_FILE_IDS_PER_PARTITION).mapToObj(x -> UUID.randomUUID().toString()).collect(Collectors.toList());
@BeforeEach
public void init() throws IOException {
@@ -143,13 +147,13 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
{
put("11", Arrays.asList("12", "13", "15"));
}
}, instantsToFiles, Collections.singletonList("11"));
}, instantsToFiles, Collections.singletonList("11"), 0, 0);
// Add one more ingestion instant. This should be 2nd slice now
instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("22"), true, "19", 2));
// Restore last ingestion
testRestore(view, Collections.singletonList("23"), true, new HashMap<>(), Collections.singletonList("22"), "24", false);
testRestore(view, Collections.singletonList("23"), new HashMap<>(), Collections.singletonList(getHoodieCommitInstant("22", true)), "24", false);
// Run one more ingestion. THis is still 2nd slice
instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("24"), true, "19", 2));
@@ -188,14 +192,117 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
Map<String, List<String>> instantsToFiles = testMultipleWriteSteps(view, Arrays.asList("12", "13", "14"));
// restore instants in reverse order till we rollback all
testRestore(view, Arrays.asList("15", "16", "17"), false, instantsToFiles, Arrays.asList("14", "13", "12"), "17",
true);
testRestore(view, Arrays.asList("15", "16", "17"), instantsToFiles,
Arrays.asList(getHoodieCommitInstant("14", false), getHoodieCommitInstant("13", false), getHoodieCommitInstant("12", false)),
"17", true);
// Add 5 non-empty ingestions back-to-back
instantsToFiles = testMultipleWriteSteps(view, Arrays.asList("18", "19", "20"));
// Clean instants.
testCleans(view, Arrays.asList("21", "22"), instantsToFiles, Arrays.asList("18", "19"));
testCleans(view, Arrays.asList("21", "22"), instantsToFiles, Arrays.asList("18", "19"), 0, 0);
}
@Test
public void testReplaceCommits() throws IOException {
SyncableFileSystemView view = getFileSystemView(metaClient);
// Add an empty ingestion
String firstEmptyInstantTs = "11";
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
metaClient.getActiveTimeline().createNewInstant(
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs));
metaClient.getActiveTimeline().saveAsComplete(
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs),
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
view.sync();
assertTrue(view.getLastInstant().isPresent());
assertEquals("11", view.getLastInstant().get().getTimestamp());
assertEquals(State.COMPLETED, view.getLastInstant().get().getState());
assertEquals(HoodieTimeline.COMMIT_ACTION, view.getLastInstant().get().getAction());
partitions.forEach(p -> assertEquals(0, view.getLatestFileSlices(p).count()));
metaClient.reloadActiveTimeline();
SyncableFileSystemView newView = getFileSystemView(metaClient);
areViewsConsistent(view, newView, 0L);
// Add 1 non-empty ingestions to COW table
Map<String, List<String>> instantsToFiles = testMultipleWriteSteps(view, Arrays.asList("12"));
// ADD replace instants
testMultipleReplaceSteps(instantsToFiles, view, Arrays.asList("13", "14"), NUM_FILE_IDS_PER_PARTITION);
// restore instants in reverse order till we rollback all replace instants
testRestore(view, Arrays.asList("15", "16"), instantsToFiles,
Arrays.asList(getHoodieReplaceInstant("14"), getHoodieReplaceInstant("13")),
"17", true, 1, fileIdsPerPartition.size());
// clear files from inmemory view for replaced instants
instantsToFiles.remove("14");
instantsToFiles.remove("13");
// add few more replace instants
testMultipleReplaceSteps(instantsToFiles, view, Arrays.asList("18", "19", "20"), NUM_FILE_IDS_PER_PARTITION);
// Clean instants.
testCleans(view, Arrays.asList("21", "22"), instantsToFiles, Arrays.asList("18", "19"), NUM_FILE_IDS_PER_PARTITION, 1);
}
private void testMultipleReplaceSteps(Map<String, List<String>> instantsToFiles, SyncableFileSystemView view, List<String> instants,
int initialExpectedSlicesPerPartition) {
int expectedSlicesPerPartition = initialExpectedSlicesPerPartition;
for (int i = 0; i < instants.size(); i++) {
try {
generateReplaceInstant(instants.get(i), instantsToFiles);
view.sync();
metaClient.reloadActiveTimeline();
SyncableFileSystemView newView = getFileSystemView(metaClient);
// 1 fileId is replaced for every partition, so subtract partitions.size()
expectedSlicesPerPartition = expectedSlicesPerPartition + fileIdsPerPartition.size() - 1;
areViewsConsistent(view, newView, expectedSlicesPerPartition * partitions.size());
} catch (IOException e) {
throw new HoodieIOException("unable to test replace", e);
}
}
}
private Map<String, List<String>> generateReplaceInstant(String replaceInstant, Map<String, List<String>> instantsToFiles) throws IOException {
Map<String, List<String>> partitionToReplacedFileIds = pickFilesToReplace(instantsToFiles);
// generate new fileIds for replace
List<String> newFileIdsToUse = IntStream.range(0, NUM_FILE_IDS_PER_PARTITION).mapToObj(x -> UUID.randomUUID().toString()).collect(Collectors.toList());
List<String> replacedFiles = addReplaceInstant(metaClient, replaceInstant,
generateDataForInstant(replaceInstant, replaceInstant, false, newFileIdsToUse),
partitionToReplacedFileIds);
instantsToFiles.put(replaceInstant, replacedFiles);
return partitionToReplacedFileIds;
}
// pick one fileId from each partition to replace and remove it from 'instantsToFiles'
private Map<String, List<String>> pickFilesToReplace(Map<String, List<String>> instantsToFiles) {
if (instantsToFiles.isEmpty()) {
return Collections.emptyMap();
}
String maxInstant = instantsToFiles.keySet().stream().max(Comparator.naturalOrder()).get();
Map<String, List<String>> partitionToFileIdsList = instantsToFiles.get(maxInstant).stream().map(file -> {
int lastPartition = file.lastIndexOf("/");
return Pair.of(file.substring(0, lastPartition), file.substring(lastPartition + 1));
}).collect(Collectors.groupingBy(Pair::getKey, Collectors.mapping(Pair::getValue, Collectors.toList())));
return partitions.stream()
.map(p -> Pair.of(p, FSUtils.getFileId(partitionToFileIdsList.get(p).get(0))))
.collect(Collectors.groupingBy(Pair::getKey, Collectors.mapping(Pair::getValue, Collectors.toList())));
}
private HoodieInstant getHoodieReplaceInstant(String timestamp) {
return new HoodieInstant(false, HoodieTimeline.REPLACE_COMMIT_ACTION, timestamp);
}
private HoodieInstant getHoodieCommitInstant(String timestamp, boolean isDeltaCommit) {
String action = isDeltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION;
return new HoodieInstant(false, action, timestamp);
}
/**
@@ -263,7 +370,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
put("11", Arrays.asList("12", "13", "14"));
}
},
instantsToFiles, Collections.singletonList("11"));
instantsToFiles, Collections.singletonList("11"), 0, 0);
scheduleCompaction(view2, "20");
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("21", "22"), true, "20", 2));
// Compaction
@@ -280,8 +387,8 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
/*
* Case where multiple restores and ingestions happened
*/
testRestore(view2, Collections.singletonList("25"), true, new HashMap<>(), Collections.singletonList("24"), "29", true);
testRestore(view2, Collections.singletonList("26"), true, new HashMap<>(), Collections.singletonList("23"), "29", false);
testRestore(view2, Collections.singletonList("25"), new HashMap<>(), Collections.singletonList(getHoodieCommitInstant("24", true)), "29", true);
testRestore(view2, Collections.singletonList("26"), new HashMap<>(), Collections.singletonList(getHoodieCommitInstant("23", true)), "29", false);
instantsToFiles.putAll(testMultipleWriteSteps(view2, Collections.singletonList("27"), true, "20", 2));
scheduleCompaction(view2, "28");
instantsToFiles.putAll(testMultipleWriteSteps(view2, Collections.singletonList("29"), true, "28", 3));
@@ -304,10 +411,11 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
* Helper to run one or more rounds of cleaning, incrementally syncing the view and then validate.
*/
private void testCleans(SyncableFileSystemView view, List<String> newCleanerInstants,
Map<String, List<String>> instantsToFiles, List<String> cleanedInstants) {
Map<String, List<String>> instantsToFiles, List<String> cleanedInstants, int numberOfFilesAddedPerInstant,
int numberOfFilesReplacedPerInstant) {
Map<String, List<String>> deltaInstantMap = cleanedInstants.stream().map(e -> Pair.of(e, new ArrayList()))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
testCleans(view, newCleanerInstants, deltaInstantMap, instantsToFiles, cleanedInstants);
testCleans(view, newCleanerInstants, deltaInstantMap, instantsToFiles, cleanedInstants, numberOfFilesAddedPerInstant, numberOfFilesReplacedPerInstant);
}
/**
@@ -321,7 +429,8 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
*/
private void testCleans(SyncableFileSystemView view, List<String> newCleanerInstants,
Map<String, List<String>> deltaInstantMap, Map<String, List<String>> instantsToFiles,
List<String> cleanedInstants) {
List<String> cleanedInstants, int numFilesAddedPerInstant, int numFilesReplacedPerInstant) {
final int netFilesAddedPerInstant = numFilesAddedPerInstant - numFilesReplacedPerInstant;
assertEquals(newCleanerInstants.size(), cleanedInstants.size());
long exp = partitions.stream().mapToLong(p1 -> view.getAllFileSlices(p1).count()).findAny().getAsLong();
LOG.info("Initial File Slices :" + exp);
@@ -333,7 +442,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
performClean(instant, filesToDelete, newCleanerInstants.get(idx));
exp -= fileIdsPerPartition.size();
exp -= fileIdsPerPartition.size() - numFilesReplacedPerInstant;
final long expTotalFileSlicesPerPartition = exp;
view.sync();
assertTrue(view.getLastInstant().isPresent());
@@ -345,7 +454,8 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
LOG.info("\tFileSlices :" + view.getAllFileSlices(p).collect(Collectors.toList()));
});
partitions.forEach(p -> assertEquals(fileIdsPerPartition.size(), view.getLatestFileSlices(p).count()));
final int instantIdx = newCleanerInstants.size() - idx;
partitions.forEach(p -> assertEquals(fileIdsPerPartition.size() + instantIdx * netFilesAddedPerInstant, view.getLatestFileSlices(p).count()));
partitions.forEach(p -> assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count()));
metaClient.reloadActiveTimeline();
@@ -362,22 +472,30 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
*
* @param view Hoodie View
* @param newRestoreInstants Restore Instants
* @param isDeltaCommit is Delta Commit ?
* @param instantsToFiles List of files associated with each instant
* @param rolledBackInstants List of rolled-back instants
* @param emptyRestoreInstant Restore instant at which table becomes empty
*/
private void testRestore(SyncableFileSystemView view, List<String> newRestoreInstants, boolean isDeltaCommit,
Map<String, List<String>> instantsToFiles, List<String> rolledBackInstants, String emptyRestoreInstant,
boolean isRestore) {
private void testRestore(SyncableFileSystemView view, List<String> newRestoreInstants,
Map<String, List<String>> instantsToFiles, List<HoodieInstant> rolledBackInstants, String emptyRestoreInstant,
boolean isRestore) {
testRestore(view, newRestoreInstants, instantsToFiles, rolledBackInstants, emptyRestoreInstant, isRestore, 0, 0);
}
private void testRestore(SyncableFileSystemView view, List<String> newRestoreInstants,
Map<String, List<String>> instantsToFiles, List<HoodieInstant> rolledBackInstants, String emptyRestoreInstant,
boolean isRestore, int totalReplacedFileSlicesPerPartition, int totalFilesAddedPerPartitionPerInstant) {
assertEquals(newRestoreInstants.size(), rolledBackInstants.size());
long initialFileSlices = partitions.stream().mapToLong(p -> view.getAllFileSlices(p).count()).findAny().getAsLong();
final int numFileSlicesAddedPerInstant = (totalFilesAddedPerPartitionPerInstant - totalReplacedFileSlicesPerPartition);
final long expectedLatestFileSlices = fileIdsPerPartition.size() + (rolledBackInstants.size()) * numFileSlicesAddedPerInstant;
IntStream.range(0, newRestoreInstants.size()).forEach(idx -> {
String instant = rolledBackInstants.get(idx);
HoodieInstant instant = rolledBackInstants.get(idx);
try {
performRestore(instant, instantsToFiles.get(instant), newRestoreInstants.get(idx), isRestore);
boolean isDeltaCommit = HoodieTimeline.DELTA_COMMIT_ACTION.equalsIgnoreCase(instant.getAction());
performRestore(instant, instantsToFiles.get(instant.getTimestamp()), newRestoreInstants.get(idx), isRestore);
final long expTotalFileSlicesPerPartition =
isDeltaCommit ? initialFileSlices : initialFileSlices - ((idx + 1) * fileIdsPerPartition.size());
isDeltaCommit ? initialFileSlices : initialFileSlices - ((idx + 1) * (fileIdsPerPartition.size() - totalReplacedFileSlicesPerPartition));
view.sync();
assertTrue(view.getLastInstant().isPresent());
LOG.info("Last Instant is :" + view.getLastInstant().get());
@@ -391,7 +509,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
)) {
partitions.forEach(p -> assertEquals(0, view.getLatestFileSlices(p).count()));
} else {
partitions.forEach(p -> assertEquals(fileIdsPerPartition.size(), view.getLatestFileSlices(p).count()));
partitions.forEach(p -> assertEquals(expectedLatestFileSlices - (idx + 1) * numFileSlicesAddedPerInstant, view.getLatestFileSlices(p).count()));
}
partitions.forEach(p -> assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count()));
@@ -432,28 +550,23 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
* @param files List of files to be deleted as part of rollback
* @param rollbackInstant Restore Instant
*/
private void performRestore(String instant, List<String> files, String rollbackInstant,
private void performRestore(HoodieInstant instant, List<String> files, String rollbackInstant,
boolean isRestore) throws IOException {
Map<String, List<String>> partititonToFiles = deleteFiles(files);
List<HoodieRollbackStat> rollbackStats = partititonToFiles.entrySet().stream().map(e ->
new HoodieRollbackStat(e.getKey(), e.getValue(), new ArrayList<>(), new HashMap<>())
).collect(Collectors.toList());
List<String> rollbacks = new ArrayList<>();
List<HoodieInstant> rollbacks = new ArrayList<>();
rollbacks.add(instant);
HoodieRollbackMetadata rollbackMetadata =
TimelineMetadataUtils.convertRollbackMetadata(rollbackInstant, Option.empty(), rollbacks, rollbackStats);
if (isRestore) {
HoodieRestoreMetadata metadata = new HoodieRestoreMetadata();
List<HoodieRollbackMetadata> rollbackM = new ArrayList<>();
rollbackM.add(rollbackMetadata);
metadata.setHoodieRestoreMetadata(CollectionUtils.createImmutableMap(rollbackInstant, rollbackM));
List<String> rollbackInstants = new ArrayList<>();
rollbackInstants.add(rollbackInstant);
metadata.setInstantsToRollback(rollbackInstants);
metadata.setStartRestoreTime(rollbackInstant);
HoodieRestoreMetadata metadata = TimelineMetadataUtils.convertRestoreMetadata(rollbackInstant,
100, Collections.singletonList(instant), CollectionUtils.createImmutableMap(rollbackInstant, rollbackM));
HoodieInstant restoreInstant = new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, rollbackInstant);
metaClient.getActiveTimeline().createNewInstant(restoreInstant);
@@ -465,6 +578,8 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, rollbackInstant),
TimelineMetadataUtils.serializeRollbackMetadata(rollbackMetadata));
}
boolean deleted = metaClient.getFs().delete(new Path(metaClient.getMetaPath(), instant.getFileName()), false);
assertTrue(deleted);
}
/**
@@ -651,10 +766,7 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
private void areViewsConsistent(SyncableFileSystemView view1, SyncableFileSystemView view2,
long expectedTotalFileSlices) {
// Timeline check
HoodieTimeline timeline1 = view1.getTimeline();
HoodieTimeline timeline2 = view2.getTimeline();
assertEquals(view1.getLastInstant(), view2.getLastInstant());
CollectionUtils.elementsEqual(timeline1.getInstants().iterator(), timeline2.getInstants().iterator());
// View Checks
Map<HoodieFileGroupId, HoodieFileGroup> fileGroupsMap1 = partitions.stream().flatMap(view1::getAllFileGroups)
@@ -702,14 +814,17 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
assertEquals(ops1, ops2);
}
private List<String> addInstant(HoodieTableMetaClient metaClient, String instant, boolean deltaCommit,
String baseInstant) throws IOException {
List<Pair<String, HoodieWriteStat>> writeStats = partitions.stream().flatMap(p -> fileIdsPerPartition.stream().map(f -> {
private List<Pair<String, HoodieWriteStat>> generateDataForInstant(String baseInstant, String instant, boolean deltaCommit) {
return generateDataForInstant(baseInstant, instant, deltaCommit, fileIdsPerPartition);
}
private List<Pair<String, HoodieWriteStat>> generateDataForInstant(String baseInstant, String instant, boolean deltaCommit, List<String> fileIds) {
return partitions.stream().flatMap(p -> fileIds.stream().map(f -> {
try {
File file = new File(basePath + "/" + p + "/"
+ (deltaCommit
? FSUtils.makeLogFileName(f, ".log", baseInstant, Integer.parseInt(instant), TEST_WRITE_TOKEN)
: FSUtils.makeDataFileName(instant, TEST_WRITE_TOKEN, f)));
? FSUtils.makeLogFileName(f, ".log", baseInstant, Integer.parseInt(instant), TEST_WRITE_TOKEN)
: FSUtils.makeDataFileName(instant, TEST_WRITE_TOKEN, f)));
file.createNewFile();
HoodieWriteStat w = new HoodieWriteStat();
w.setFileId(f);
@@ -719,7 +834,11 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
throw new HoodieException(e);
}
})).collect(Collectors.toList());
}
private List<String> addInstant(HoodieTableMetaClient metaClient, String instant, boolean deltaCommit,
String baseInstant) throws IOException {
List<Pair<String, HoodieWriteStat>> writeStats = generateDataForInstant(baseInstant, instant, deltaCommit);
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
writeStats.forEach(e -> metadata.addWriteStat(e.getKey(), e.getValue()));
HoodieInstant inflightInstant = new HoodieInstant(true,
@@ -735,6 +854,19 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
return writeStats.stream().map(e -> e.getValue().getPath()).collect(Collectors.toList());
}
private List<String> addReplaceInstant(HoodieTableMetaClient metaClient, String instant,
List<Pair<String, HoodieWriteStat>> writeStats,
Map<String, List<String>> partitionToReplaceFileIds) throws IOException {
HoodieReplaceCommitMetadata replaceCommitMetadata = new HoodieReplaceCommitMetadata();
writeStats.forEach(e -> replaceCommitMetadata.addWriteStat(e.getKey(), e.getValue()));
replaceCommitMetadata.setPartitionToReplaceFileIds(partitionToReplaceFileIds);
HoodieInstant inflightInstant = new HoodieInstant(true, HoodieTimeline.REPLACE_COMMIT_ACTION, instant);
metaClient.getActiveTimeline().createNewInstant(inflightInstant);
metaClient.getActiveTimeline().saveAsComplete(inflightInstant,
Option.of(replaceCommitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
return writeStats.stream().map(e -> e.getValue().getPath()).collect(Collectors.toList());
}
@Override
protected HoodieTableType getTableType() {
return HoodieTableType.MERGE_ON_READ;

View File

@@ -19,9 +19,11 @@
package org.apache.hudi.common.testutils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.IOType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -29,10 +31,9 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.table.view.TableFileSystemView;
import org.apache.hudi.exception.HoodieException;
import org.apache.hadoop.fs.FileSystem;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
@@ -76,6 +77,15 @@ public class FileCreateUtils {
}
}
private static void createMetaFile(String basePath, String instantTime, String suffix, byte[] content) throws IOException {
Path parentPath = Paths.get(basePath, HoodieTableMetaClient.METAFOLDER_NAME);
Files.createDirectories(parentPath);
Path metaFilePath = parentPath.resolve(instantTime + suffix);
if (Files.notExists(metaFilePath)) {
Files.write(metaFilePath, content);
}
}
public static void createCommit(String basePath, String instantTime) throws IOException {
createMetaFile(basePath, instantTime, HoodieTimeline.COMMIT_EXTENSION);
}
@@ -100,6 +110,18 @@ public class FileCreateUtils {
createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_DELTA_COMMIT_EXTENSION);
}
public static void createReplaceCommit(String basePath, String instantTime, HoodieReplaceCommitMetadata metadata) throws IOException {
createMetaFile(basePath, instantTime, HoodieTimeline.REPLACE_COMMIT_EXTENSION, metadata.toJsonString().getBytes(StandardCharsets.UTF_8));
}
public static void createRequestedReplaceCommit(String basePath, String instantTime) throws IOException {
createMetaFile(basePath, instantTime, HoodieTimeline.REQUESTED_REPLACE_COMMIT_EXTENSION);
}
public static void createInflightReplaceCommit(String basePath, String instantTime) throws IOException {
createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_REPLACE_COMMIT_EXTENSION);
}
private static void createAuxiliaryMetaFile(String basePath, String instantTime, String suffix) throws IOException {
Path parentPath = Paths.get(basePath, HoodieTableMetaClient.AUXILIARYFOLDER_NAME);
Files.createDirectories(parentPath);

View File

@@ -330,30 +330,42 @@ public class HoodieTestDataGenerator {
}
public static void createCommitFile(String basePath, String instantTime, Configuration configuration) {
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
createCommitFile(basePath, instantTime, configuration, commitMetadata);
}
public static void createCommitFile(String basePath, String instantTime, Configuration configuration, HoodieCommitMetadata commitMetadata) {
Arrays.asList(HoodieTimeline.makeCommitFileName(instantTime), HoodieTimeline.makeInflightCommitFileName(instantTime),
HoodieTimeline.makeRequestedCommitFileName(instantTime))
.forEach(f -> {
Path commitFile = new Path(
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + f);
FSDataOutputStream os = null;
try {
FileSystem fs = FSUtils.getFs(basePath, configuration);
os = fs.create(commitFile, true);
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
// Write empty commit metadata
os.writeBytes(new String(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
} finally {
if (null != os) {
try {
os.close();
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
}
}
});
.forEach(f -> createMetadataFile(f, basePath, configuration, commitMetadata));
}
private static void createMetadataFile(String f, String basePath, Configuration configuration, HoodieCommitMetadata commitMetadata) {
Path commitFile = new Path(
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + f);
FSDataOutputStream os = null;
try {
FileSystem fs = FSUtils.getFs(basePath, configuration);
os = fs.create(commitFile, true);
// Write empty commit metadata
os.writeBytes(new String(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
} finally {
if (null != os) {
try {
os.close();
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
}
}
}
public static void createReplaceFile(String basePath, String instantTime, Configuration configuration, HoodieCommitMetadata commitMetadata) {
Arrays.asList(HoodieTimeline.makeReplaceFileName(instantTime), HoodieTimeline.makeInflightReplaceFileName(instantTime),
HoodieTimeline.makeRequestedReplaceFileName(instantTime))
.forEach(f -> createMetadataFile(f, basePath, configuration, commitMetadata));
}
public static void createEmptyCleanRequestedFile(String basePath, String instantTime, Configuration configuration)
@@ -382,9 +394,9 @@ public class HoodieTestDataGenerator {
new Path(basePath + "/" + HoodieTableMetaClient.AUXILIARYFOLDER_NAME + "/" + instant.getFileName());
FileSystem fs = FSUtils.getFs(basePath, configuration);
try (FSDataOutputStream os = fs.create(commitFile, true)) {
HoodieCompactionPlan workload = new HoodieCompactionPlan();
HoodieCompactionPlan workload = HoodieCompactionPlan.newBuilder().setVersion(1).build();
// Write empty commit metadata
os.writeBytes(new String(TimelineMetadataUtils.serializeCompactionPlan(workload).get(), StandardCharsets.UTF_8));
os.write(TimelineMetadataUtils.serializeCompactionPlan(workload).get());
}
}

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.common.testutils;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.IOType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -49,10 +50,13 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.createDeltaCommit
import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightCommit;
import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightCompaction;
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.createMarkerFile;
import static org.apache.hudi.common.testutils.FileCreateUtils.createReplaceCommit;
import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedCommit;
import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedCompaction;
import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedDeltaCommit;
import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedReplaceCommit;
import static org.apache.hudi.common.testutils.FileCreateUtils.logFileName;
public class HoodieTestTable {
@@ -145,6 +149,15 @@ public class HoodieTestTable {
return this;
}
public HoodieTestTable addReplaceCommit(String instantTime, HoodieReplaceCommitMetadata metadata) throws Exception {
createRequestedReplaceCommit(basePath, instantTime);
createInflightReplaceCommit(basePath, instantTime);
createReplaceCommit(basePath, instantTime, metadata);
currentInstantTime = instantTime;
metaClient = HoodieTableMetaClient.reload(metaClient);
return this;
}
public HoodieTestTable addRequestedCompaction(String instantTime) throws IOException {
createRequestedCompaction(basePath, instantTime);
currentInstantTime = instantTime;

View File

@@ -18,6 +18,21 @@
package org.apache.hudi.common.testutils;
import com.esotericsoftware.kryo.Kryo;
import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;
import com.esotericsoftware.kryo.serializers.JavaSerializer;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.util.StringUtils;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieActionInstant;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
@@ -52,22 +67,6 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieIOException;
import com.esotericsoftware.kryo.Kryo;
import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;
import com.esotericsoftware.kryo.serializers.JavaSerializer;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.util.StringUtils;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.File;

View File

@@ -0,0 +1,96 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.util;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.junit.jupiter.api.Test;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_SCHEMA;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class TestCommitUtils {
@Test
public void testCommitMetadataCreation() {
List<HoodieWriteStat> writeStats = new ArrayList<>();
writeStats.add(createWriteStat("p1", "f1"));
writeStats.add(createWriteStat("p2", "f2"));
Map<String, List<String>> partitionToReplaceFileIds = new HashMap<>();
List<String> replacedFileIds = new ArrayList<>();
replacedFileIds.add("f0");
partitionToReplaceFileIds.put("p1", replacedFileIds);
HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(writeStats, partitionToReplaceFileIds,
Option.empty(),
WriteOperationType.INSERT,
TRIP_SCHEMA,
HoodieTimeline.DELTA_COMMIT_ACTION);
assertFalse(commitMetadata instanceof HoodieReplaceCommitMetadata);
assertEquals(2, commitMetadata.getPartitionToWriteStats().size());
assertEquals("f1", commitMetadata.getPartitionToWriteStats().get("p1").get(0).getFileId());
assertEquals("f2", commitMetadata.getPartitionToWriteStats().get("p2").get(0).getFileId());
assertEquals(WriteOperationType.INSERT, commitMetadata.getOperationType());
assertEquals(TRIP_SCHEMA, commitMetadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY));
}
@Test
public void testReplaceMetadataCreation() {
List<HoodieWriteStat> writeStats = new ArrayList<>();
writeStats.add(createWriteStat("p1", "f1"));
writeStats.add(createWriteStat("p2", "f2"));
Map<String, List<String>> partitionToReplaceFileIds = new HashMap<>();
List<String> replacedFileIds = new ArrayList<>();
replacedFileIds.add("f0");
partitionToReplaceFileIds.put("p1", replacedFileIds);
HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(writeStats, partitionToReplaceFileIds,
Option.empty(),
WriteOperationType.INSERT,
TRIP_SCHEMA,
HoodieTimeline.REPLACE_COMMIT_ACTION);
assertTrue(commitMetadata instanceof HoodieReplaceCommitMetadata);
HoodieReplaceCommitMetadata replaceCommitMetadata = (HoodieReplaceCommitMetadata) commitMetadata;
assertEquals(1, replaceCommitMetadata.getPartitionToReplaceFileIds().size());
assertEquals("f0", replaceCommitMetadata.getPartitionToReplaceFileIds().get("p1").get(0));
assertEquals(2, commitMetadata.getPartitionToWriteStats().size());
assertEquals("f1", commitMetadata.getPartitionToWriteStats().get("p1").get(0).getFileId());
assertEquals("f2", commitMetadata.getPartitionToWriteStats().get("p2").get(0).getFileId());
assertEquals(WriteOperationType.INSERT, commitMetadata.getOperationType());
assertEquals(TRIP_SCHEMA, commitMetadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY));
}
private HoodieWriteStat createWriteStat(String partition, String fileId) {
HoodieWriteStat writeStat1 = new HoodieWriteStat();
writeStat1.setPartitionPath(partition);
writeStat1.setFileId(fileId);
return writeStat1;
}
}