Skip to content

[ENG-38901] Support Hudi table version 9 timeline layout#185

Open
tiennguyen-onehouse wants to merge 6 commits intomainfrom
ENG-38901
Open

[ENG-38901] Support Hudi table version 9 timeline layout#185
tiennguyen-onehouse wants to merge 6 commits intomainfrom
ENG-38901

Conversation

@tiennguyen-onehouse
Copy link
Copy Markdown

Summary

  • Add support for Hudi timeline layout version 2 (table version 9) while maintaining backward compatibility with layout version 1
  • Read hoodie.timeline.layout.version and hoodie.table.version from hoodie.properties and propagate version info through the pipeline
  • Update timeline path construction, regex patterns, timestamp extraction, instant grouping, and archived file parsing to handle both V1 and V2 layouts
  • Add new action types (clustering, logcompaction) used by Hudi 1.0+

Changes

Change Description
1.1 Read tableVersion and timelineLayoutVersion from hoodie.properties into ParsedHudiProperties and Table models
1.2 Update getPathSuffixForTimeline() — V2 uses .hoodie/timeline/ (active) and .hoodie/timeline/history/ (archived)
1.3 Update ACTIVE_COMMIT_INSTANT_PATTERN to match V9 completed instants ({ts}_{completionTs}.action)
1.4 Add ARCHIVED_COMMIT_INSTANT_PATTERN_V2 for parquet files, manifest files, and _version_
1.5 Fix getCommitIdFromActiveTimelineInstant() to strip completion timestamp before parsing
1.6 Fix instant grouping in ActiveTimelineInstantBatcher so V9 completed files group with their requested/inflight files
1.7 Update getNumericPartFromArchivedCommit() to handle V2 parquet, manifest, and _version_ files
1.8 Add clustering and logcompaction to WHITELISTED_ACTION_TYPES
1.9 Make constructStorageUri() and getFileNameWithPrefix() layout-version-aware

ClickUp

https://app.clickup.com/t/86e0k91b2

Test plan

  • Run existing unit tests — all V1 (layout version 1) tests should pass without changes
  • Verify new HoodiePropertiesReaderTest.testReadHoodiePropertiesV9 passes (reads V9 properties)
  • Verify new ActiveTimelineInstantBatcherTest.testCreateBatchWithV9CompletedInstants passes (V9 grouping)
  • Verify TableMetadataUploaderServiceTest passes with new properties reader mocks for existing tables
  • Integration test with a real V9 table in S3 to validate end-to-end timeline extraction

🤖 Generated with Claude Code

@nimahajan
Copy link
Copy Markdown

@tiennguyen-onehouse tiennguyen-onehouse force-pushed the ENG-38901 branch 2 times, most recently from e5e47cc to 81b2a73 Compare March 26, 2026 22:11
@tiennguyen-onehouse
Copy link
Copy Markdown
Author

/push-image ENG-38901-hudi-v9-test

@tiennguyen-onehouse tiennguyen-onehouse force-pushed the ENG-38901 branch 5 times, most recently from 92acc5d to cfc021a Compare March 26, 2026 23:10
@tiennguyen-onehouse tiennguyen-onehouse requested a review from a team as a code owner March 26, 2026 23:10
Add support for Hudi timeline layout version 2 (used by table version 9)
while maintaining backward compatibility with layout version 1.

Changes:
- Read hoodie.timeline.layout.version and hoodie.table.version from
  hoodie.properties and propagate through Table model
- Update active/archived timeline path construction for V2 layout
  (.hoodie/timeline/ and .hoodie/timeline/history/)
- Update regex patterns to match V9 completed instant filenames
  ({timestamp}_{completionTimestamp}.action)
- Fix timestamp extraction and instant grouping for V9 format
- Add V9 archived file parsing (parquet, manifest, _version_)
- Add "clustering" and "logcompaction" action types
- Update path-stripping helpers for V2 layout awareness

Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
@tiennguyen-onehouse
Copy link
Copy Markdown
Author

/push-image ENG-38901

Copy link
Copy Markdown
Author

@tiennguyen-onehouse tiennguyen-onehouse left a comment

Choose a reason for hiding this comment

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

/push-image ENG-38901

tiennguyen-onehouse and others added 3 commits April 2, 2026 13:47
…pertiesReaderTest

Address SonarCloud issue: use Java text block instead of string
concatenation for test properties content.

