Implement Savepoints and required metadata timeline (#86)
- Introduce avro to save clean metadata with details about the last commit that was retained - Save rollback metadata in the meta timeline - Create savepoint metadata and add API to createSavepoint, deleteSavepoint and rollbackToSavepoint - Savepointed commit should not be rolledback or cleaned or archived - introduce cli commands to show, create and rollback to savepoints - Write unit tests to test savepoints and rollbackToSavepoints
This commit is contained in:
@@ -18,8 +18,10 @@ package com.uber.hoodie;
|
||||
|
||||
import com.google.common.collect.Iterables;
|
||||
|
||||
import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
@@ -29,6 +31,7 @@ import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
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.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.ParquetUtils;
|
||||
@@ -38,11 +41,9 @@ import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieStorageConfig;
|
||||
import com.uber.hoodie.exception.HoodieRollbackException;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import com.uber.hoodie.io.HoodieCleaner;
|
||||
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.SparkConf;
|
||||
@@ -58,6 +59,8 @@ import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedHashMap;
|
||||
@@ -71,6 +74,7 @@ import java.util.stream.Stream;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
public class TestHoodieClient implements Serializable {
|
||||
private transient JavaSparkContext jsc = null;
|
||||
@@ -335,12 +339,206 @@ public class TestHoodieClient implements Serializable {
|
||||
readClient.readSince("000").count());
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testCreateSavepoint() throws Exception {
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder()
|
||||
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1)
|
||||
.build()).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
|
||||
/**
|
||||
* Write 1 (only inserts)
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
|
||||
List<WriteStatus> statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
/**
|
||||
* Write 2 (updates)
|
||||
*/
|
||||
newCommitTime = "002";
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
client.savepoint(new HoodieSavepointMetadata("hoodie-unit-test",
|
||||
HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date()), "test"));
|
||||
try {
|
||||
client.rollback(newCommitTime);
|
||||
fail("Rollback of a savepoint was allowed " + newCommitTime);
|
||||
} catch (HoodieRollbackException e) {
|
||||
// this is good
|
||||
}
|
||||
|
||||
/**
|
||||
* Write 3 (updates)
|
||||
*/
|
||||
newCommitTime = "003";
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
/**
|
||||
* Write 4 (updates)
|
||||
*/
|
||||
newCommitTime = "004";
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
List<String> partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
final TableFileSystemView view = table.getFileSystemView();
|
||||
List<HoodieDataFile> dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
Stream<List<HoodieDataFile>> files = view.getEveryVersionInPartition(s);
|
||||
return files.flatMap(Collection::stream).filter(f -> f.getCommitTime().equals("002"));
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
assertEquals("The data files for commit 002 should not be cleaned", 3, dataFiles.size());
|
||||
|
||||
// Delete savepoint
|
||||
assertFalse(table.getCompletedSavepointTimeline().empty());
|
||||
client.deleteSavepoint(
|
||||
table.getCompletedSavepointTimeline().getInstants().findFirst().get().getTimestamp());
|
||||
// rollback and reupsert 004
|
||||
client.rollback(newCommitTime);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
final TableFileSystemView view1 = table.getFileSystemView();
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
Stream<List<HoodieDataFile>> files = view1.getEveryVersionInPartition(s);
|
||||
return files.flatMap(Collection::stream).filter(f -> f.getCommitTime().equals("002"));
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
assertEquals("The data files for commit 002 should be cleaned now", 0, dataFiles.size());
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testRollbackToSavepoint() throws Exception {
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder()
|
||||
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1)
|
||||
.build()).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
|
||||
/**
|
||||
* Write 1 (only inserts)
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
/**
|
||||
* Write 2 (updates)
|
||||
*/
|
||||
newCommitTime = "002";
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
client.savepoint(new HoodieSavepointMetadata("hoodie-unit-test",
|
||||
HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date()), "test"));
|
||||
|
||||
/**
|
||||
* Write 3 (updates)
|
||||
*/
|
||||
newCommitTime = "003";
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
List<String> partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
final TableFileSystemView view1 = table.getFileSystemView();
|
||||
|
||||
List<HoodieDataFile> dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
Stream<List<HoodieDataFile>> files = view1.getEveryVersionInPartition(s);
|
||||
return files.flatMap(Collection::stream).filter(f -> f.getCommitTime().equals("003"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 003 should be present", 3, dataFiles.size());
|
||||
|
||||
|
||||
/**
|
||||
* Write 4 (updates)
|
||||
*/
|
||||
newCommitTime = "004";
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
final TableFileSystemView view2 = table.getFileSystemView();
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
Stream<List<HoodieDataFile>> files = view2.getEveryVersionInPartition(s);
|
||||
return files.flatMap(Collection::stream).filter(f -> f.getCommitTime().equals("004"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 004 should be present", 3, dataFiles.size());
|
||||
|
||||
|
||||
// rolling back to a non existent savepoint must not succeed
|
||||
try {
|
||||
client.rollbackToSavepoint("001");
|
||||
fail("Rolling back to non-existent savepoint should not be allowed");
|
||||
} catch (HoodieRollbackException e) {
|
||||
// this is good
|
||||
}
|
||||
|
||||
// rollback to savepoint 002
|
||||
HoodieInstant savepoint =
|
||||
table.getCompletedSavepointTimeline().getInstants().findFirst().get();
|
||||
client.rollbackToSavepoint(savepoint.getTimestamp());
|
||||
|
||||
metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
final TableFileSystemView view3 = table.getFileSystemView();
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
Stream<List<HoodieDataFile>> files = view3.getEveryVersionInPartition(s);
|
||||
return files.flatMap(Collection::stream).filter(f -> f.getCommitTime().equals("002"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 002 be available", 3, dataFiles.size());
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
Stream<List<HoodieDataFile>> files = view3.getEveryVersionInPartition(s);
|
||||
return files.flatMap(Collection::stream).filter(f -> f.getCommitTime().equals("003"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 003 should be rolled back", 0, dataFiles.size());
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
Stream<List<HoodieDataFile>> files = view3.getEveryVersionInPartition(s);
|
||||
return files.flatMap(Collection::stream).filter(f -> f.getCommitTime().equals("004"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("The data files for commit 004 should be rolled back", 0, dataFiles.size());
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testInsertAndCleanByVersions() throws Exception {
|
||||
int maxVersions = 2; // keep upto 2 versions for each file
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder()
|
||||
.withCleanerPolicy(HoodieCleaner.CleaningPolicy.KEEP_LATEST_FILE_VERSIONS)
|
||||
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS)
|
||||
.retainFileVersions(maxVersions).build()).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
|
||||
@@ -365,6 +563,13 @@ public class TestHoodieClient implements Serializable {
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
assertFalse(table.getCompletedCommitTimeline().empty());
|
||||
String commitTime =
|
||||
table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp();
|
||||
assertFalse(table.getCompletedCleanTimeline().empty());
|
||||
assertEquals("The clean instant should be the same as the commit instant", commitTime,
|
||||
table.getCompletedCleanTimeline().getInstants().findFirst().get().getTimestamp());
|
||||
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
|
||||
@@ -425,7 +630,7 @@ public class TestHoodieClient implements Serializable {
|
||||
int maxCommits = 3; // keep upto 3 commits from the past
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder()
|
||||
.withCleanerPolicy(HoodieCleaner.CleaningPolicy.KEEP_LATEST_FILE_VERSIONS)
|
||||
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS)
|
||||
.retainCommits(maxCommits).build()).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
|
||||
@@ -450,6 +655,13 @@ public class TestHoodieClient implements Serializable {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
|
||||
assertFalse(table.getCompletedCommitTimeline().empty());
|
||||
String commitTime =
|
||||
table.getCompletedCommitTimeline().getInstants().findFirst().get().getTimestamp();
|
||||
assertFalse(table.getCompletedCleanTimeline().empty());
|
||||
assertEquals("The clean instant should be the same as the commit instant", commitTime,
|
||||
table.getCompletedCleanTimeline().getInstants().findFirst().get().getTimestamp());
|
||||
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
|
||||
@@ -843,6 +1055,7 @@ public class TestHoodieClient implements Serializable {
|
||||
|
||||
|
||||
|
||||
|
||||
@After
|
||||
public void clean() {
|
||||
if (basePath != null) {
|
||||
|
||||
@@ -16,7 +16,9 @@
|
||||
|
||||
package com.uber.hoodie.io;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
@@ -26,6 +28,10 @@ import com.uber.hoodie.table.HoodieTable;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
/**
|
||||
@@ -47,7 +53,7 @@ public class TestHoodieCleaner {
|
||||
public void testKeepLatestFileVersions() throws IOException {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withCleanerPolicy(HoodieCleaner.CleaningPolicy.KEEP_LATEST_FILE_VERSIONS)
|
||||
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS)
|
||||
.retainFileVersions(1).build()).build();
|
||||
|
||||
// make 1 commit, with 1 file per partition
|
||||
@@ -60,8 +66,8 @@ public class TestHoodieCleaner {
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
||||
|
||||
HoodieCleaner cleaner = new HoodieCleaner(table, config);
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]));
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]).getSuccessDeleteFiles().size());
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]).getSuccessDeleteFiles().size());
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[1], "000", file1P1C0));
|
||||
|
||||
@@ -76,8 +82,8 @@ public class TestHoodieCleaner {
|
||||
table = HoodieTable.getHoodieTable(metadata, config);
|
||||
|
||||
cleaner = new HoodieCleaner(table, config);
|
||||
assertEquals("Must clean 1 file" , 1, cleaner.clean(partitionPaths[0]));
|
||||
assertEquals("Must clean 1 file" , 1, cleaner.clean(partitionPaths[1]));
|
||||
assertEquals("Must clean 1 file" , 1, cleaner.clean(partitionPaths[0]).getSuccessDeleteFiles().size());
|
||||
assertEquals("Must clean 1 file" , 1, cleaner.clean(partitionPaths[1]).getSuccessDeleteFiles().size());
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[1], "001", file2P1C1));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0));
|
||||
@@ -93,14 +99,14 @@ public class TestHoodieCleaner {
|
||||
table = HoodieTable.getHoodieTable(metadata, config);
|
||||
|
||||
cleaner = new HoodieCleaner(table, config);
|
||||
assertEquals("Must clean two files" , 2, cleaner.clean(partitionPaths[0]));
|
||||
assertEquals("Must clean two files" , 2, cleaner.clean(partitionPaths[0]).getSuccessDeleteFiles().size());
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file1P0C0));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "002", file3P0C2));
|
||||
|
||||
// No cleaning on partially written file, with no commit.
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file3P0C2); // update
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]).getSuccessDeleteFiles().size());
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "002", file3P0C2));
|
||||
}
|
||||
|
||||
@@ -109,7 +115,7 @@ public class TestHoodieCleaner {
|
||||
public void testKeepLatestCommits() throws IOException {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withCleanerPolicy(HoodieCleaner.CleaningPolicy.KEEP_LATEST_COMMITS)
|
||||
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS)
|
||||
.retainCommits(2).build()).build();
|
||||
|
||||
|
||||
@@ -123,8 +129,8 @@ public class TestHoodieCleaner {
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
||||
|
||||
HoodieCleaner cleaner = new HoodieCleaner(table, config);
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]));
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]).getSuccessDeleteFiles().size());
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]).getSuccessDeleteFiles().size());
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[1], "000", file1P1C0));
|
||||
|
||||
@@ -139,8 +145,8 @@ public class TestHoodieCleaner {
|
||||
table = HoodieTable.getHoodieTable(metadata, config);
|
||||
|
||||
cleaner = new HoodieCleaner(table, config);
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]));
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]).getSuccessDeleteFiles().size());
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]).getSuccessDeleteFiles().size());
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[1], "001", file2P1C1));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0));
|
||||
@@ -158,7 +164,7 @@ public class TestHoodieCleaner {
|
||||
cleaner = new HoodieCleaner(table, config);
|
||||
assertEquals(
|
||||
"Must not clean any file. We have to keep 1 version before the latest commit time to keep",
|
||||
0, cleaner.clean(partitionPaths[0]));
|
||||
0, cleaner.clean(partitionPaths[0]).getSuccessDeleteFiles().size());
|
||||
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0));
|
||||
|
||||
@@ -173,7 +179,7 @@ public class TestHoodieCleaner {
|
||||
|
||||
cleaner = new HoodieCleaner(table, config);
|
||||
assertEquals(
|
||||
"Must not clean one old file", 1, cleaner.clean(partitionPaths[0]));
|
||||
"Must not clean one old file", 1, cleaner.clean(partitionPaths[0]).getSuccessDeleteFiles().size());
|
||||
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file1P0C0));
|
||||
@@ -185,7 +191,7 @@ public class TestHoodieCleaner {
|
||||
|
||||
// No cleaning on partially written file, with no commit.
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "004", file3P0C2); // update
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]).getSuccessDeleteFiles().size());
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file1P0C0));
|
||||
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1));
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user