Skip to content

[HUDI-9376] Archived timeline backwards compatibility #13264

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
wants to merge 4 commits into
base: master
Choose a base branch
from

Conversation

alexr17
Copy link
Contributor

@alexr17 alexr17 commented May 5, 2025

Change Logs

There is some backwards compatibility issues with reading archived timeline instants pre 0.8.

1st issue is that Avro was upgraded in 0.12 to a version which does not allow for default Union to be null. This breaks reading these old instants in HoodieDataBlock.
2nd issue is that it is possible for archived instants to have same timestamp for different actions, especially given that older versions used seconds for instant rather than milliseconds. However the archived timeline does not assume this and assumes each instant has unique timestamp.

Impact

Now the archived timeline stores a map of string (instant) -> string (actiontype) -> map <instant, byte[]>

We also retry failures to getSchemaFromHeader with .setValidateDefaults(false) in case of AvroTypeException

Risk level (write none, low medium or high below)

Low

Documentation Update

None

Contributor's checklist

  • Read through contributor's guide
  • Change Logs and Impact were stated clearly
  • Adequate tests were added if applicable
  • CI passed

@github-actions github-actions bot added the size:M PR with lines of changes in (100, 300] label May 5, 2025
@@ -50,4 +50,12 @@ class TestHiveClientUtils {
assert(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "hive")
assert(HiveClientUtils.getSingletonClientForMetadata(spark) == hiveClient)
}

@AfterAll
Copy link
Contributor

@yihua yihua May 6, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: this change seems unrelated. We can keep it separate if CI still passes.

@@ -371,7 +372,16 @@ protected Option<String> getRecordKey(HoodieRecord record) {

protected Schema getSchemaFromHeader() {
String schemaStr = getLogBlockHeader().get(HeaderMetadataType.SCHEMA);
SCHEMA_MAP.computeIfAbsent(schemaStr, (schemaString) -> new Schema.Parser().parse(schemaString));
SCHEMA_MAP.computeIfAbsent(schemaStr,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we add a unit test at the data log block layer, by writing a schema string to a log block that passes in the old Avro version and fails in the new Avro version, and validate getSchemaFromHeader() that it returns the expected schema?

@@ -58,7 +59,8 @@ public class ArchivedTimelineV1 extends BaseTimelineV1 implements HoodieArchived
private static final String ACTION_STATE = "actionState";
private static final String STATE_TRANSITION_TIME = "stateTransitionTime";
private HoodieTableMetaClient metaClient;
private final Map<String, Map<HoodieInstant.State, byte[]>> readCommits = new HashMap<>();
// The first key is the timestamp -> multiple action types -> hoodie instant state and contents
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does ArchivedTimelineV2 handle the same instant time with multiple action types properly, e.g., during table upgrade from version 6 to 8?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For v2 we introduced ActiveAction to wrap up all the actions of one instant.

Comment on lines +797 to +805
.getResource("/archivecommits/.commits_.archive.681_1-0-1").getPath());

assertDoesNotThrow(() -> readAndValidateArchivedFile(path, metaClient));
}

@Test
void shouldReadArchivedFileFrom2022AndValidateContent() {
Path path = new Path(TestArchivedTimelineV1.class
.getResource("/archivecommits/.commits_.archive.1_1-0-1").getPath());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we generate smaller artifacts of archival files? I also see that they contain specific schema. Let's make sure there is no sensitive information there.

}

@Test
void shouldReadArchivedFileFrom2022AndValidateContent() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Make this Hudi-version specific instead of year?

assertDoesNotThrow(() -> readAndValidateArchivedFile(path, metaClient));
}

void readAndValidateArchivedFile(Path path, HoodieTableMetaClient metaClient) throws IOException {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is .archive_commit_older_schema.data used?

@hudi-bot
Copy link

hudi-bot commented May 7, 2025

CI report:

Bot commands @hudi-bot supports the following commands:
  • @hudi-bot run azure re-run the last Azure build

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
size:M PR with lines of changes in (100, 300]
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants