rymurr commented on code in PR #14966:
URL: https://github.com/apache/iceberg/pull/14966#discussion_r2668453176


##########
azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java:
##########
@@ -70,6 +73,27 @@ public ADLSFileIO() {}
     this.azureProperties = azureProperties;
   }
 
+  /**
+   * Constructor with custom DataLakeFileSystemClient supplier.
+   *
+   * <p>Calling {@link ADLSFileIO#initialize(Map)} will overwrite the 
properties and azureProperties
+   * set in this constructor, but the clientSupplier will continue to be used.
+   *
+   * <p>Unlike the no-arg constructor, this constructor initializes properties 
and azureProperties
+   * immediately, allowing immediate use without calling {@link 
ADLSFileIO#initialize(Map)}.
+   *
+   * <p>Note: The provided client will be used for all file system operations. 
If your use case
+   * requires accessing multiple containers, ensure the client is configured 
appropriately or use
+   * the default constructor with {@link ADLSFileIO#initialize(Map)}.
+   *
+   * @param clientSupplier client supplier
+   */
+  public ADLSFileIO(SerializableSupplier<DataLakeFileSystemClient> 
clientSupplier) {
+    this.clientSupplier = clientSupplier;
+    this.properties = SerializableMap.copyOf(Maps.newHashMap());

Review Comment:
   Why do we do this when it isn't done in teh no-op constructor?



##########
azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java:
##########
@@ -103,22 +127,30 @@ public Map<String, String> properties() {
   }
 
   public DataLakeFileSystemClient client(String path) {
-    ADLSLocation location = new ADLSLocation(path);
-    return client(location);
+    if (clientSupplier != null) {
+      return clientSupplier.get();
+    } else {
+      ADLSLocation location = new ADLSLocation(path);
+      return client(location);
+    }
   }
 
   @VisibleForTesting
   DataLakeFileSystemClient client(ADLSLocation location) {
-    DataLakeFileSystemClientBuilder clientBuilder =
-        new DataLakeFileSystemClientBuilder().httpClient(HTTP);
+    if (clientSupplier != null) {
+      return clientSupplier.get();
+    } else {

Review Comment:
   nit: the else is redundant here and just causes unneccessary whitespace 
changes below



##########
azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java:
##########
@@ -103,22 +127,30 @@ public Map<String, String> properties() {
   }
 
   public DataLakeFileSystemClient client(String path) {
-    ADLSLocation location = new ADLSLocation(path);
-    return client(location);
+    if (clientSupplier != null) {

Review Comment:
   I am concerned about the lack of caching here and below for hte client. In 
AWSs IO we cache the client whereas here we create a new one every time. That 
could be very expensive



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to