-
Notifications
You must be signed in to change notification settings - Fork 183
Handle empty EarliestCommitToRetain #732
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
Open
alexr17
wants to merge
1
commit into
apache:main
Choose a base branch
from
alexr17:alexr/handle-earliest-commit-to-retain-empty
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
253 changes: 253 additions & 0 deletions
253
xtable-core/src/test/java/org/apache/xtable/hudi/TestHudiConversionSource.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,253 @@ | ||
/* | ||
* 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.xtable.hudi; | ||
|
||
import static org.junit.jupiter.api.Assertions.assertFalse; | ||
import static org.junit.jupiter.api.Assertions.assertTrue; | ||
import static org.mockito.ArgumentMatchers.any; | ||
import static org.mockito.Mockito.mock; | ||
import static org.mockito.Mockito.when; | ||
|
||
import java.time.Instant; | ||
import java.util.Arrays; | ||
import java.util.Collections; | ||
import java.util.Properties; | ||
|
||
import org.apache.hadoop.conf.Configuration; | ||
import org.junit.jupiter.api.Test; | ||
|
||
import org.apache.hudi.avro.model.HoodieCleanMetadata; | ||
import org.apache.hudi.common.table.HoodieTableMetaClient; | ||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; | ||
import org.apache.hudi.common.table.timeline.HoodieInstant; | ||
import org.apache.hudi.common.table.timeline.HoodieTimeline; | ||
import org.apache.hudi.common.util.Option; | ||
|
||
class TestHudiConversionSource { | ||
|
||
@Test | ||
void testIsIncrementalSyncSafeFromWithNullEarliestCommitToRetainNoCleanInstants() | ||
throws Exception { | ||
// Mock the dependencies | ||
HoodieTableMetaClient mockMetaClient = mock(HoodieTableMetaClient.class); | ||
HoodieActiveTimeline mockActiveTimeline = mock(HoodieActiveTimeline.class); | ||
HoodieTimeline mockCleanerTimeline = mock(HoodieTimeline.class); | ||
HoodieTimeline mockFilteredCleanerTimeline = mock(HoodieTimeline.class); | ||
HoodieTimeline mockCompletedCommitsTimeline = mock(HoodieTimeline.class); | ||
HoodieInstant mockCleanInstant = mock(HoodieInstant.class); | ||
HoodieInstant mockCommitInstant = mock(HoodieInstant.class); | ||
HoodieCleanMetadata mockCleanMetadata = mock(HoodieCleanMetadata.class); | ||
|
||
// Set up the mock chain for cleaner timeline | ||
when(mockMetaClient.getActiveTimeline()).thenReturn(mockActiveTimeline); | ||
|
||
// Mock the Hadoop configuration to prevent NPE in HudiDataFileExtractor | ||
Configuration hadoopConf = new Configuration(); | ||
hadoopConf.addResource("core-default.xml"); | ||
when(mockMetaClient.getHadoopConf()).thenReturn(hadoopConf); | ||
|
||
// Mock table config to prevent NPE in HudiDataFileExtractor | ||
org.apache.hudi.common.table.HoodieTableConfig mockTableConfig = | ||
mock(org.apache.hudi.common.table.HoodieTableConfig.class); | ||
when(mockMetaClient.getTableConfig()).thenReturn(mockTableConfig); | ||
when(mockTableConfig.isMetadataTableAvailable()).thenReturn(false); | ||
when(mockActiveTimeline.getCleanerTimeline()).thenReturn(mockCleanerTimeline); | ||
when(mockCleanerTimeline.filterCompletedInstants()).thenReturn(mockCleanerTimeline); | ||
when(mockCleanerTimeline.lastInstant()).thenReturn(Option.of(mockCleanInstant)); | ||
when(mockActiveTimeline.deserializeInstantContent(mockCleanInstant, HoodieCleanMetadata.class)) | ||
.thenReturn(mockCleanMetadata); | ||
|
||
// Set up the key behavior: earliestCommitToRetain is null | ||
when(mockCleanMetadata.getEarliestCommitToRetain()).thenReturn(null); | ||
|
||
// Set up mocks for handleEmptyEarliestCommitToRetain - no clean instants after last sync | ||
when(mockCleanerTimeline.filter(any())).thenReturn(mockFilteredCleanerTimeline); | ||
when(mockFilteredCleanerTimeline.getInstants()).thenReturn(Collections.emptyList()); | ||
|
||
// Set up the mock chain for commit timeline (for doesCommitExistsAsOfInstant) | ||
when(mockActiveTimeline.filterCompletedInstants()).thenReturn(mockCompletedCommitsTimeline); | ||
when(mockCompletedCommitsTimeline.findInstantsBeforeOrEquals(any(String.class))) | ||
.thenReturn(mockCompletedCommitsTimeline); | ||
when(mockCompletedCommitsTimeline.lastInstant()).thenReturn(Option.of(mockCommitInstant)); | ||
|
||
// Create the HudiConversionSource with proper Configuration | ||
Configuration conf = new Configuration(); | ||
conf.addResource("core-default.xml"); | ||
conf.addResource("core-site.xml"); | ||
conf.addResource("hdfs-default.xml"); | ||
conf.addResource("hdfs-site.xml"); | ||
HudiConversionSource hudiConversionSource = | ||
new HudiConversionSource( | ||
mockMetaClient, mock(HudiSourcePartitionSpecExtractor.class), conf, new Properties()); | ||
|
||
// Test that isIncrementalSyncSafeFrom returns true when earliestCommitToRetain is null | ||
// and no clean instants after last sync | ||
Instant testInstant = Instant.now().minusSeconds(3600); // 1 hour ago | ||
boolean result = hudiConversionSource.isIncrementalSyncSafeFrom(testInstant); | ||
|
||
// This should return true because when earliestCommitToRetain is null and no clean instants | ||
// after last sync, | ||
// handleEmptyEarliestCommitToRetain returns false, making isAffectedByCleanupProcess return | ||
// false | ||
assertTrue( | ||
result, | ||
"isIncrementalSyncSafeFrom should return true when earliestCommitToRetain is null and no clean instants after last sync"); | ||
} | ||
|
||
@Test | ||
void testIsIncrementalSyncSafeFromWithNullEarliestCommitToRetainWithCleanInstants() | ||
throws Exception { | ||
// Mock the dependencies | ||
HoodieTableMetaClient mockMetaClient = mock(HoodieTableMetaClient.class); | ||
HoodieActiveTimeline mockActiveTimeline = mock(HoodieActiveTimeline.class); | ||
HoodieTimeline mockCleanerTimeline = mock(HoodieTimeline.class); | ||
HoodieTimeline mockFilteredCleanerTimeline = mock(HoodieTimeline.class); | ||
HoodieTimeline mockCompletedCommitsTimeline = mock(HoodieTimeline.class); | ||
HoodieInstant mockCleanInstant = mock(HoodieInstant.class); | ||
HoodieInstant mockCleanInstantAfterSync = mock(HoodieInstant.class); | ||
HoodieInstant mockCommitInstant = mock(HoodieInstant.class); | ||
HoodieCleanMetadata mockCleanMetadata = mock(HoodieCleanMetadata.class); | ||
|
||
// Set up the mock chain for cleaner timeline | ||
when(mockMetaClient.getActiveTimeline()).thenReturn(mockActiveTimeline); | ||
|
||
// Mock the Hadoop configuration to prevent NPE in HudiDataFileExtractor | ||
Configuration hadoopConf = new Configuration(); | ||
hadoopConf.addResource("core-default.xml"); | ||
when(mockMetaClient.getHadoopConf()).thenReturn(hadoopConf); | ||
|
||
// Mock table config to prevent NPE in HudiDataFileExtractor | ||
org.apache.hudi.common.table.HoodieTableConfig mockTableConfig = | ||
mock(org.apache.hudi.common.table.HoodieTableConfig.class); | ||
when(mockMetaClient.getTableConfig()).thenReturn(mockTableConfig); | ||
when(mockTableConfig.isMetadataTableAvailable()).thenReturn(false); | ||
when(mockActiveTimeline.getCleanerTimeline()).thenReturn(mockCleanerTimeline); | ||
when(mockCleanerTimeline.filterCompletedInstants()).thenReturn(mockCleanerTimeline); | ||
when(mockCleanerTimeline.lastInstant()).thenReturn(Option.of(mockCleanInstant)); | ||
when(mockActiveTimeline.deserializeInstantContent(mockCleanInstant, HoodieCleanMetadata.class)) | ||
.thenReturn(mockCleanMetadata); | ||
|
||
// Set up the key behavior: earliestCommitToRetain is null | ||
when(mockCleanMetadata.getEarliestCommitToRetain()).thenReturn(null); | ||
|
||
// Set up mocks for handleEmptyEarliestCommitToRetain - clean instants exist after last sync | ||
when(mockCleanerTimeline.filter(any())).thenReturn(mockFilteredCleanerTimeline); | ||
when(mockFilteredCleanerTimeline.getInstants()) | ||
.thenReturn(Arrays.asList(mockCleanInstantAfterSync)); | ||
|
||
// Set up the mock chain for commit timeline (for doesCommitExistsAsOfInstant) | ||
when(mockActiveTimeline.filterCompletedInstants()).thenReturn(mockCompletedCommitsTimeline); | ||
when(mockCompletedCommitsTimeline.findInstantsBeforeOrEquals(any(String.class))) | ||
.thenReturn(mockCompletedCommitsTimeline); | ||
when(mockCompletedCommitsTimeline.lastInstant()).thenReturn(Option.of(mockCommitInstant)); | ||
|
||
// Create the HudiConversionSource with proper Configuration | ||
Configuration conf = new Configuration(); | ||
conf.addResource("core-default.xml"); | ||
conf.addResource("core-site.xml"); | ||
conf.addResource("hdfs-default.xml"); | ||
conf.addResource("hdfs-site.xml"); | ||
HudiConversionSource hudiConversionSource = | ||
new HudiConversionSource( | ||
mockMetaClient, mock(HudiSourcePartitionSpecExtractor.class), conf, new Properties()); | ||
|
||
// Test that isIncrementalSyncSafeFrom returns false when earliestCommitToRetain is null | ||
// but clean instants exist after last sync | ||
Instant testInstant = Instant.now().minusSeconds(3600); // 1 hour ago | ||
boolean result = hudiConversionSource.isIncrementalSyncSafeFrom(testInstant); | ||
|
||
// This should return false because when earliestCommitToRetain is null and clean instants exist | ||
// after last sync, | ||
// handleEmptyEarliestCommitToRetain returns true, making isAffectedByCleanupProcess return true | ||
assertFalse( | ||
result, | ||
"isIncrementalSyncSafeFrom should return false when earliestCommitToRetain is null but clean instants exist after last sync"); | ||
} | ||
|
||
@Test | ||
void testIsIncrementalSyncSafeFromWithEmptyEarliestCommitToRetainWithCleanInstants() | ||
throws Exception { | ||
// Mock the dependencies | ||
HoodieTableMetaClient mockMetaClient = mock(HoodieTableMetaClient.class); | ||
HoodieActiveTimeline mockActiveTimeline = mock(HoodieActiveTimeline.class); | ||
HoodieTimeline mockCleanerTimeline = mock(HoodieTimeline.class); | ||
HoodieTimeline mockFilteredCleanerTimeline = mock(HoodieTimeline.class); | ||
HoodieTimeline mockCompletedCommitsTimeline = mock(HoodieTimeline.class); | ||
HoodieInstant mockCleanInstant = mock(HoodieInstant.class); | ||
HoodieInstant mockCleanInstantAfterSync = mock(HoodieInstant.class); | ||
HoodieInstant mockCommitInstant = mock(HoodieInstant.class); | ||
HoodieCleanMetadata mockCleanMetadata = mock(HoodieCleanMetadata.class); | ||
|
||
// Set up the mock chain for cleaner timeline | ||
when(mockMetaClient.getActiveTimeline()).thenReturn(mockActiveTimeline); | ||
|
||
// Mock the Hadoop configuration to prevent NPE in HudiDataFileExtractor | ||
Configuration hadoopConf = new Configuration(); | ||
hadoopConf.addResource("core-default.xml"); | ||
when(mockMetaClient.getHadoopConf()).thenReturn(hadoopConf); | ||
|
||
// Mock table config to prevent NPE in HudiDataFileExtractor | ||
org.apache.hudi.common.table.HoodieTableConfig mockTableConfig = | ||
mock(org.apache.hudi.common.table.HoodieTableConfig.class); | ||
when(mockMetaClient.getTableConfig()).thenReturn(mockTableConfig); | ||
when(mockTableConfig.isMetadataTableAvailable()).thenReturn(false); | ||
when(mockActiveTimeline.getCleanerTimeline()).thenReturn(mockCleanerTimeline); | ||
when(mockCleanerTimeline.filterCompletedInstants()).thenReturn(mockCleanerTimeline); | ||
when(mockCleanerTimeline.lastInstant()).thenReturn(Option.of(mockCleanInstant)); | ||
when(mockActiveTimeline.deserializeInstantContent(mockCleanInstant, HoodieCleanMetadata.class)) | ||
.thenReturn(mockCleanMetadata); | ||
|
||
// Set up the key behavior: earliestCommitToRetain is empty string | ||
when(mockCleanMetadata.getEarliestCommitToRetain()).thenReturn(""); | ||
|
||
// Set up mocks for handleEmptyEarliestCommitToRetain - clean instants exist after last sync | ||
when(mockCleanerTimeline.filter(any())).thenReturn(mockFilteredCleanerTimeline); | ||
when(mockFilteredCleanerTimeline.getInstants()) | ||
.thenReturn(Arrays.asList(mockCleanInstantAfterSync)); | ||
|
||
// Set up the mock chain for commit timeline (for doesCommitExistsAsOfInstant) | ||
when(mockActiveTimeline.filterCompletedInstants()).thenReturn(mockCompletedCommitsTimeline); | ||
when(mockCompletedCommitsTimeline.findInstantsBeforeOrEquals(any(String.class))) | ||
.thenReturn(mockCompletedCommitsTimeline); | ||
when(mockCompletedCommitsTimeline.lastInstant()).thenReturn(Option.of(mockCommitInstant)); | ||
|
||
// Create the HudiConversionSource with proper Configuration | ||
Configuration conf = new Configuration(); | ||
conf.addResource("core-default.xml"); | ||
conf.addResource("core-site.xml"); | ||
conf.addResource("hdfs-default.xml"); | ||
conf.addResource("hdfs-site.xml"); | ||
HudiConversionSource hudiConversionSource = | ||
new HudiConversionSource( | ||
mockMetaClient, mock(HudiSourcePartitionSpecExtractor.class), conf, new Properties()); | ||
|
||
// Test that isIncrementalSyncSafeFrom returns false when earliestCommitToRetain is | ||
// empty/whitespace | ||
// and clean instants exist after last sync | ||
Instant testInstant = Instant.now().minusSeconds(3600); // 1 hour ago | ||
boolean result = hudiConversionSource.isIncrementalSyncSafeFrom(testInstant); | ||
|
||
// This should return false because when earliestCommitToRetain is empty/whitespace and clean | ||
// instants exist after last sync, | ||
// handleEmptyEarliestCommitToRetain returns true, making isAffectedByCleanupProcess return true | ||
assertFalse( | ||
result, | ||
"isIncrementalSyncSafeFrom should return false when earliestCommitToRetain is empty/whitespace and clean instants exist after last sync"); | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@alexr17 There are few build failures, GREATER_THAN import is missing. Can you add it?