Removing compaction action type and associated compaction timeline operations, replace with commit action type
This commit is contained in:
committed by
vinoth chandar
parent
a1c0d0dbad
commit
44839b88c6
@@ -649,7 +649,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
table = HoodieTable.getHoodieTable(metadata, getConfig());
|
||||
timeline = table.getCommitTimeline();
|
||||
timeline = table.getCommitsTimeline();
|
||||
|
||||
TableFileSystemView fsView = table.getFileSystemView();
|
||||
// Need to ensure the following
|
||||
@@ -1493,10 +1493,10 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
HoodieTestUtils.doesCommitExist(basePath, commitTime));
|
||||
|
||||
// Get parquet file paths from commit metadata
|
||||
String actionType = table.getCompactedCommitActionType();
|
||||
String actionType = table.getCommitActionType();
|
||||
HoodieInstant commitInstant =
|
||||
new HoodieInstant(false, actionType, commitTime);
|
||||
HoodieTimeline commitTimeline = table.getCompletedCompactionCommitTimeline();
|
||||
HoodieTimeline commitTimeline = table.getCommitTimeline().filterCompletedInstants();
|
||||
HoodieCommitMetadata commitMetadata =
|
||||
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commitInstant).get());
|
||||
String basePath = table.getMetaClient().getBasePath();
|
||||
|
||||
@@ -84,7 +84,7 @@ public class TestHoodieCommitArchiveLog {
|
||||
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTimeline timeline =
|
||||
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
||||
metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
|
||||
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
|
||||
|
||||
@@ -155,13 +155,13 @@ public class TestHoodieCommitArchiveLog {
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "103");
|
||||
|
||||
HoodieTimeline timeline =
|
||||
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
||||
metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
|
||||
assertEquals("Loaded 4 commits and the count should match", 4, timeline.countInstants());
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
assertTrue(result);
|
||||
timeline =
|
||||
metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline()
|
||||
metadata.getActiveTimeline().reload().getCommitsTimeline()
|
||||
.filterCompletedInstants();
|
||||
assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4,
|
||||
timeline.countInstants());
|
||||
@@ -183,12 +183,12 @@ public class TestHoodieCommitArchiveLog {
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "105");
|
||||
|
||||
HoodieTimeline timeline =
|
||||
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
||||
metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
assertTrue(result);
|
||||
timeline =
|
||||
metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline()
|
||||
metadata.getActiveTimeline().reload().getCommitsTimeline()
|
||||
.filterCompletedInstants();
|
||||
assertTrue("Archived commits should always be safe",
|
||||
timeline.containsOrBeforeTimelineStarts("100"));
|
||||
@@ -217,12 +217,12 @@ public class TestHoodieCommitArchiveLog {
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "105");
|
||||
|
||||
HoodieTimeline timeline =
|
||||
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
||||
metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
assertTrue(result);
|
||||
timeline =
|
||||
metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline()
|
||||
metadata.getActiveTimeline().reload().getCommitsTimeline()
|
||||
.filterCompletedInstants();
|
||||
assertEquals(
|
||||
"Since we have a savepoint at 101, we should never archive any commit after 101 (we only archive 100)",
|
||||
|
||||
@@ -21,7 +21,7 @@ import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
@@ -49,7 +49,6 @@ import org.junit.rules.TemporaryFolder;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
@@ -124,7 +123,7 @@ public class TestHoodieCompactor {
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||
|
||||
HoodieCompactionMetadata result =
|
||||
HoodieCommitMetadata result =
|
||||
compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime());
|
||||
String basePath = table.getMetaClient().getBasePath();
|
||||
assertTrue("If there is nothing to compact, result will be empty",
|
||||
@@ -178,7 +177,7 @@ public class TestHoodieCompactor {
|
||||
metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config);
|
||||
|
||||
HoodieCompactionMetadata result =
|
||||
HoodieCommitMetadata result =
|
||||
compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime());
|
||||
|
||||
// Verify that recently written compacted data file has no log file
|
||||
@@ -199,7 +198,7 @@ public class TestHoodieCompactor {
|
||||
"After compaction there should be no log files visiable on a Realtime view",
|
||||
slice.getLogFiles().collect(Collectors.toList()).isEmpty());
|
||||
}
|
||||
assertTrue(result.getPartitionToCompactionWriteStats().containsKey(partitionPath));
|
||||
assertTrue(result.getPartitionToWriteStats().containsKey(partitionPath));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -182,7 +182,7 @@ public class TestMergeOnReadTable {
|
||||
FileStatus[] allFiles = HoodieTestUtils
|
||||
.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient,
|
||||
hoodieTable.getCompletedCompactionCommitTimeline(), allFiles);
|
||||
hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles);
|
||||
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
|
||||
assertTrue(!dataFilesToRead.findAny().isPresent());
|
||||
|
||||
@@ -231,7 +231,7 @@ public class TestMergeOnReadTable {
|
||||
// verify that there is a commit
|
||||
table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, cfg.getBasePath(), true), getConfig(false));
|
||||
HoodieTimeline timeline = table.getCompletedCompactionCommitTimeline();
|
||||
HoodieTimeline timeline = table.getCommitTimeline().filterCompletedInstants();
|
||||
assertEquals("Expecting a single commit.", 1,
|
||||
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
|
||||
String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp();
|
||||
@@ -299,7 +299,7 @@ public class TestMergeOnReadTable {
|
||||
FileStatus[] allFiles = HoodieTestUtils
|
||||
.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient,
|
||||
hoodieTable.getCompletedCompactionCommitTimeline(), allFiles);
|
||||
hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles);
|
||||
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
|
||||
assertTrue(!dataFilesToRead.findAny().isPresent());
|
||||
|
||||
@@ -455,7 +455,7 @@ public class TestMergeOnReadTable {
|
||||
FileStatus[] allFiles = HoodieTestUtils
|
||||
.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||
TableFileSystemView.ReadOptimizedView roView = new HoodieTableFileSystemView(metaClient,
|
||||
hoodieTable.getCompletedCompactionCommitTimeline(), allFiles);
|
||||
hoodieTable.getCommitTimeline().filterCompletedInstants(), allFiles);
|
||||
Stream<HoodieDataFile> dataFilesToRead = roView.getLatestDataFiles();
|
||||
assertTrue(!dataFilesToRead.findAny().isPresent());
|
||||
|
||||
@@ -524,11 +524,11 @@ public class TestMergeOnReadTable {
|
||||
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
|
||||
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
|
||||
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompactionCommitTimeline(),
|
||||
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(),
|
||||
allFiles);
|
||||
|
||||
final String compactedCommitTime = metaClient.getActiveTimeline().reload()
|
||||
.getCommitsAndCompactionsTimeline().lastInstant().get().getTimestamp();
|
||||
.getCommitsTimeline().lastInstant().get().getTimestamp();
|
||||
|
||||
assertTrue(roView.getLatestDataFiles().filter(file -> {
|
||||
if (compactedCommitTime.equals(file.getCommitTime())) {
|
||||
@@ -543,7 +543,7 @@ public class TestMergeOnReadTable {
|
||||
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
|
||||
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
|
||||
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompactionCommitTimeline(),
|
||||
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(),
|
||||
allFiles);
|
||||
|
||||
assertFalse(roView.getLatestDataFiles().filter(file -> {
|
||||
|
||||
Reference in New Issue
Block a user