Co-Authored-By: Claude Opus 4.6 (1M context) <noreply@anthropic.com>
Text blocks require Java 15+. The CI build compiles with Java 11,
so reverting to string concatenation.

Co-Authored-By: Claude Opus 4.6 (1M context) <noreply@anthropic.com>
- Fix V2 archived parquet comparison to use full filename when numeric
  keys match, preventing files with same timestamp but different sequence
  numbers from being skipped on restart
- Add error handling for readHoodieProperties in existing-table path
  to avoid silently falling back to V1 defaults on failure
- Cache ParsedHudiProperties to avoid re-reading on every sync cycle
- Extract TIMELINE_LAYOUT_VERSION_V2 constant replacing magic number 2
- Rename V2_PARQUET_NUMERIC_PATTERN to V2_ARCHIVED_PARQUET_TIMESTAMP_PATTERN
- Use consistent char separator style in path construction
- Remove redundant if/else branch in stubUploadInstantsCallsV2 test

Co-Authored-By: Claude Opus 4.6 (1M context) <noreply@anthropic.com>
Copy link
Copy Markdown
Author

@tiennguyen-onehouse tiennguyen-onehouse left a comment

Choose a reason for hiding this comment

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

/push-image ENG-38884-v9-it

public static final Pattern ARCHIVED_COMMIT_INSTANT_PATTERN =
Pattern.compile("\\.commits_\\.archive\\.\\d+_\\d+-\\d+-\\d+");
public static final Pattern ARCHIVED_COMMIT_INSTANT_PATTERN_V2 =
Pattern.compile("\\d+_\\d+_\\d+\\.parquet|manifest_\\d+|" + VERSION_MARKER_FILE);
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

this is too broad. Can we add ^ and $ at begin and end for each pattern ?

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

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

These patterns are used exclusively with .matches(), which implicitly anchors the match to the entire string (equivalent to ^pattern$). Adding ^ and $ would be redundant. The V2-specific extraction patterns (V2_ARCHIVED_PARQUET_TIMESTAMP_PATTERN and V2_MANIFEST_NUMERIC_PATTERN) already have anchors since they're used with .find().

private final LakeViewExtractorMetrics hudiMetadataExtractorMetrics;
private final ExecutorService executorService;
private final ObjectMapper mapper;
private final Map<String, ParsedHudiProperties> propertiesCache = new ConcurrentHashMap<>();
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

should this be cached forever or add ttl so hudi properties are read periodically ? Ex : If a table is upgraded from v6 to v9 while LakeView is running, the stale cached version will be used indefinitely.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

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

The cache stores a few lightweight entries (one per table). Table versions do not change at runtime — upgrades require a full Hudi migration that would also restart LakeView. Added a comment explaining this.

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Since this map is of all hudi properties but is currently used only to read table version and timeline layout version, it is okay. Suggest updating the comment to indicate that other table properties may not need be migration dependent and so could be stale..

