-
Notifications
You must be signed in to change notification settings - Fork 3k
Add DataLakeFileSystemClient constructor in ADLSFileIO #14966
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
sarthaksin1857
wants to merge
1
commit into
apache:main
Choose a base branch
from
sarthaksin1857:add-adls-supplier-fileIo
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
296 changes: 296 additions & 0 deletions
296
azure/src/test/java/org/apache/iceberg/azure/adlsv2/TestADLSFileIO.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,296 @@ | ||
| /* | ||
| * 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.iceberg.azure.adlsv2; | ||
|
|
||
| import static org.assertj.core.api.Assertions.assertThat; | ||
| import static org.mockito.Mockito.mock; | ||
| import static org.mockito.Mockito.when; | ||
|
|
||
| import com.azure.storage.file.datalake.DataLakeFileClient; | ||
| import com.azure.storage.file.datalake.DataLakeFileSystemClient; | ||
| import java.io.IOException; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
| import org.apache.iceberg.TestHelpers; | ||
| import org.apache.iceberg.io.FileIO; | ||
| import org.apache.iceberg.io.InputFile; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; | ||
| import org.apache.iceberg.util.SerializableFunction; | ||
| import org.junit.jupiter.api.Test; | ||
| import org.junit.jupiter.params.ParameterizedTest; | ||
| import org.junit.jupiter.params.provider.MethodSource; | ||
|
|
||
| public class TestADLSFileIO { | ||
|
|
||
| @Test | ||
| public void testConstructorWithClientSupplier() { | ||
| DataLakeFileSystemClient mockClient = mock(DataLakeFileSystemClient.class); | ||
| SerializableFunction<ADLSLocation, DataLakeFileSystemClient> supplier = location -> mockClient; | ||
|
|
||
| ADLSFileIO fileIO = new ADLSFileIO(supplier); | ||
|
|
||
| // Verify properties are initialized (should not throw NPE) | ||
| assertThat(fileIO.properties()).isNotNull(); | ||
| assertThat(fileIO.properties()).isEmpty(); | ||
| } | ||
|
|
||
| @Test | ||
| public void testConstructorWithClientSupplierAndInitialize() { | ||
| DataLakeFileSystemClient mockClient = mock(DataLakeFileSystemClient.class); | ||
| SerializableFunction<ADLSLocation, DataLakeFileSystemClient> supplier = location -> mockClient; | ||
|
|
||
| ADLSFileIO fileIO = new ADLSFileIO(supplier); | ||
| fileIO.initialize(ImmutableMap.of("key1", "value1")); | ||
|
|
||
| // Verify properties from initialize are used | ||
| assertThat(fileIO.properties()).containsEntry("key1", "value1"); | ||
| } | ||
|
|
||
| @Test | ||
| public void testClientSupplierIsUsed() { | ||
| DataLakeFileSystemClient mockClient = mock(DataLakeFileSystemClient.class); | ||
|
|
||
| SerializableFunction<ADLSLocation, DataLakeFileSystemClient> supplier = location -> mockClient; | ||
|
|
||
| ADLSFileIO fileIO = new ADLSFileIO(supplier); | ||
| fileIO.initialize(ImmutableMap.of()); | ||
|
|
||
| // Call client method to verify supplier is invoked | ||
| DataLakeFileSystemClient client = | ||
| fileIO.client("abfs://container@account.dfs.core.windows.net/path/to/file"); | ||
|
|
||
| assertThat(client).isEqualTo(mockClient); | ||
| } | ||
|
|
||
| @Test | ||
| public void testClientSupplierWithoutInitialize() { | ||
| DataLakeFileSystemClient mockClient = mock(DataLakeFileSystemClient.class); | ||
| DataLakeFileClient mockFileClient = mock(DataLakeFileClient.class); | ||
|
|
||
| when(mockClient.getFileClient("path/to/file")).thenReturn(mockFileClient); | ||
|
|
||
| SerializableFunction<ADLSLocation, DataLakeFileSystemClient> supplier = location -> mockClient; | ||
|
|
||
| ADLSFileIO fileIO = new ADLSFileIO(supplier); | ||
|
|
||
| // Should work without calling initialize() | ||
| // This verifies azureProperties is initialized in constructor | ||
| assertThat(fileIO.properties()).isNotNull(); | ||
| assertThat(fileIO.properties()).isEmpty(); | ||
|
|
||
| // Should be able to create files without NPE | ||
| InputFile inputFile = | ||
| fileIO.newInputFile("abfs://container@account.dfs.core.windows.net/path/to/file"); | ||
| assertThat(inputFile).isNotNull(); | ||
| } | ||
|
|
||
| @Test | ||
| public void testNoArgConstructor() { | ||
| ADLSFileIO fileIO = new ADLSFileIO(); | ||
|
|
||
| // Properties should be null before initialization | ||
| // Initialize with empty map to avoid NPE | ||
| fileIO.initialize(ImmutableMap.of()); | ||
|
|
||
| assertThat(fileIO.properties()).isNotNull(); | ||
| assertThat(fileIO.properties()).isEmpty(); | ||
| } | ||
|
|
||
| @ParameterizedTest | ||
| @MethodSource("org.apache.iceberg.TestHelpers#serializers") | ||
| public void testSerializationWithClientSupplier( | ||
| TestHelpers.RoundTripSerializer<FileIO> roundTripSerializer) | ||
| throws IOException, ClassNotFoundException { | ||
| // Use an AtomicInteger to track supplier invocations across serialization | ||
| AtomicInteger supplierInvocationCount = new AtomicInteger(0); | ||
|
|
||
| SerializableFunction<ADLSLocation, DataLakeFileSystemClient> supplier = | ||
| location -> { | ||
| supplierInvocationCount.incrementAndGet(); | ||
| // Return null - we're only testing serialization, not actual client usage | ||
| return null; | ||
| }; | ||
|
|
||
| ADLSFileIO fileIO = new ADLSFileIO(supplier); | ||
| fileIO.initialize(ImmutableMap.of("key1", "value1", "key2", "value2")); | ||
|
|
||
| // Verify original FileIO works | ||
| assertThat(fileIO.properties()).containsEntry("key1", "value1"); | ||
| assertThat(fileIO.properties()).containsEntry("key2", "value2"); | ||
|
|
||
| // Serialize and deserialize | ||
| FileIO deserializedFileIO = roundTripSerializer.apply(fileIO); | ||
|
|
||
| // Verify properties are preserved after deserialization | ||
| assertThat(deserializedFileIO.properties()).isEqualTo(fileIO.properties()); | ||
|
|
||
| // Verify the deserialized FileIO is an ADLSFileIO and can call client() | ||
| assertThat(deserializedFileIO).isInstanceOf(ADLSFileIO.class); | ||
| ADLSFileIO deserializedADLSFileIO = (ADLSFileIO) deserializedFileIO; | ||
|
|
||
| // Call client() to verify the supplier was serialized and can be invoked | ||
| DataLakeFileSystemClient client = | ||
| deserializedADLSFileIO.client("abfs://container@account.dfs.core.windows.net/path"); | ||
| // The supplier returns null, so client should be null | ||
| assertThat(client).isNull(); | ||
| } | ||
|
|
||
| @ParameterizedTest | ||
| @MethodSource("org.apache.iceberg.TestHelpers#serializers") | ||
| public void testSerializationWithNoArgConstructor( | ||
| TestHelpers.RoundTripSerializer<FileIO> roundTripSerializer) | ||
| throws IOException, ClassNotFoundException { | ||
| ADLSFileIO fileIO = new ADLSFileIO(); | ||
| fileIO.initialize(ImmutableMap.of("key1", "value1", "key2", "value2")); | ||
|
|
||
| // Serialize and deserialize | ||
| FileIO deserializedFileIO = roundTripSerializer.apply(fileIO); | ||
|
|
||
| // Verify properties are preserved after deserialization | ||
| assertThat(deserializedFileIO.properties()).isEqualTo(fileIO.properties()); | ||
| } | ||
|
|
||
| @Test | ||
| public void testClientSupplierIsCachedPerContainer() { | ||
| DataLakeFileSystemClient mockClient1 = mock(DataLakeFileSystemClient.class); | ||
| DataLakeFileSystemClient mockClient2 = mock(DataLakeFileSystemClient.class); | ||
| AtomicInteger supplierInvocationCount = new AtomicInteger(0); | ||
|
|
||
| SerializableFunction<ADLSLocation, DataLakeFileSystemClient> supplier = | ||
| location -> { | ||
| supplierInvocationCount.incrementAndGet(); | ||
| // Return different clients for different containers | ||
| return location.container().orElse("").equals("container1") ? mockClient1 : mockClient2; | ||
| }; | ||
|
|
||
| ADLSFileIO fileIO = new ADLSFileIO(supplier); | ||
|
|
||
| // Same container - should cache | ||
| DataLakeFileSystemClient client1 = | ||
| fileIO.client("abfs://container1@account.dfs.core.windows.net/path1"); | ||
| DataLakeFileSystemClient client2 = | ||
| fileIO.client("abfs://container1@account.dfs.core.windows.net/path2"); | ||
|
|
||
| assertThat(supplierInvocationCount.get()).isEqualTo(1); | ||
| assertThat(client1).isSameAs(client2); | ||
|
|
||
| // Different container - should call supplier again | ||
| DataLakeFileSystemClient client3 = | ||
| fileIO.client("abfs://container2@account.dfs.core.windows.net/path3"); | ||
|
|
||
| assertThat(supplierInvocationCount.get()).isEqualTo(2); | ||
| assertThat(client3).isSameAs(mockClient2); | ||
| } | ||
|
|
||
| @Test | ||
| public void testClientCachedPerStorageAccountAndContainer() { | ||
| DataLakeFileSystemClient mockClient1 = mock(DataLakeFileSystemClient.class); | ||
| DataLakeFileSystemClient mockClient2 = mock(DataLakeFileSystemClient.class); | ||
| DataLakeFileSystemClient mockClient3 = mock(DataLakeFileSystemClient.class); | ||
| AtomicInteger supplierInvocationCount = new AtomicInteger(0); | ||
|
|
||
| SerializableFunction<ADLSLocation, DataLakeFileSystemClient> supplier = | ||
| location -> { | ||
| supplierInvocationCount.incrementAndGet(); | ||
| String host = location.host(); | ||
| String container = location.container().orElse(""); | ||
| if (host.equals("account1.dfs.core.windows.net") && container.equals("container")) { | ||
| return mockClient1; | ||
| } else if (host.equals("account2.dfs.core.windows.net") | ||
| && container.equals("container")) { | ||
| return mockClient2; | ||
| } else { | ||
| return mockClient3; | ||
| } | ||
| }; | ||
|
|
||
| ADLSFileIO fileIO = new ADLSFileIO(supplier); | ||
|
|
||
| // Same account, same container - should cache | ||
| DataLakeFileSystemClient client1 = | ||
| fileIO.client("abfs://container@account1.dfs.core.windows.net/path1"); | ||
| DataLakeFileSystemClient client2 = | ||
| fileIO.client("abfs://container@account1.dfs.core.windows.net/path2"); | ||
|
|
||
| assertThat(supplierInvocationCount.get()).isEqualTo(1); | ||
| assertThat(client1).isSameAs(client2); | ||
| assertThat(client1).isSameAs(mockClient1); | ||
|
|
||
| // Different account, same container - should call supplier again | ||
| DataLakeFileSystemClient client3 = | ||
| fileIO.client("abfs://container@account2.dfs.core.windows.net/path3"); | ||
|
|
||
| assertThat(supplierInvocationCount.get()).isEqualTo(2); | ||
| assertThat(client3).isSameAs(mockClient2); | ||
| assertThat(client3).isNotSameAs(client1); | ||
|
|
||
| // Same account as first, different container - should call supplier again | ||
| DataLakeFileSystemClient client4 = | ||
| fileIO.client("abfs://other@account1.dfs.core.windows.net/path4"); | ||
|
|
||
| assertThat(supplierInvocationCount.get()).isEqualTo(3); | ||
| assertThat(client4).isSameAs(mockClient3); | ||
| } | ||
|
|
||
| @Test | ||
| public void testClientSupplierCachingIsThreadSafe() throws Exception { | ||
| DataLakeFileSystemClient mockClient = mock(DataLakeFileSystemClient.class); | ||
| AtomicInteger supplierInvocationCount = new AtomicInteger(0); | ||
|
|
||
| SerializableFunction<ADLSLocation, DataLakeFileSystemClient> supplier = | ||
| location -> { | ||
| supplierInvocationCount.incrementAndGet(); | ||
| return mockClient; | ||
| }; | ||
|
|
||
| ADLSFileIO fileIO = new ADLSFileIO(supplier); | ||
|
|
||
| // Run multiple threads concurrently calling client() with same container | ||
| int numThreads = 10; | ||
| Thread[] threads = new Thread[numThreads]; | ||
| DataLakeFileSystemClient[] results = new DataLakeFileSystemClient[numThreads]; | ||
|
|
||
| for (int i = 0; i < numThreads; i++) { | ||
| final int index = i; | ||
| threads[i] = | ||
| new Thread( | ||
| () -> { | ||
| results[index] = | ||
| fileIO.client("abfs://container@account.dfs.core.windows.net/path/" + index); | ||
| }); | ||
| } | ||
|
|
||
| // Start all threads | ||
| for (Thread thread : threads) { | ||
| thread.start(); | ||
| } | ||
|
|
||
| // Wait for all threads to complete | ||
| for (Thread thread : threads) { | ||
| thread.join(); | ||
| } | ||
|
|
||
| // Verify supplier was only called once even with concurrent access (same container) | ||
| assertThat(supplierInvocationCount.get()).isEqualTo(1); | ||
|
|
||
| // Verify all threads got the same client | ||
| for (DataLakeFileSystemClient result : results) { | ||
| assertThat(result).isSameAs(mockClient); | ||
| } | ||
| } | ||
| } |
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.
I think this can be created in teh constructor or on line 63 (then it doesn't need to be volatile either). Then we don't need any locking
Uh oh!
There was an error while loading. Please reload this page.
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.
The cache is transient and so it must be lazily constructed. Given that, I don't think there is any way to get around locks. All the other implementations do it this way for the same reason