1
0

Ensure Cleaner and Archiver do not delete file-slices and workload marked for compaction

This commit is contained in:
Balaji Varadarajan
2018-05-31 14:16:19 -07:00
committed by vinoth chandar
parent 0a0451a765
commit 9b78523d62
10 changed files with 666 additions and 76 deletions

View File

@@ -19,12 +19,16 @@ package com.uber.hoodie;
import static com.uber.hoodie.common.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH;
import static com.uber.hoodie.common.HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH;
import static com.uber.hoodie.common.HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH;
import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION;
import static com.uber.hoodie.common.table.HoodieTimeline.GREATER;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import com.google.common.collect.Iterables;
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
import com.uber.hoodie.common.HoodieCleanStat;
import com.uber.hoodie.common.model.FileSlice;
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieDataFile;
@@ -38,12 +42,16 @@ import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.CompactionUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
@@ -53,7 +61,10 @@ import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.TreeSet;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -61,6 +72,7 @@ import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.scheduler.SparkListener;
import org.apache.spark.scheduler.SparkListenerTaskEnd;
import org.apache.spark.util.AccumulatorV2;
import org.junit.Assert;
import org.junit.Test;
import scala.Option;
import scala.collection.Iterator;
@@ -82,7 +94,7 @@ public class TestCleaner extends TestHoodieClientBase {
* @param insertFn Insertion API for testing
* @throws Exception in case of error
*/
private void insertFirstBigBatchForClientCleanerTest(
private String insertFirstBigBatchForClientCleanerTest(
HoodieWriteConfig cfg,
HoodieWriteClient client,
Function2<List<HoodieRecord>, String, Integer> recordGenFunction,
@@ -118,6 +130,7 @@ public class TestCleaner extends TestHoodieClientBase {
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), jsc, table).collect();
checkTaggedRecords(taggedRecords, newCommitTime);
return newCommitTime;
}
/**
@@ -185,21 +198,51 @@ public class TestCleaner extends TestHoodieClientBase {
insertFirstBigBatchForClientCleanerTest(cfg, client, recordInsertGenWrappedFunction, insertFn);
Map<String, String> selectedFileIdForCompaction = new HashMap<>();
Map<String, FileSlice> compactionFileIdToLatestFileSlice = new HashMap<>();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc);
for (String partitionPath : dataGen.getPartitionPaths()) {
TableFileSystemView fsView = table.getFileSystemView();
Optional<Boolean> added = fsView.getAllFileGroups(partitionPath).findFirst()
.map(fg -> {
selectedFileIdForCompaction.put(fg.getId(), partitionPath);
fg.getLatestFileSlice().map(fs -> compactionFileIdToLatestFileSlice.put(fg.getId(), fs));
return true;
});
if (added.isPresent()) {
// Select only one file-group for compaction
break;
}
}
// Create workload with selected file-slices
List<Pair<String, FileSlice>> partitionFileSlicePairs = compactionFileIdToLatestFileSlice.entrySet().stream()
.map(e -> Pair.of(selectedFileIdForCompaction.get(e.getKey()), e.getValue())).collect(Collectors.toList());
HoodieCompactionPlan compactionPlan =
CompactionUtils.buildFromFileSlices(partitionFileSlicePairs, Optional.empty(), Optional.empty());
List<String> instantTimes = HoodieTestUtils.monotonicIncreasingCommitTimestamps(9, 1);
String compactionTime = instantTimes.get(0);
table.getActiveTimeline().saveToCompactionRequested(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionTime),
AvroUtils.serializeCompactionPlan(compactionPlan));
instantTimes = instantTimes.subList(1, instantTimes.size());
// Keep doing some writes and clean inline. Make sure we have expected number of files
// remaining.
HoodieTestUtils.monotonicIncreasingCommitTimestamps(8, 1).stream().forEach(newCommitTime -> {
for (String newInstantTime : instantTimes) {
try {
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = recordUpsertGenWrappedFunction.apply(newCommitTime, 100);
client.startCommitWithTime(newInstantTime);
List<HoodieRecord> records = recordUpsertGenWrappedFunction.apply(newInstantTime, 100);
List<WriteStatus> statuses =
upsertFn.apply(client, jsc.parallelize(records, 1), newCommitTime).collect();
upsertFn.apply(client, jsc.parallelize(records, 1), newInstantTime).collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc);
HoodieTimeline timeline = metadata.getCommitsTimeline();
metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc);
HoodieTimeline timeline = table.getMetaClient().getCommitsTimeline();
TableFileSystemView fsView = table.getFileSystemView();
// Need to ensure the following
@@ -221,25 +264,39 @@ public class TestCleaner extends TestHoodieClientBase {
List<HoodieFileGroup> fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList());
for (HoodieFileGroup fileGroup : fileGroups) {
// No file has no more than max versions
String fileId = fileGroup.getId();
List<HoodieDataFile> dataFiles = fileGroup.getAllDataFiles().collect(Collectors.toList());
if (selectedFileIdForCompaction.containsKey(fileGroup.getId())) {
// Ensure latest file-slice selected for compaction is retained
String oldestCommitRetained =
fileGroup.getAllDataFiles().map(HoodieDataFile::getCommitTime).sorted().findFirst().get();
assertTrue("fileId " + fileId + " has more than " + maxVersions + " versions",
dataFiles.size() <= maxVersions);
Optional<HoodieDataFile> dataFileForCompactionPresent =
fileGroup.getAllDataFiles().filter(df -> {
return compactionFileIdToLatestFileSlice.get(fileGroup.getId())
.getBaseInstantTime().equals(df.getCommitTime());
}).findAny();
Assert.assertTrue("Data File selected for compaction is retained",
dataFileForCompactionPresent.isPresent());
} else {
// file has no more than max versions
String fileId = fileGroup.getId();
List<HoodieDataFile> dataFiles = fileGroup.getAllDataFiles().collect(Collectors.toList());
// Each file, has the latest N versions (i.e cleaning gets rid of older versions)
List<String> commitedVersions = new ArrayList<>(fileIdToVersions.get(fileId));
for (int i = 0; i < dataFiles.size(); i++) {
assertEquals("File " + fileId + " does not have latest versions on commits" + commitedVersions,
Iterables.get(dataFiles, i).getCommitTime(), commitedVersions.get(commitedVersions.size() - 1 - i));
assertTrue("fileId " + fileId + " has more than " + maxVersions + " versions",
dataFiles.size() <= maxVersions);
// Each file, has the latest N versions (i.e cleaning gets rid of older versions)
List<String> commitedVersions = new ArrayList<>(fileIdToVersions.get(fileId));
for (int i = 0; i < dataFiles.size(); i++) {
assertEquals("File " + fileId + " does not have latest versions on commits" + commitedVersions,
Iterables.get(dataFiles, i).getCommitTime(), commitedVersions.get(commitedVersions.size() - 1 - i));
}
}
}
}
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
});
}
}
/**
@@ -679,6 +736,168 @@ public class TestCleaner extends TestHoodieClientBase {
stageOneShuffleReadTaskRecordsCountMap.values().stream().filter(a -> a > 10 && a < 100).count() == 3);
}
/**
* Test Keep Latest Commits when there are pending compactions
*/
@Test
public void testKeepLatestCommitsWithPendingCompactions() throws IOException {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy(
HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()).build();
// Deletions:
// . FileId Parquet Logs Total Retained Commits
// FileId7 5 10 15 009, 011
// FileId6 5 10 15 009
// FileId5 3 6 9 005
// FileId4 2 4 6 003
// FileId3 1 2 3 001
// FileId2 0 0 0 000
// FileId1 0 0 0 000
testPendingCompactions(config, 48, 18);
}
/**
* Test Keep Latest Versions when there are pending compactions
*/
@Test
public void testKeepLatestVersionsWithPendingCompactions() throws IOException {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withCleanerPolicy(
HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(2).build()).build();
// Deletions:
// . FileId Parquet Logs Total Retained Commits
// FileId7 5 10 15 009, 011
// FileId6 4 8 12 007, 009
// FileId5 2 4 6 003 005
// FileId4 1 2 3 001, 003
// FileId3 0 0 0 000, 001
// FileId2 0 0 0 000
// FileId1 0 0 0 000
testPendingCompactions(config, 36, 9);
}
/**
* Common test method for validating pending compactions
*
* @param config Hoodie Write Config
* @param expNumFilesDeleted Number of files deleted
*/
public void testPendingCompactions(HoodieWriteConfig config, int expNumFilesDeleted,
int expNumFilesUnderCompactionDeleted) throws IOException {
HoodieTableMetaClient metaClient = HoodieTestUtils.initTableType(jsc.hadoopConfiguration(), basePath,
HoodieTableType.MERGE_ON_READ);
String[] instants = new String[]{"000", "001", "003", "005", "007", "009", "011", "013"};
String[] compactionInstants = new String[]{"002", "004", "006", "008", "010"};
Map<String, String> expFileIdToPendingCompaction = new HashMap<>();
Map<String, String> fileIdToLatestInstantBeforeCompaction = new HashMap<>();
Map<String, List<FileSlice>> compactionInstantsToFileSlices = new HashMap<>();
for (String instant : instants) {
HoodieTestUtils.createCommitFiles(basePath, instant);
}
// Generate 7 file-groups. First one has only one slice and no pending compaction. File Slices (2 - 5) has
// multiple versions with pending compaction. File Slices (6 - 7) have multiple file-slices but not under
// compactions
// FileIds 2-5 will be under compaction
int maxNumFileIds = 7;
String[] fileIds = new String[]
{"fileId1", "fileId2", "fileId3", "fileId4", "fileId5", "fileId6", "fileId7"};
int maxNumFileIdsForCompaction = 4;
for (int i = 0; i < maxNumFileIds; i++) {
final String fileId = HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, instants[0],
fileIds[i]);
HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, instants[0],
fileId, Optional.empty());
HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, instants[0],
fileId, Optional.of(2));
fileIdToLatestInstantBeforeCompaction.put(fileId, instants[0]);
for (int j = 1; j <= i; j++) {
if (j == i && j <= maxNumFileIdsForCompaction) {
expFileIdToPendingCompaction.put(fileId, compactionInstants[j]);
HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config,
jsc);
FileSlice slice = table.getRTFileSystemView().getLatestFileSlices(DEFAULT_FIRST_PARTITION_PATH)
.filter(fs -> fs.getFileId().equals(fileId)).findFirst().get();
List<FileSlice> slices = new ArrayList<>();
if (compactionInstantsToFileSlices.containsKey(compactionInstants[j])) {
slices = compactionInstantsToFileSlices.get(compactionInstants[j]);
}
slices.add(slice);
compactionInstantsToFileSlices.put(compactionInstants[j], slices);
// Add log-files to simulate delta-commits after pending compaction
HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, compactionInstants[j],
fileId, Optional.empty());
HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, compactionInstants[j],
fileId, Optional.of(2));
} else {
HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, instants[j], fileId);
HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, instants[j], fileId,
Optional.empty());
HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, instants[j], fileId,
Optional.of(2));
fileIdToLatestInstantBeforeCompaction.put(fileId, instants[j]);
}
}
}
// Setup pending compaction plans
for (String instant : compactionInstants) {
List<FileSlice> fileSliceList = compactionInstantsToFileSlices.get(instant);
if (null != fileSliceList) {
HoodieTestUtils.createCompactionRequest(metaClient, instant,
fileSliceList.stream().map(fs -> Pair.of(DEFAULT_FIRST_PARTITION_PATH, fs)).collect(Collectors.toList()));
}
}
// Clean now
HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config,
jsc);
List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc);
// Test for safety
final HoodieTable hoodieTable = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config,
jsc);
expFileIdToPendingCompaction.entrySet().stream().forEach(entry -> {
String fileId = entry.getKey();
String baseInstantForCompaction = fileIdToLatestInstantBeforeCompaction.get(fileId);
Optional<FileSlice> fileSliceForCompaction =
hoodieTable.getRTFileSystemView().getLatestFileSlicesBeforeOrOn(DEFAULT_FIRST_PARTITION_PATH,
baseInstantForCompaction).filter(fs -> fs.getFileId().equals(fileId)).findFirst();
Assert.assertTrue("Base Instant for Compaction must be preserved", fileSliceForCompaction.isPresent());
Assert.assertTrue("FileSlice has data-file", fileSliceForCompaction.get().getDataFile().isPresent());
Assert.assertEquals("FileSlice has log-files", 2,
fileSliceForCompaction.get().getLogFiles().count());
});
// Test for progress (Did we clean some files ?)
long numFilesUnderCompactionDeleted =
hoodieCleanStats.stream().flatMap(cleanStat -> {
return convertPathToFileIdWithCommitTime(metaClient, cleanStat.getDeletePathPatterns()).map(
fileIdWithCommitTime -> {
if (expFileIdToPendingCompaction.containsKey(fileIdWithCommitTime.getKey())) {
Assert.assertTrue("Deleted instant time must be less than pending compaction",
HoodieTimeline.compareTimestamps(
fileIdToLatestInstantBeforeCompaction.get(fileIdWithCommitTime.getKey()),
fileIdWithCommitTime.getValue(), GREATER));
return true;
}
return false;
});
}).filter(x -> x).count();
long numDeleted = hoodieCleanStats.stream()
.flatMap(cleanStat -> cleanStat.getDeletePathPatterns().stream()).count();
// Tighter check for regression
Assert.assertEquals("Correct number of files deleted", expNumFilesDeleted, numDeleted);
Assert.assertEquals("Correct number of files under compaction deleted",
expNumFilesUnderCompactionDeleted, numFilesUnderCompactionDeleted);
}
/**
* Utility method to create temporary data files
*
@@ -703,4 +922,23 @@ public class TestCleaner extends TestHoodieClientBase {
private int getTotalTempFiles() throws IOException {
return fs.listStatus(new Path(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME)).length;
}
private Stream<Pair<String, String>> convertPathToFileIdWithCommitTime(
final HoodieTableMetaClient metaClient, List<String> paths) {
Predicate<String> roFilePredicate = path ->
path.contains(metaClient.getTableConfig().getROFileFormat().getFileExtension());
Predicate<String> rtFilePredicate = path ->
path.contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension());
Stream<Pair<String, String>> stream1 = paths.stream().filter(roFilePredicate)
.map(fullPath -> {
String fileName = Paths.get(fullPath).getFileName().toString();
return Pair.of(FSUtils.getFileId(fileName), FSUtils.getCommitTime(fileName));
});
Stream<Pair<String, String>> stream2 = paths.stream().filter(rtFilePredicate)
.map(path -> {
return Pair.of(FSUtils.getFileIdFromLogPath(new Path(path)),
FSUtils.getBaseCommitTimeFromLogPath(new Path(path)));
});
return Stream.concat(stream1, stream2);
}
}

View File

@@ -303,7 +303,7 @@ public class TestClientRollback extends TestHoodieClientBase {
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
// Turn auto rollback on
new HoodieWriteClient(jsc, config, true);
new HoodieWriteClient(jsc, config, true).startCommit();
assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));

View File

@@ -138,6 +138,15 @@ public class HoodieTestDataGenerator {
}
}
public static void createCompactionRequestedFile(String basePath, String commitTime, Configuration configuration)
throws IOException {
Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
+ HoodieTimeline.makeRequestedCompactionFileName(commitTime));
FileSystem fs = FSUtils.getFs(basePath, configuration);
FSDataOutputStream os = fs.create(commitFile, true);
os.close();
}
public static void createCompactionAuxiliaryMetadata(String basePath, HoodieInstant instant,
Configuration configuration) throws IOException {
Path commitFile = new Path(

View File

@@ -16,7 +16,9 @@
package com.uber.hoodie.io;
import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import com.google.common.collect.Sets;
@@ -30,7 +32,9 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import java.io.File;
@@ -133,11 +137,47 @@ public class TestHoodieCommitArchiveLog {
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 4).build())
.forTable("test-trip-table").build();
HoodieTestUtils.init(hadoopConf, basePath);
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "100"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "100"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "101"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "101"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "102"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "102"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "103"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "103"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "104"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "104"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "104", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "105"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "105"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath);
@@ -172,6 +212,37 @@ public class TestHoodieCommitArchiveLog {
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList()));
// Check compaction instants
List<HoodieInstant> instants =
HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
new Path(metaClient.getMetaAuxiliaryPath()),
HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE);
assertEquals("Should delete all compaction instants < 104", 4, instants.size());
assertFalse("Requested Compaction must be absent for 100", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "100")));
assertFalse("Inflight Compaction must be absent for 100", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "100")));
assertFalse("Requested Compaction must be absent for 101", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "101")));
assertFalse("Inflight Compaction must be absent for 101", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "101")));
assertFalse("Requested Compaction must be absent for 102", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "102")));
assertFalse("Inflight Compaction must be absent for 102", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "102")));
assertFalse("Requested Compaction must be absent for 103", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "103")));
assertFalse("Inflight Compaction must be absent for 103", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "103")));
assertTrue("Requested Compaction must be present for 104", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "104")));
assertTrue("Inflight Compaction must be present for 104", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "104")));
assertTrue("Requested Compaction must be present for 105", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "105")));
assertTrue("Inflight Compaction must be present for 105", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "105")));
//read the file
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(dfs,
new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")),
@@ -210,9 +281,33 @@ public class TestHoodieCommitArchiveLog {
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "100"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "100"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "101"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "101"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "102"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "102"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "103"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "103"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf());
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
@@ -221,6 +316,28 @@ public class TestHoodieCommitArchiveLog {
assertTrue(result);
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4, timeline.countInstants());
List<HoodieInstant> instants =
HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
new Path(metaClient.getMetaAuxiliaryPath()),
HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE);
assertEquals("Should not delete any aux compaction files when maxCommitsToKeep is 5", 8, instants.size());
assertTrue("Requested Compaction must be present for 100", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "100")));
assertTrue("Inflight Compaction must be present for 100", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "100")));
assertTrue("Requested Compaction must be present for 101", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "101")));
assertTrue("Inflight Compaction must be present for 101", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "101")));
assertTrue("Requested Compaction must be present for 102", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "102")));
assertTrue("Inflight Compaction must be present for 102", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "102")));
assertTrue("Requested Compaction must be present for 103", instants.contains(
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "103")));
assertTrue("Inflight Compaction must be present for 103", instants.contains(
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, "103")));
}
@Test
@@ -281,6 +398,53 @@ public class TestHoodieCommitArchiveLog {
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103")));
}
@Test
public void testArchiveCommitCompactionNoHole() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf());
HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "101", dfs.getConf());
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "101"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf());
HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "104", dfs.getConf());
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, "104"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "106", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "107", dfs.getConf());
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline();
assertEquals("Loaded 6 commits and the count should match", 8, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired(jsc);
assertTrue(result);
timeline = metaClient.getActiveTimeline().reload().getCommitsAndCompactionTimeline();
assertFalse("Instants before oldest pending compaction can be removed",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "100")));
assertEquals(
"Since we have a pending compaction at 101, we should never archive any commit "
+ "after 101 (we only " + "archive 100)", 7, timeline.countInstants());
assertTrue("Requested Compaction must still be present",
timeline.containsInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "102")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "105")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "106")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "107")));
}
private void verifyInflightInstants(HoodieTableMetaClient metaClient, int expectedTotalInstants) {
HoodieTimeline timeline = metaClient.getActiveTimeline().reload()
.getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION)).filterInflights();