return new BigDecimal(activeTimeLineInstant.split("\\.")[0]);
String timestampPart = activeTimeLineInstant.split("\\.")[0];
if (timestampPart.contains("_")) {
timestampPart = timestampPart.split("_")[0];
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Suggest extracting timestamp into its own utility method. Its repeated 3 times in this file

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

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

The timestamp extraction in getCommitIdFromActiveTimelineInstant is already its own utility method. The split(".", 3) in isSavepointCommit and isRollbackCommit extracts the action type (not the timestamp), but those two methods do duplicate each other. Extracted a shared hasActionType(File, String) helper to consolidate them.

Arrays.asList(
generateFileObj("should_be_ignored", false),
generateFileObj("20260130205837315_20260201000250371.commit", false),
generateFileObj("20260130205837315_20260201000250371.inflight", false),
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

why does inflight and requested have completion time ?

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

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

Good catch. In Hudi V9 (timeline layout V2), pending instants (inflight/requested) only have the requested timestamp — the completion time doesn't exist yet. Fixed the test filenames to use single-timestamp format for pending instants (e.g., 20260130205837315.commit.inflight, 20260130205837315.commit.requested).

- Add comment explaining why propertiesCache has no TTL
- Extract hasActionType() helper to consolidate isSavepointCommit/isRollbackCommit
- Fix V9 test filenames: pending instants use single-timestamp format

Co-Authored-By: Claude Sonnet 4.6 <noreply@anthropic.com>
@sonarqubecloud
Copy link
Copy Markdown

Comment thread lakeview/src/main/java/ai/onehouse/metadata_extractor/HoodiePropertiesReader.java Outdated
Comment thread lakeview/src/main/java/ai/onehouse/metadata_extractor/HoodiePropertiesReader.java Outdated
@nsivabalan
Copy link
Copy Markdown

sorry, I am not fully done yet. somehow clicked on submit. Please hold on to get all my feedback

Copy link
Copy Markdown

@nsivabalan nsivabalan left a comment

Choose a reason for hiding this comment

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

Entire design of archived timeline v2 need to be revisited.

For v1, files are named monotonically.

  .commits_.archive.1
  .commits_.archive.2
  .commits_.archive.3
.

and so, the checkpointing based on file name would suffice.

but in case of Archived timeline V2, its lsm timeline. we could have parquet files from different levels in the history directory.

├── _version_                          # Single file: contains latest manifest version number (e.g., "12")
  ├── manifest_1                         # Manifest snapshot v1 (JSON listing valid parquet files)
  ├── manifest_2                         # Manifest snapshot v2
  ├── ...
  ├── manifest_12                        # Latest manifest snapshot
  ├── 20240101120000000_20240101130000000_0.parquet   # L0 data files
  ├── 20240101130000001_20240101140000000_0.parquet   # L0
  ├── 20240101140000001_20240101150000000_0.parquet   # L0
  ├── ...
  ├── 20240101120000000_20240101200000000_1.parquet   # L1 (compacted from L0 files)
  └── ...

w/o reading the manifest file, we can never know which files to read. And even the checkpointing has to be thought through.
better checkpointing is based on commit time ranges.

lets say below layout
each L0 contains 10 commit intants and there are 6 parquet files.
L1 contains 100 instants in each and contains 10 files.
L2 contains 1000 instants in each and contains 5.

So, in total we have
5000 + 1000 + 60 = 6600 commit intants.

say we wanted to do 2500 per batch.
in round 1: we consume from L2_F1_startTime -> mid(L2_F1_start, L2_F1_end)
in round2: we consume from mid(L2_F1_start, L2_F1_end) -> L2_F1_end
in round3: we consume from L1_F1 -> L1_F10 and all files in L0 as well.

@nsivabalan
Copy link
Copy Markdown

and btw, archived dir location is diff for V2

.hoodie/timeline/history/

@nsivabalan
Copy link
Copy Markdown

nsivabalan commented Apr 14, 2026

let me share a quick write up on LSM timeline and you can come up w/ a design for it.

LSM Timeline in Hudi 1.x

Where it lives

All LSM timeline files live under:
<table_base_path>/.hoodie/timeline/history/
This is the ARCHIVELOG_FOLDER (default: "archived"), accessed via metaClient.getArchivePath().

The LSM Tree Layout

The archived timeline uses an LSM-tree structure with parquet data files, manifest files, and a version file:

.hoodie/archived/
├── version # Single file: contains latest manifest version number (e.g., "12")
├── manifest_1 # Manifest snapshot v1 (JSON listing valid parquet files)
├── manifest_2 # Manifest snapshot v2
├── ...
├── manifest_12 # Latest manifest snapshot
├── 20240101120000000_20240101130000000_0.parquet # L0 data files
├── 20240101130000001_20240101140000000_0.parquet # L0
├── 20240101140000001_20240101150000000_0.parquet # L0
├── ...
├── 20240101120000000_20240101200000000_1.parquet # L1 (compacted from L0 files)
└── ...

File Naming Convention

Data files follow this pattern:
${min_instant}${max_instant}${level}.parquet

┌─────────────┬───────────────────────────────────────────────────────────────────────────────────────────┐
│ Part │ Meaning │
├─────────────┼───────────────────────────────────────────────────────────────────────────────────────────┤
│ min_instant │ Earliest instant timestamp in the file (e.g., 20240101120000000) │
├─────────────┼───────────────────────────────────────────────────────────────────────────────────────────┤
│ max_instant │ Latest instant timestamp in the file (e.g., 20240101130000000) │
├─────────────┼───────────────────────────────────────────────────────────────────────────────────────────┤
│ level │ LSM layer number: 0 = freshly written, 1 = compacted from L0, 2 = compacted from L1, etc. │
└─────────────┴───────────────────────────────────────────────────────────────────────────────────────────┘

Parsed via regex: ^(\d+)(\d+)(\d).parquet in LSMTimeline.java:117

Manifest files: manifest_N where N is a monotonically increasing version number (starts at 1).

Version file: version — single file containing the integer of the latest manifest version.

How New Files Are Created (Archival)

The flow is driven by TimelineArchiverV2.archiveIfRequired():

  1. Determine what to archive: Collects completed commit instants that exceed maxInstantsToKeep on the active timeline, respecting boundaries like earliest
    pending instant, savepoints, compaction on metadata table, etc.
  2. Write to LSM: LSMTimelineWriter.write() takes a batch of ActiveActions and:
    - Creates a new L0 parquet file named {minInstantTime}_{maxInstantTime}_0.parquet
    - Each instant is serialized as a HoodieLSMTimelineInstant Avro record
    - If the file already exists and is committed (in manifest), it skips (idempotency). If corrupt (not in manifest), it deletes and retries.
  3. Update manifest: After the parquet file is written:
    - Reads current manifest version from version
    - Reads the manifest file for the current snapshot's file list
    - Creates a new manifest (manifest_{N+1}) with the new file added
    - Updates version to point to the new manifest
  4. Delete from active timeline: Removes the archived instant files from .hoodie/ (pending first, then completed, in order to avoid confusing concurrent
    readers).
  5. Compact and clean: Triggers compactAndClean() after archival.

Compaction

Uses a universal compaction strategy (LSMTimelineWriter.compactAndClean()):

Trigger: When >= N files (default 10, configured by hoodie.timeline.compaction.batch.size) accumulate at a given layer.

Process:

  1. List all files at layer L0 from the manifest
  2. If count >= batch size, take the oldest N files (sorted by min instant time), respecting a max file size threshold
  3. Read all records from those N files, write them into a single new file at layer L+1
  4. The new file name: {globalMin}{globalMax}{L+1}.parquet — computed from the min/max across all source files
  5. Update the manifest: remove the N source files, add the compacted file
  6. Cascade: Check if L1 now also has >= N files; if so, compact L1 into L2, and so on

Example:
Before compaction (10 files in L0):
t01_t02_0.parquet, t03_t04_0.parquet, ... t19_t20_0.parquet L0

After compaction:
t21_t22_0.parquet (new writes still go to L0) L0
t01_t20_1.parquet (compacted) L1

With 10 instants per L0 file, each L1 file covers ~100 instants, each L2 file ~1000 instants, etc. The benchmark shows reading 1000 instants costs ~10ms.

Cleaning

Triggered after compaction. Retains the latest N manifest snapshot versions (default 3 + number of compacted layers). Older manifests and any data files
not referenced by retained manifests are deleted.

Reader Workflow (Snapshot Isolation)

  1. Read version to get the latest snapshot version
  2. Read manifest_{version} to get the list of valid parquet files
  3. Read the parquet files, using the min/max instant in file names for data skipping (only read files whose time range overlaps the query range)

This gives snapshot isolation — the writer creates new manifests atomically, while readers always see a consistent snapshot via the manifest they read.

Copy link
Copy Markdown
Author

@tiennguyen-onehouse tiennguyen-onehouse left a comment

Choose a reason for hiding this comment

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

/push-image 1.1-Apr14

Rewrites the V2 archived path so LakeView reads _version_ and manifest_N
and mirrors files by diffing successive manifest versions, making the
path compaction-safe. Checkpoints advance by manifest version instead of
filename, so subsequent syncs correctly pick up new archivals after
Hudi's LSM compaction rewrites the file set.

Also addresses round 3 review feedback from nsivabalan:
- Extract hoodie table/timeline version defaults to constants
- Dedupe concurrent hoodie.properties reads via computeIfAbsent
- Skip table instead of silently falling back to v6/v1 on read failure
- Add optional completionTime field to ActiveTimelineInstant for v9

Co-Authored-By: Claude Opus 4.6 (1M context) <noreply@anthropic.com>
Copy link
Copy Markdown
Author

@tiennguyen-onehouse tiennguyen-onehouse left a comment

Choose a reason for hiding this comment

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

Issue addressed:

  1. Filename ordering is meaningless in LSM. Compaction rewrites [A, B, C] at L0 into a single L1 file X whose minInstant may be older than existing files. Sorting by leading timestamp and picking "what comes after the checkpoint" gives the wrong set.
  2. The version tiebreaker was a dead end. With version as lastUploadedFile, every subsequent file appeared "already uploaded." Combined with archivedCommitsProcessed=true, V2 archived became a one-shot bootstrap and never resynced — any instant Hudi archived later was invisible to the backend forever.
  3. Race condition in-between compaction in the archived timeline. We need to checkpoint using latest manifest version and upload only the diff between the last checkpoint.

Solution:

Manifest-driven, checkpointed by manifest version. New LSMTimelineManifestReader reads version and the tiny JSON manifest_N from source.

Each sync cycle:

  1. Read version → currentVersion. If equal to checkpoint.lastArchivedManifestVersion, fast path: nothing to do.
  2. Read manifest_currentVersion and manifest_previousVersion. Diff the parquet lists.
  3. Upload in strict order: parquets → manifest_currentVersion → version. The manifest arrives after every parquet it references, and version
    is the very last write.
  4. One checkpoint upsert at the end, advancing lastArchivedManifestVersion.

Added lastArchivedManifestVersion (int, default 0) to Checkpoint. Jackson deserializes old checkpoints with the field absent using the default —
no migration. V1 path is untouched.

Notes:

The key observation: gateway-controller PR #8797 already switched the backend to TimelineFactory.createArchivedTimeline(...), which reads via
ArchivedTimelineV2 (consults version → manifest_N → parquets). So LakeView doesn't need to understand LSM semantics — it just needs to mirror
files such that the backend can read a consistent snapshot.

A manifest_N file is an immutable snapshot that says "to read the archived timeline at version N, read exactly these parquet files." So the
correctness question reduces to: after a successful sync, is every file in manifest_N on the backend, and does version point to N? That's all
the checkpoint needs to guarantee.

@tiennguyen-onehouse
Copy link
Copy Markdown
Author

Changes in this commit

Review comments addressed

  • HoodiePropertiesReader.java:48,50 — extracted HOODIE_TABLE_VERSION_DEFAULT and TIMELINE_LAYOUT_VERSION_DEFAULT to MetadataExtractorConstants and reused across HoodiePropertiesReader, ParsedHudiProperties, and Table.
  • TableMetadataUploaderService.java:166 — cache is now Map<String, CompletableFuture<ParsedHudiProperties>>, loaded via computeIfAbsent in a new getOrLoadProperties() helper. Concurrent table batches dedupe on in-flight reads. Failed reads evict via whenCompleteAsync on the executor (off the CHM call stack to avoid the recursive-update guard).
  • TableMetadataUploaderService.java:185 — on hoodie.properties read failure, the table is now skipped (future resolves to false) instead of silently falling back to default v6/v1 settings. Cache entry is evicted so the next sync retries.
  • ActiveTimelineInstantBatcher.java:287ActiveTimelineInstant now carries a nullable completionTime field, populated only for V9 instants whose filename embeds requestTime_completionTime.

V2 archived timeline redesign

  • New LSMTimelineManifestReader — reads _version_ and manifest_N JSON from source. No Hudi dependency; Jackson-only parsing.
  • TimelineCommitInstantsUploader.batchUploadWithCheckpoint branches on layout version: V1 archived keeps the existing listAllFilesInDir path, V2 archived goes through the new executeManifestDrivenArchivedUpload.
  • New flow: read current manifest → diff against previous manifest → upload only net-new parquets → upload new manifest → upload _version_ last. Checkpointed by lastArchivedManifestVersion, not filename.
  • Checkpoint extended with lastArchivedManifestVersion (int, default 0). Jackson deserializes old checkpoints with the field absent using the default — no migration needed.
  • Removed dead code: V2 branches in getNumericPartFromArchivedCommit, V2_ARCHIVED_PARQUET_TIMESTAMP_PATTERN, V2_MANIFEST_NUMERIC_PATTERN, and the same-leading-timestamp tiebreaker from round 2.

Tests

Replaced the old testUploadInstantsInArchivedTimelineV2 with four cases covering the new path:

  • _BootstrapMirror — first sync, no previous manifest, mirrors everything in current manifest.
  • _IncrementalAfterCompaction — compaction between runs, verifies only the net-new compacted file gets uploaded.
  • _NoArchivesYet — empty history/, marks archived as processed with zero API calls.
  • _AlreadyAtLatestManifest — fast path when source version matches checkpoint.

Also updated TimelineCommitInstantsUploaderTest and ContinueOnIncompleteCommitStrategyTest constructors for the new LSMTimelineManifestReader dependency.

All 249 lakeview tests pass except testProvidesOkHttpClientWithSysProxy, which was already failing on the clean branch (pre-existing env issue unrelated to this PR).

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants