-
Notifications
You must be signed in to change notification settings - Fork 3.4k
Add support for absolute path in Delta Lake connector #17038
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
ebyhr
merged 1 commit into
trinodb:master
from
vinay-kl:databricks-shollow-cloned-tables-support
Apr 3, 2024
Merged
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
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
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -101,12 +101,45 @@ public class TestDeltaLakeSplitManager | |
| 0); | ||
| private final HiveTransactionHandle transactionHandle = new HiveTransactionHandle(true); | ||
|
|
||
| @Test | ||
| public void testAbsolutePathSplits() | ||
| throws Exception | ||
| { | ||
| testAbsolutePathSplits("file://path/to/file", "file://path/to/file"); | ||
| testAbsolutePathSplits("abfs://[email protected]/path/to/file", "abfs://[email protected]/path/to/file"); | ||
| testAbsolutePathSplits("hdfs://path/to/file", "hdfs://path/to/file"); | ||
| testAbsolutePathSplits("s3://my-s3-bucket/path/to//file", "s3://my-s3-bucket/path/to//file"); | ||
| testAbsolutePathSplits("s3://my-s3-bucket/path/to//file/", "s3://my-s3-bucket/path/to//file/"); | ||
| testAbsolutePathSplits("gs://my-gcp-bucket/path/to/file", "gs://my-gcp-bucket/path/to/file"); | ||
| testAbsolutePathSplits("abfs://[email protected]/+ab+/a%25/a%2525/path/to/file", "abfs://[email protected]/+ab+/a%/a%25/path/to/file"); | ||
| } | ||
|
|
||
| private void testAbsolutePathSplits(String absoluteRawEncodedFilePath, String absoluteDecodedParsedFilePath) | ||
| throws Exception | ||
| { | ||
| long fileSize = 20_000; | ||
| List<AddFileEntry> addFileEntries = ImmutableList.of(addFileEntryOfSize(absoluteRawEncodedFilePath, fileSize)); | ||
| DeltaLakeConfig deltaLakeConfig = new DeltaLakeConfig() | ||
| .setMaxSplitSize(DataSize.ofBytes(5_000)); | ||
| double minimumAssignedSplitWeight = deltaLakeConfig.getMinimumAssignedSplitWeight(); | ||
|
|
||
| DeltaLakeSplitManager splitManager = setupSplitManager(addFileEntries, deltaLakeConfig); | ||
| List<DeltaLakeSplit> splits = getSplits(splitManager, deltaLakeConfig); | ||
| List<DeltaLakeSplit> expected = ImmutableList.of( | ||
| makeSplit(absoluteDecodedParsedFilePath, 0, 5_000, fileSize, minimumAssignedSplitWeight), | ||
| makeSplit(absoluteDecodedParsedFilePath, 5_000, 5_000, fileSize, minimumAssignedSplitWeight), | ||
| makeSplit(absoluteDecodedParsedFilePath, 10_000, 5_000, fileSize, minimumAssignedSplitWeight), | ||
| makeSplit(absoluteDecodedParsedFilePath, 15_000, 5_000, fileSize, minimumAssignedSplitWeight)); | ||
|
|
||
| assertThat(splits).isEqualTo(expected); | ||
| } | ||
|
|
||
| @Test | ||
| public void testSplitSizes() | ||
| throws ExecutionException, InterruptedException | ||
| { | ||
| long fileSize = 50_000; | ||
| List<AddFileEntry> addFileEntries = ImmutableList.of(addFileEntryOfSize(fileSize)); | ||
| List<AddFileEntry> addFileEntries = ImmutableList.of(addFileEntryOfSize(FILE_PATH, fileSize)); | ||
| DeltaLakeConfig deltaLakeConfig = new DeltaLakeConfig() | ||
| .setMaxSplitSize(DataSize.ofBytes(20_000)); | ||
| double minimumAssignedSplitWeight = deltaLakeConfig.getMinimumAssignedSplitWeight(); | ||
|
|
@@ -115,9 +148,9 @@ public void testSplitSizes() | |
| List<DeltaLakeSplit> splits = getSplits(splitManager, deltaLakeConfig); | ||
|
|
||
| List<DeltaLakeSplit> expected = ImmutableList.of( | ||
| makeSplit(0, 20_000, fileSize, minimumAssignedSplitWeight), | ||
| makeSplit(20_000, 20_000, fileSize, minimumAssignedSplitWeight), | ||
| makeSplit(40_000, 10_000, fileSize, minimumAssignedSplitWeight)); | ||
| makeSplit(FULL_PATH, 0, 20_000, fileSize, minimumAssignedSplitWeight), | ||
| makeSplit(FULL_PATH, 20_000, 20_000, fileSize, minimumAssignedSplitWeight), | ||
| makeSplit(FULL_PATH, 40_000, 10_000, fileSize, minimumAssignedSplitWeight)); | ||
|
|
||
| assertThat(splits).isEqualTo(expected); | ||
| } | ||
|
|
@@ -128,7 +161,7 @@ public void testSplitsFromMultipleFiles() | |
| { | ||
| long firstFileSize = 1_000; | ||
| long secondFileSize = 20_000; | ||
| List<AddFileEntry> addFileEntries = ImmutableList.of(addFileEntryOfSize(firstFileSize), addFileEntryOfSize(secondFileSize)); | ||
| List<AddFileEntry> addFileEntries = ImmutableList.of(addFileEntryOfSize(FILE_PATH, firstFileSize), addFileEntryOfSize(FILE_PATH, secondFileSize)); | ||
| DeltaLakeConfig deltaLakeConfig = new DeltaLakeConfig() | ||
| .setMaxSplitSize(DataSize.ofBytes(10_000)); | ||
| double minimumAssignedSplitWeight = deltaLakeConfig.getMinimumAssignedSplitWeight(); | ||
|
|
@@ -137,9 +170,9 @@ public void testSplitsFromMultipleFiles() | |
|
|
||
| List<DeltaLakeSplit> splits = getSplits(splitManager, deltaLakeConfig); | ||
| List<DeltaLakeSplit> expected = ImmutableList.of( | ||
| makeSplit(0, 1_000, firstFileSize, minimumAssignedSplitWeight), | ||
| makeSplit(0, 10_000, secondFileSize, minimumAssignedSplitWeight), | ||
| makeSplit(10_000, 10_000, secondFileSize, minimumAssignedSplitWeight)); | ||
| makeSplit(FULL_PATH, 0, 1_000, firstFileSize, minimumAssignedSplitWeight), | ||
| makeSplit(FULL_PATH, 0, 10_000, secondFileSize, minimumAssignedSplitWeight), | ||
| makeSplit(FULL_PATH, 10_000, 10_000, secondFileSize, minimumAssignedSplitWeight)); | ||
| assertThat(splits).isEqualTo(expected); | ||
| } | ||
|
|
||
|
|
@@ -211,15 +244,15 @@ public Stream<AddFileEntry> getActiveFiles( | |
| new DefaultCachingHostAddressProvider()); | ||
| } | ||
|
|
||
| private AddFileEntry addFileEntryOfSize(long fileSize) | ||
| private AddFileEntry addFileEntryOfSize(String path, long fileSize) | ||
| { | ||
| return new AddFileEntry(FILE_PATH, ImmutableMap.of(), fileSize, 0, false, Optional.empty(), Optional.empty(), ImmutableMap.of(), Optional.empty()); | ||
| return new AddFileEntry(path, ImmutableMap.of(), fileSize, 0, false, Optional.empty(), Optional.empty(), ImmutableMap.of(), Optional.empty()); | ||
| } | ||
|
|
||
| private DeltaLakeSplit makeSplit(long start, long splitSize, long fileSize, double minimumAssignedSplitWeight) | ||
| private DeltaLakeSplit makeSplit(String path, long start, long splitSize, long fileSize, double minimumAssignedSplitWeight) | ||
| { | ||
| SplitWeight splitWeight = SplitWeight.fromProportion(clamp((double) fileSize / splitSize, minimumAssignedSplitWeight, 1.0)); | ||
| return new DeltaLakeSplit(FULL_PATH, start, splitSize, fileSize, Optional.empty(), 0, Optional.empty(), splitWeight, TupleDomain.all(), ImmutableMap.of()); | ||
| return new DeltaLakeSplit(path, start, splitSize, fileSize, Optional.empty(), 0, Optional.empty(), splitWeight, TupleDomain.all(), ImmutableMap.of()); | ||
| } | ||
|
|
||
| private List<DeltaLakeSplit> getSplits(DeltaLakeSplitManager splitManager, DeltaLakeConfig deltaLakeConfig) | ||
|
|
||
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.
Uh oh!
There was an error while loading. Please reload this page.