Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[HUDI-7532] Include only compaction instants for lastCompaction in getDeltaCommitsSinceLatestCompaction #10915

Merged
merged 3 commits into from
May 15, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -320,7 +320,7 @@ private static String printAllCompactions(HoodieDefaultTimeline timeline,
.filter(pair -> pair.getRight() != null)
.collect(Collectors.toList());

Set<String> committedInstants = timeline.getCommitTimeline().filterCompletedInstants()
Set<String> committedInstants = timeline.getCommitAndReplaceTimeline().filterCompletedInstants()
.getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet());

List<Comparable[]> rows = new ArrayList<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -247,7 +247,7 @@ private HoodieTableFileSystemView buildFileSystemView(String globRegex, String m

HoodieTimeline timeline;
if (basefileOnly) {
timeline = metaClient.getActiveTimeline().getCommitTimeline();
timeline = metaClient.getActiveTimeline().getCommitAndReplaceTimeline();
} else if (excludeCompaction) {
timeline = metaClient.getActiveTimeline().getCommitsTimeline();
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -232,7 +232,7 @@ storage, new StoragePath(logFilePathPattern)).stream()
.withReaderSchema(readerSchema)
.withLatestInstantTime(
client.getActiveTimeline()
.getCommitTimeline().lastInstant().get().getTimestamp())
.getCommitAndReplaceTimeline().lastInstant().get().getTimestamp())
.withReverseReader(
Boolean.parseBoolean(
HoodieReaderConfig.COMPACTION_REVERSE_LOG_READ_ENABLE.defaultValue()))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,7 @@ public String addPartitionMeta(

HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
String latestCommit =
client.getActiveTimeline().getCommitTimeline().lastInstant().get().getTimestamp();
client.getActiveTimeline().getCommitAndReplaceTimeline().lastInstant().get().getTimestamp();
List<String> partitionPaths =
FSUtils.getAllPartitionFoldersThreeLevelsDown(HoodieCLI.storage, client.getBasePath());
StoragePath basePath = client.getBasePathV2();
Expand Down Expand Up @@ -240,7 +240,7 @@ public String migratePartitionMeta(
Option<StoragePath> baseFormatFile =
HoodiePartitionMetadata.baseFormatMetaPathIfExists(HoodieCLI.storage, partition);
String latestCommit =
client.getActiveTimeline().getCommitTimeline().lastInstant().get().getTimestamp();
client.getActiveTimeline().getCommitAndReplaceTimeline().lastInstant().get().getTimestamp();

String[] row = new String[] {
partitionPath,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ public String writeAmplificationStats(
long totalRecordsWritten = 0;

HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().getActiveTimeline();
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
HoodieTimeline timeline = activeTimeline.getCommitAndReplaceTimeline().filterCompletedInstants();

List<Comparable[]> rows = new ArrayList<>();
DecimalFormat df = new DecimalFormat("#.00");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ public class CommitUtil {

public static long countNewRecords(HoodieTableMetaClient metaClient, List<String> commitsToCatchup) throws IOException {
long totalNew = 0;
HoodieTimeline timeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
HoodieTimeline timeline = metaClient.reloadActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants();
for (String commit : commitsToCatchup) {
HoodieCommitMetadata c = HoodieCommitMetadata.fromBytes(
timeline.getInstantDetails(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commit)).get(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -207,7 +207,7 @@ private void testRefreshCommand(String command) throws IOException {
assertTrue(prepareTable());

HoodieTimeline timeline =
HoodieCLI.getTableMetaClient().getActiveTimeline().getCommitTimeline().filterCompletedInstants();
HoodieCLI.getTableMetaClient().getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants();
assertEquals(0, timeline.countInstants(), "There should have no instant at first");

// generate four savepoints
Expand All @@ -218,14 +218,14 @@ private void testRefreshCommand(String command) throws IOException {

// Before refresh, no instant
timeline =
HoodieCLI.getTableMetaClient().getActiveTimeline().getCommitTimeline().filterCompletedInstants();
HoodieCLI.getTableMetaClient().getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants();
assertEquals(0, timeline.countInstants(), "there should have no instant");

Object result = shell.evaluate(() -> command);
assertTrue(ShellEvaluationResultUtil.isSuccess(result));

timeline =
HoodieCLI.getTableMetaClient().getActiveTimeline().getCommitTimeline().filterCompletedInstants();
HoodieCLI.getTableMetaClient().getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants();

// After refresh, there are 4 instants
assertEquals(4, timeline.countInstants(), "there should have 4 instants");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,7 @@ public void testRollbackToSavepoint() throws IOException {
assertEquals(1, timeline.getRestoreTimeline().countInstants());

// 103 instant had rollback
assertFalse(timeline.getCommitTimeline().containsInstant(
assertFalse(timeline.getCommitAndReplaceTimeline().containsInstant(
new HoodieInstant(HoodieInstant.State.COMPLETED, "commit", "103")));
}

Expand Down Expand Up @@ -182,9 +182,9 @@ public void testRollbackToSavepointWithMetadataTableEnable() throws Exception {
assertEquals(1, timeline.getRestoreTimeline().countInstants());

// 103 and 104 instant had rollback
assertFalse(timeline.getCommitTimeline().containsInstant(
assertFalse(timeline.getCommitAndReplaceTimeline().containsInstant(
new HoodieInstant(HoodieInstant.State.COMPLETED, "commit", "103")));
assertFalse(timeline.getCommitTimeline().containsInstant(
assertFalse(timeline.getCommitAndReplaceTimeline().containsInstant(
new HoodieInstant(HoodieInstant.State.COMPLETED, "commit", "104")));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -143,7 +143,7 @@ public static Option<HoodieConsistentHashingMetadata> loadMetadata(HoodieTable t
&& maxCommitMetaFileTs.equals(HoodieConsistentHashingMetadata.getTimestampFromFile(maxMetadataFile.getPath().getName()))) {
return loadMetadataFromGivenFile(table, maxMetadataFile);
}
HoodieTimeline completedCommits = metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
HoodieTimeline completedCommits = metaClient.getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants();

// fix the in-consistency between un-committed and committed hashing metadata files.
List<FileStatus> fixed = new ArrayList<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1357,7 +1357,7 @@ protected void compactIfNecessary(BaseHoodieWriteClient writeClient) {

protected void cleanIfNecessary(BaseHoodieWriteClient writeClient) {
Option<HoodieInstant> lastCompletedCompactionInstant = metadataMetaClient.reloadActiveTimeline()
.getCommitTimeline().filterCompletedInstants().lastInstant();
.getCommitAndReplaceTimeline().filterCompletedInstants().lastInstant();
if (lastCompletedCompactionInstant.isPresent()
&& metadataMetaClient.getActiveTimeline().filterCompletedInstants()
.findInstantsAfter(lastCompletedCompactionInstant.get().getTimestamp()).countInstants() < 3) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,7 @@ private void assignInserts(WorkloadProfile profile, HoodieEngineContext context)
// for new inserts, compute buckets depending on how many records we have for each partition
Set<String> partitionPaths = profile.getPartitionPaths();
long averageRecordSize =
averageBytesPerRecord(table.getMetaClient().getActiveTimeline().getCommitTimeline().filterCompletedInstants(),
averageBytesPerRecord(table.getMetaClient().getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants(),
config);
LOG.info("AvgRecordSize => " + averageRecordSize);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1729,7 +1729,7 @@ public void testMetadataMultiWriter() throws Exception {
assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "0000004")));

// Compaction may occur if the commits completed in order
assertTrue(metadataMetaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants().countInstants() <= 1);
assertTrue(metadataMetaClient.getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants().countInstants() <= 1);

// Validation
validateMetadata(writeClients[0]);
Expand Down Expand Up @@ -1776,7 +1776,7 @@ public void testMultiWriterForDoubleLocking() throws Exception {

// 6 commits and 2 cleaner commits.
assertEquals(metadataMetaClient.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants().countInstants(), 8);
assertTrue(metadataMetaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants().countInstants() <= 1);
assertTrue(metadataMetaClient.getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants().countInstants() <= 1);
// Validation
validateMetadata(writeClient);
}
Expand Down Expand Up @@ -2049,7 +2049,7 @@ public void testCleaningArchivingAndCompaction() throws Exception {
// There should not be any compaction yet and we have not performed more than maxDeltaCommitsBeforeCompaction
// deltacommits (1 will be due to bootstrap)
HoodieActiveTimeline metadataTimeline = metadataMetaClient.reloadActiveTimeline();
assertEquals(metadataTimeline.getCommitTimeline().filterCompletedInstants().countInstants(), 0);
assertEquals(metadataTimeline.getCommitAndReplaceTimeline().filterCompletedInstants().countInstants(), 0);
assertEquals(metadataTimeline.getCommitsTimeline().filterCompletedInstants().countInstants(), maxDeltaCommitsBeforeCompaction - 1);
assertEquals(datasetMetaClient.getArchivedTimeline().reload().countInstants(), 0);

Expand All @@ -2059,7 +2059,7 @@ public void testCleaningArchivingAndCompaction() throws Exception {
client.startCommitWithTime(newCommitTime);
client.insert(records, newCommitTime);
metadataTimeline = metadataMetaClient.reloadActiveTimeline();
assertEquals(metadataTimeline.getCommitTimeline().filterCompletedInstants().countInstants(), 1);
assertEquals(metadataTimeline.getCommitAndReplaceTimeline().filterCompletedInstants().countInstants(), 1);
assertEquals(metadataTimeline.getCommitsTimeline().filterCompletedInstants().countInstants(), maxDeltaCommitsBeforeCompaction + 1);
assertEquals(datasetMetaClient.getArchivedTimeline().reload().countInstants(), 0);

Expand All @@ -2080,7 +2080,7 @@ public void testCleaningArchivingAndCompaction() throws Exception {

// Ensure no more compactions took place due to the leftover inflight commit
metadataTimeline = metadataMetaClient.reloadActiveTimeline();
assertEquals(metadataTimeline.getCommitTimeline().filterCompletedInstants().countInstants(), 1);
assertEquals(metadataTimeline.getCommitAndReplaceTimeline().filterCompletedInstants().countInstants(), 1);
assertEquals(metadataTimeline.getDeltaCommitTimeline().filterCompletedInstants().countInstants(),
((2 * maxDeltaCommitsBeforeCompaction) + (maxDeltaCommitsBeforeCompaction /* clean from dataset */) + 1)/* clean in metadata table */);

Expand All @@ -2095,7 +2095,7 @@ public void testCleaningArchivingAndCompaction() throws Exception {

// Ensure compactions took place
metadataTimeline = metadataMetaClient.reloadActiveTimeline();
assertEquals(metadataTimeline.getCommitTimeline().filterCompletedInstants().countInstants(), 2);
assertEquals(metadataTimeline.getCommitAndReplaceTimeline().filterCompletedInstants().countInstants(), 2);
assertEquals(metadataTimeline.getDeltaCommitTimeline().filterCompletedInstants().countInstants(),
((2 * maxDeltaCommitsBeforeCompaction) + (maxDeltaCommitsBeforeCompaction + 1 /* clean from dataset */) + 2 /* clean in metadata table */));
assertTrue(datasetMetaClient.getArchivedTimeline().reload().countInstants() > 0);
Expand Down Expand Up @@ -2445,7 +2445,7 @@ public void testRepeatedActionWithSameInstantTime() throws Exception {
client.upsert(records, newCommitTime);
}
}
assertEquals(metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants().countInstants(), 3);
assertEquals(metaClient.reloadActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants().countInstants(), 3);

try (HoodieJavaWriteClient client = new HoodieJavaWriteClient(engineContext, writeConfig)) {
// Perform a clean
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -517,7 +517,7 @@ private void testUpsertsInternal(HoodieWriteConfig config,
0, 150);

HoodieActiveTimeline activeTimeline = new HoodieActiveTimeline(metaClient, false);
List<HoodieInstant> instants = activeTimeline.getCommitTimeline().getInstants();
List<HoodieInstant> instants = activeTimeline.getCommitAndReplaceTimeline().getInstants();
assertEquals(5, instants.size());
assertEquals(new HoodieInstant(COMPLETED, COMMIT_ACTION, "001"),
instants.get(0));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -867,7 +867,7 @@ private List<WriteStatus> getWriteStatusAndVerifyDeleteOperation(String newCommi

// verify that there is a commit
HoodieTableMetaClient metaClient = createMetaClient();
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitAndReplaceTimeline();

if (assertForCommit) {
assertEquals(3, timeline.findInstantsAfter(initCommitTime, Integer.MAX_VALUE).countInstants(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,7 @@ public void readLocalWriteHDFS() throws Exception {
// Read from hdfs
FileSystem fs = HadoopFSUtils.getFs(dfsBasePath, HoodieTestUtils.getDefaultStorageConf());
HoodieTableMetaClient metaClient = HoodieTestUtils.createMetaClient(HadoopFSUtils.getStorageConf(fs.getConf()), dfsBasePath);
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitAndReplaceTimeline();
Dataset<Row> readRecords = HoodieClientTestUtils.readCommit(dfsBasePath, sqlContext, timeline, readCommitTime);
assertEquals(readRecords.count(), records.size());

Expand All @@ -156,7 +156,7 @@ public void readLocalWriteHDFS() throws Exception {
LOG.info("Reading from path: " + tablePath);
fs = HadoopFSUtils.getFs(tablePath, HoodieTestUtils.getDefaultStorageConf());
metaClient = HoodieTestUtils.createMetaClient(new HadoopStorageConfiguration(fs.getConf()), tablePath);
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
timeline = new HoodieActiveTimeline(metaClient).getCommitAndReplaceTimeline();
Dataset<Row> localReadRecords =
HoodieClientTestUtils.readCommit(tablePath, sqlContext, timeline, writeCommitTime);
assertEquals(localReadRecords.count(), localRecords.size());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -309,7 +309,7 @@ public void testCopyOnWriteTable(boolean shouldAllowDroppedColumns) throws Excep
(String s, Integer a) -> evolvedRecords, SparkRDDWriteClient::insert, true, numRecords, 3 * numRecords, 6, false);

// new commit
HoodieTimeline curTimeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
HoodieTimeline curTimeline = metaClient.reloadActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants();
assertTrue(curTimeline.lastInstant().get().getTimestamp().equals("006"));
checkReadRecords("000", 3 * numRecords);

Expand All @@ -333,7 +333,7 @@ public void testCopyOnWriteTable(boolean shouldAllowDroppedColumns) throws Excep

private void checkReadRecords(String instantTime, int numExpectedRecords) throws IOException {
if (tableType == HoodieTableType.COPY_ON_WRITE) {
HoodieTimeline timeline = metaClient.reloadActiveTimeline().getCommitTimeline();
HoodieTimeline timeline = metaClient.reloadActiveTimeline().getCommitAndReplaceTimeline();
assertEquals(numExpectedRecords, HoodieClientTestUtils.countRecordsOptionallySince(jsc, basePath, sqlContext, timeline, Option.of(instantTime)));
} else {
// TODO: This code fails to read records under the following conditions:
Expand Down
Loading
Loading