-
Notifications
You must be signed in to change notification settings - Fork 8
[ENG-38901] Support Hudi table version 9 timeline layout #185
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
Merged
Merged
Changes from all commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
6665a15
[ENG-38901] Support Hudi table version 9 timeline layout
tiennguyen-onehouse 4f284aa
[ENG-38901] Replace string concatenation with text block in HoodiePro…
tiennguyen-onehouse f7d520b
[ENG-38901] Revert text block change - CI uses Java 11
tiennguyen-onehouse 665b6ab
[ENG-38901] Address code review findings for Hudi V9 timeline support
tiennguyen-onehouse 28bbe92
[ENG-38901] Address code review findings round 2
tiennguyen-onehouse b174725
[ENG-38901] Redesign V2 archived timeline upload to be manifest-driven
tiennguyen-onehouse 132d8d8
[ENG-38901] Fix lakeview-sync-tool compile: add LSMTimelineManifestRe…
tiennguyen-onehouse 0b77028
[ENG-38901] Reduce cognitive complexity and fix SonarCloud issues
tiennguyen-onehouse bc8bef5
[ENG-38901] Add LSMTimelineManifestReader unit tests to fix coverage gap
tiennguyen-onehouse 426a91f
[ENG-38901] Address review: propagate manifest read failure, add file…
tiennguyen-onehouse 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
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
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
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
155 changes: 155 additions & 0 deletions
155
lakeview/src/main/java/ai/onehouse/metadata_extractor/LSMTimelineManifestReader.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,155 @@ | ||
| package ai.onehouse.metadata_extractor; | ||
|
|
||
| import static ai.onehouse.constants.MetadataExtractorConstants.MANIFEST_FILE_PREFIX; | ||
| import static ai.onehouse.constants.MetadataExtractorConstants.VERSION_MARKER_FILE; | ||
|
|
||
| import ai.onehouse.RuntimeModule.TableMetadataUploadObjectStorageAsyncClient; | ||
| import ai.onehouse.storage.AsyncStorageClient; | ||
| import ai.onehouse.storage.StorageUtils; | ||
| import com.fasterxml.jackson.databind.JsonNode; | ||
| import com.fasterxml.jackson.databind.ObjectMapper; | ||
| import com.google.inject.Inject; | ||
| import java.io.IOException; | ||
| import java.io.UncheckedIOException; | ||
| import java.nio.charset.StandardCharsets; | ||
| import java.util.ArrayList; | ||
| import java.util.Collections; | ||
| import java.util.List; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import javax.annotation.Nonnull; | ||
| import lombok.Value; | ||
| import lombok.extern.slf4j.Slf4j; | ||
|
|
||
| /** | ||
| * Reads the Hudi LSM (V2) archived timeline manifest layout that lives under {@code | ||
| * .hoodie/timeline/history/}. The layout, written by Hudi 1.x, is: | ||
| * | ||
| * <pre> | ||
| * _version_ # text file containing the latest manifest version | ||
| * manifest_N # JSON: { "files": [{ "fileName": "...", "fileLen": N }] } | ||
| * {minInstant}_{maxInstant}_{level}.parquet | ||
| * </pre> | ||
| * | ||
| * <p>The manifest is the source of truth for which parquet files are valid in the current | ||
| * snapshot. Compaction rewrites multiple low-level files into a single higher-level file, leaving | ||
| * orphaned data files on storage; only files referenced by the latest manifest should be treated | ||
| * as live. LakeView mirrors files to the backend rather than parsing them, so it only needs the | ||
| * filenames the manifest lists - no Hudi dependency required. | ||
| */ | ||
| @Slf4j | ||
| public class LSMTimelineManifestReader { | ||
| private final AsyncStorageClient asyncStorageClient; | ||
| private final StorageUtils storageUtils; | ||
| private final ObjectMapper mapper = new ObjectMapper(); | ||
|
|
||
| @Inject | ||
| public LSMTimelineManifestReader( | ||
| @Nonnull @TableMetadataUploadObjectStorageAsyncClient AsyncStorageClient asyncStorageClient, | ||
| @Nonnull StorageUtils storageUtils) { | ||
| this.asyncStorageClient = asyncStorageClient; | ||
| this.storageUtils = storageUtils; | ||
| } | ||
|
|
||
| /** | ||
| * Reads {@code _version_} and the corresponding {@code manifest_N} from the given history | ||
| * directory. Returns {@link ManifestSnapshot#empty()} when the {@code _version_} file does not | ||
| * exist (no archives have been written yet). | ||
| */ | ||
| public CompletableFuture<ManifestSnapshot> readLatestManifest(String historyDirectoryUri) { | ||
| String versionFileUri = storageUtils.constructFileUri(historyDirectoryUri, VERSION_MARKER_FILE); | ||
| return asyncStorageClient | ||
| .readFileAsBytes(versionFileUri) | ||
| .exceptionally( | ||
| throwable -> { | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Only treat as empty if version itself was missing. |
||
| // Only treat as empty when _version_ itself is missing (no archives written yet). | ||
| log.info( | ||
| "No V2 archived timeline _version_ file at {} (treating as empty). Reason: {}", | ||
| versionFileUri, | ||
| throwable.getMessage()); | ||
| return null; | ||
| }) | ||
| .thenCompose( | ||
| versionBytes -> { | ||
| if (versionBytes == null) { | ||
| return CompletableFuture.completedFuture(ManifestSnapshot.empty()); | ||
| } | ||
| int version = parseVersionFile(versionBytes); | ||
| // Manifest read failures propagate to the caller so the upload is retried | ||
| // on the next sync cycle rather than silently treating a readable _version_ | ||
| // as "no archives." | ||
| return readManifestForVersion(historyDirectoryUri, version) | ||
| .thenApply(files -> ManifestSnapshot.of(version, files)); | ||
| }); | ||
| } | ||
|
|
||
| /** | ||
| * Reads {@code manifest_N} for the given version and returns the parquet filenames it | ||
| * references. Returns an empty list if the manifest cannot be read - callers treat this as | ||
| * "previous snapshot is gone, fall back to bootstrap" rather than a hard failure. | ||
| */ | ||
| public CompletableFuture<List<String>> readManifestFileNames( | ||
| String historyDirectoryUri, int version) { | ||
| return readManifestForVersion(historyDirectoryUri, version) | ||
| .exceptionally( | ||
| throwable -> { | ||
| log.warn( | ||
| "Failed to read manifest_{} from {}; treating as missing. Reason: {}", | ||
| version, | ||
| historyDirectoryUri, | ||
| throwable.getMessage()); | ||
| return Collections.emptyList(); | ||
| }); | ||
| } | ||
|
|
||
| private CompletableFuture<List<String>> readManifestForVersion( | ||
| String historyDirectoryUri, int version) { | ||
| String manifestUri = | ||
| storageUtils.constructFileUri(historyDirectoryUri, MANIFEST_FILE_PREFIX + version); | ||
| return asyncStorageClient | ||
| .readFileAsBytes(manifestUri) | ||
| .thenApply( | ||
| bytes -> { | ||
| try { | ||
| return parseManifestFileNames(bytes); | ||
| } catch (IOException e) { | ||
| throw new UncheckedIOException( | ||
| "Failed to parse LSM manifest at " + manifestUri, e); | ||
| } | ||
| }); | ||
| } | ||
|
|
||
| private static int parseVersionFile(byte[] bytes) { | ||
| String contents = new String(bytes, StandardCharsets.UTF_8).trim(); | ||
| return Integer.parseInt(contents); | ||
| } | ||
|
|
||
| private List<String> parseManifestFileNames(byte[] bytes) throws IOException { | ||
| JsonNode root = mapper.readTree(bytes); | ||
| JsonNode files = root.get("files"); | ||
| List<String> result = new ArrayList<>(); | ||
| if (files != null && files.isArray()) { | ||
| for (JsonNode entry : files) { | ||
| JsonNode name = entry.get("fileName"); | ||
| if (name != null && !name.asText().isEmpty()) { | ||
| result.add(name.asText()); | ||
| } | ||
| } | ||
| } | ||
| return result; | ||
| } | ||
|
|
||
| /** Snapshot of the latest LSM manifest: version number plus the parquet filenames it lists. */ | ||
| @Value(staticConstructor = "of") | ||
| public static class ManifestSnapshot { | ||
| int version; | ||
| @Nonnull List<String> parquetFileNames; | ||
|
|
||
| public static ManifestSnapshot empty() { | ||
| return ManifestSnapshot.of(0, Collections.emptyList()); | ||
| } | ||
|
|
||
| public boolean isEmpty() { | ||
| return version == 0; | ||
| } | ||
| } | ||
| } | ||
Oops, something went wrong.
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.
this is too broad. Can we add ^ and $ at begin and end for each pattern ?
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.
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_PATTERNandV2_MANIFEST_NUMERIC_PATTERN) already have anchors since they're used with.find().