ChaladiMohanVamsi commented on code in PR #11577:
URL: https://github.com/apache/iceberg/pull/11577#discussion_r1886935941


##########
azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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 com.azure.core.credential.AzureSasCredential;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.commons.io.IOUtils;
+import org.apache.iceberg.azure.AzureProperties;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.rest.ErrorHandlers;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.RESTUtil;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.iceberg.rest.auth.OAuth2Util;
+import org.apache.iceberg.rest.credentials.Credential;
+import org.apache.iceberg.rest.responses.LoadCredentialsResponse;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.SerializableMap;
+import org.apache.iceberg.util.ThreadPools;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class VendedAdlsCredentialProvider implements Serializable, 
AutoCloseable {
+  private static final Logger LOG = 
LoggerFactory.getLogger(VendedAdlsCredentialProvider.class);
+
+  private static final String THREAD_PREFIX = "adls-vended-credential-refresh";
+  public static final String URI = "credentials.uri";
+
+  private final SerializableMap<String, String> properties;
+  private transient volatile Map<String, AzureSasCredentialRefresher>
+      azureSasCredentialRefresherMap;
+  private transient volatile RESTClient client;
+  private transient volatile ScheduledExecutorService refreshExecutor;
+
+  public VendedAdlsCredentialProvider(Map<String, String> properties) {
+    Preconditions.checkArgument(null != properties, "Invalid properties: 
null");
+    Preconditions.checkArgument(null != properties.get(URI), "Invalid URI: 
null");
+    this.properties = SerializableMap.copyOf(properties);
+    azureSasCredentialRefresherMap = Maps.newHashMap();
+  }
+
+  public AzureSasCredential credentialForAccount(String storageAccount) {
+    Map<String, AzureSasCredentialRefresher> refresherForAccountMap =
+        azureSasCredentialRefresherMap();
+    if (refresherForAccountMap.containsKey(storageAccount)) {
+      return refresherForAccountMap.get(storageAccount).azureSasCredential();
+    } else {
+      AzureSasCredentialRefresher azureSasCredentialRefresher =
+          new AzureSasCredentialRefresher(
+              () -> this.sasTokenWithExpiration(storageAccount), 
credentialRefreshExecutor());
+      refresherForAccountMap.put(storageAccount, azureSasCredentialRefresher);
+      return azureSasCredentialRefresher.azureSasCredential();
+    }
+  }
+
+  private Pair<String, Long> sasTokenWithExpiration(String storageAccount) {
+    LoadCredentialsResponse response = fetchCredentials();
+    List<Credential> adlsCredentials =
+        response.credentials().stream()
+            .filter(c -> c.prefix().contains(storageAccount))
+            .collect(Collectors.toList());
+    Preconditions.checkState(
+        !adlsCredentials.isEmpty(),
+        String.format("Invalid ADLS Credentials for storage-account %s: 
empty", storageAccount));
+    Preconditions.checkState(
+        adlsCredentials.size() == 1,
+        "Invalid ADLS Credentials: only one ADLS credential should exist per 
storage-account");
+
+    Credential adlsCredential = adlsCredentials.get(0);
+    checkCredential(adlsCredential, AzureProperties.ADLS_SAS_TOKEN_PREFIX + 
storageAccount);
+    checkCredential(
+        adlsCredential, AzureProperties.ADLS_SAS_TOKEN_EXPIRES_AT_MS_PREFIX + 
storageAccount);
+
+    String updatedSasToken =
+        adlsCredential.config().get(AzureProperties.ADLS_SAS_TOKEN_PREFIX + 
storageAccount);
+    Long tokenExpiresAtMillis =
+        Long.parseLong(
+            adlsCredential
+                .config()
+                .get(AzureProperties.ADLS_SAS_TOKEN_EXPIRES_AT_MS_PREFIX + 
storageAccount));
+
+    return Pair.of(updatedSasToken, tokenExpiresAtMillis);
+  }
+
+  private Map<String, AzureSasCredentialRefresher> 
azureSasCredentialRefresherMap() {
+    if (this.azureSasCredentialRefresherMap == null) {
+      synchronized (this) {
+        if (this.azureSasCredentialRefresherMap == null) {
+          this.azureSasCredentialRefresherMap = Maps.newHashMap();
+        }
+      }
+    }
+    return this.azureSasCredentialRefresherMap;
+  }
+
+  private ScheduledExecutorService credentialRefreshExecutor() {
+    if (this.refreshExecutor == null) {
+      synchronized (this) {
+        if (this.refreshExecutor == null) {
+          this.refreshExecutor = ThreadPools.newScheduledPool(THREAD_PREFIX, 
1);
+        }
+      }
+    }
+    return this.refreshExecutor;
+  }
+
+  private RESTClient httpClient() {
+    if (null == client) {
+      synchronized (this) {
+        if (null == client) {
+          client = 
HTTPClient.builder(properties).uri(properties.get(URI)).build();
+        }
+      }
+    }
+
+    return client;
+  }
+
+  private LoadCredentialsResponse fetchCredentials() {
+    Map<String, String> headers =
+        RESTUtil.merge(
+            configHeaders(properties),

Review Comment:
   Tried to implement the similar behaviour present in 
[RESTSessionCatalog](https://github.com/apache/iceberg/blob/38c8daa4eae8a75ab46571f1efce1609100f53dd/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java#L262)
 for loadTable(), where catalog can be configured to pass explicit headers to 
server by setting the configuration with header. prefix.
   
   While I was testing these changes with vended credentials from loadTable 
response (workaround to mimic new credential endpoint), I saw the possibility 
that REST server implementations may use headers as feature rollout gates/ 
additional metadata to decide credential expiration time. Similarly new 
credential endpoint may also support accepting headers from client as 
additional metadata. 



-- 
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: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to