From a63232a11fea330485bd20dfdcc4399e3192e941 Mon Sep 17 00:00:00 2001 From: Mohan Vamsi Date: Mon, 18 Nov 2024 14:30:50 +0530 Subject: [PATCH 01/21] Support vended credentials in ADLSFileIO. --- .../apache/iceberg/azure/AzureProperties.java | 32 +- .../adlsv2/AzureSasCredentialRefresher.java | 69 ++++ .../VendedAzureSasCredentialProvider.java | 186 ++++++++++ .../iceberg/azure/AzurePropertiesTest.java | 43 +++ .../VendedAzureSasCredentialProviderTest.java | 323 ++++++++++++++++++ build.gradle | 2 + 6 files changed, 654 insertions(+), 1 deletion(-) create mode 100644 azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java create mode 100644 azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java create mode 100644 azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java diff --git a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java index a7f9885a4726..c56a9a3df2a3 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java +++ b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java @@ -18,31 +18,50 @@ */ package org.apache.iceberg.azure; +import static org.apache.iceberg.azure.adlsv2.VendedAzureSasCredentialProvider.URI; + import com.azure.identity.DefaultAzureCredentialBuilder; import com.azure.storage.common.StorageSharedKeyCredential; import com.azure.storage.file.datalake.DataLakeFileSystemClientBuilder; import java.io.Serializable; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.Optional; +import org.apache.iceberg.azure.adlsv2.VendedAzureSasCredentialProvider; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.PropertyUtil; public class AzureProperties implements Serializable { public static final String ADLS_SAS_TOKEN_PREFIX = "adls.sas-token."; + public static final String ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX = "adls.sas-token-expire-at-ms."; public static final String ADLS_CONNECTION_STRING_PREFIX = "adls.connection-string."; public static final String ADLS_READ_BLOCK_SIZE = "adls.read.block-size-bytes"; public static final String ADLS_WRITE_BLOCK_SIZE = "adls.write.block-size-bytes"; public static final String ADLS_SHARED_KEY_ACCOUNT_NAME = "adls.auth.shared-key.account.name"; public static final String ADLS_SHARED_KEY_ACCOUNT_KEY = "adls.auth.shared-key.account.key"; + /** + * When set, the {@link org.apache.iceberg.azure.adlsv2.VendedAzureSasCredentialProvider} will be + * used to fetch and refresh vended credentials from this endpoint. + */ + public static final String REFRESH_CREDENTIALS_ENDPOINT = "client.refresh-credentials-endpoint"; + + /** Controls whether vended credentials should be refreshed or not. Defaults to true. */ + public static final String REFRESH_CREDENTIALS_ENABLED = "client.refresh-credentials-enabled"; + private Map adlsSasTokens = Collections.emptyMap(); private Map adlsConnectionStrings = Collections.emptyMap(); private Map.Entry namedKeyCreds; private Integer adlsReadBlockSize; private Long adlsWriteBlockSize; + private VendedAzureSasCredentialProvider vendedAzureSasCredentialProvider; + private String refreshCredentialsEndpoint; + private boolean refreshCredentialsEnabled; + public AzureProperties() {} public AzureProperties(Map properties) { @@ -67,6 +86,15 @@ public AzureProperties(Map properties) { if (properties.containsKey(ADLS_WRITE_BLOCK_SIZE)) { this.adlsWriteBlockSize = Long.parseLong(properties.get(ADLS_WRITE_BLOCK_SIZE)); } + this.refreshCredentialsEndpoint = properties.get(REFRESH_CREDENTIALS_ENDPOINT); + this.refreshCredentialsEnabled = + PropertyUtil.propertyAsBoolean(properties, REFRESH_CREDENTIALS_ENABLED, true); + Map credentialProviderProperties = new HashMap<>(properties); + if (refreshCredentialsEnabled && !Strings.isNullOrEmpty(refreshCredentialsEndpoint)) { + credentialProviderProperties.put(URI, refreshCredentialsEndpoint); + this.vendedAzureSasCredentialProvider = + new VendedAzureSasCredentialProvider(credentialProviderProperties); + } } public Optional adlsReadBlockSize() { @@ -90,7 +118,9 @@ public Optional adlsWriteBlockSize() { */ public void applyClientConfiguration(String account, DataLakeFileSystemClientBuilder builder) { String sasToken = adlsSasTokens.get(account); - if (sasToken != null && !sasToken.isEmpty()) { + if (refreshCredentialsEnabled && !Strings.isNullOrEmpty(refreshCredentialsEndpoint)) { + builder.credential(vendedAzureSasCredentialProvider.getCredential(account)); + } else if (sasToken != null && !sasToken.isEmpty()) { builder.sasToken(sasToken); } else if (namedKeyCreds != null) { builder.credential( diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java new file mode 100644 index 000000000000..8a9046b8d6d4 --- /dev/null +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java @@ -0,0 +1,69 @@ +/* + * 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.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; +import org.apache.commons.lang3.tuple.Pair; + +public class AzureSasCredentialRefresher { + private final Supplier> sasTokenWithExpirationSupplier; + private final ScheduledExecutorService refreshExecutor; + + private final AzureSasCredential azureSasCredential; + + private static final long MAX_REFRESH_WINDOW_MILLIS = 300_000; // 5 minutes; + private static final long MIN_REFRESH_WAIT_MILLIS = 10; + + public AzureSasCredentialRefresher( + Supplier> sasTokenWithExpirationSupplier, + ScheduledExecutorService refreshExecutor) { + this.sasTokenWithExpirationSupplier = sasTokenWithExpirationSupplier; + this.refreshExecutor = refreshExecutor; + Pair sasTokenWithExpiration = sasTokenWithExpirationSupplier.get(); + this.azureSasCredential = new AzureSasCredential(sasTokenWithExpiration.getLeft()); + scheduleRefresh(System.currentTimeMillis(), sasTokenWithExpiration.getRight()); + } + + public AzureSasCredential get() { + return this.azureSasCredential; + } + + private void scheduleRefresh(long startTimeMillis, Long expireAtMillis) { + + long expireInMillis = expireAtMillis - startTimeMillis; + long refreshWindowMillis = Math.min(expireInMillis / 10, MAX_REFRESH_WINDOW_MILLIS); + long waitIntervalMillis = expireInMillis - refreshWindowMillis; + long elapsedMillis = System.currentTimeMillis() - startTimeMillis; + long timeToWait = Math.max(waitIntervalMillis - elapsedMillis, MIN_REFRESH_WAIT_MILLIS); + this.refreshExecutor.schedule( + () -> { + long refreshStartTime = System.currentTimeMillis(); + Pair sasTokenWithExpiration = sasTokenWithExpirationSupplier.get(); + azureSasCredential.update(sasTokenWithExpiration.getLeft()); + if (sasTokenWithExpiration.getRight() != null) { + this.scheduleRefresh(refreshStartTime, sasTokenWithExpiration.getRight()); + } + }, + timeToWait, + TimeUnit.MILLISECONDS); + } +} diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java new file mode 100644 index 000000000000..6ebdd923aaf3 --- /dev/null +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java @@ -0,0 +1,186 @@ +/* + * 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.apache.iceberg.azure.AzureProperties.ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX; +import static org.apache.iceberg.azure.AzureProperties.ADLS_SAS_TOKEN_PREFIX; + +import com.azure.core.credential.AzureSasCredential; +import java.io.Serializable; +import java.util.HashMap; +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.commons.lang3.tuple.Pair; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.ErrorHandlers; +import org.apache.iceberg.rest.HTTPClient; +import org.apache.iceberg.rest.RESTClient; +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.SerializableMap; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class VendedAzureSasCredentialProvider implements Serializable, AutoCloseable { + private static final Logger LOG = LoggerFactory.getLogger(VendedAzureSasCredentialProvider.class); + + private final SerializableMap properties; + private transient volatile Map + azureSasCredentialRefresherMap; + private transient volatile RESTClient client; + private transient volatile ScheduledExecutorService refreshExecutor; + + public static final String URI = "credentials.uri"; + private static final String THREAD_PREFIX = "adls-fileio-credential-refresh"; + + public VendedAzureSasCredentialProvider(Map properties) { + Preconditions.checkArgument(null != properties, "Invalid properties: null"); + Preconditions.checkArgument(null != properties.get(URI), "Invalid URI: null"); + this.properties = SerializableMap.copyOf(properties); + azureSasCredentialRefresherMap = new HashMap<>(); + } + + public AzureSasCredential getCredential(String storageAccount) { + Map refresherMap = azureSasCredentialRefresherMap(); + if (refresherMap.containsKey(storageAccount)) { + return refresherMap.get(storageAccount).get(); + } else { + AzureSasCredentialRefresher azureSasCredentialRefresher = + new AzureSasCredentialRefresher( + () -> this.getSasTokenWithExpiration(storageAccount), credentialRefreshExecutor()); + refresherMap.put(storageAccount, azureSasCredentialRefresher); + return azureSasCredentialRefresher.get(); + } + } + + private Pair getSasTokenWithExpiration(String storageAccount) { + LoadCredentialsResponse response = fetchCredentials(); + List 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, ADLS_SAS_TOKEN_PREFIX + storageAccount); + checkCredential(adlsCredential, ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + storageAccount); + + String updatedSasToken = adlsCredential.config().get(ADLS_SAS_TOKEN_PREFIX + storageAccount); + String tokenExpiresAtMillis = + adlsCredential.config().get(ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + storageAccount); + + Long expiresAtMs = Long.parseLong(tokenExpiresAtMillis); + + return Pair.of(updatedSasToken, expiresAtMs); + } + + private Map azureSasCredentialRefresherMap() { + if (this.azureSasCredentialRefresherMap == null) { + synchronized (this) { + if (this.azureSasCredentialRefresherMap == null) { + this.azureSasCredentialRefresherMap = new HashMap<>(); + } + } + } + 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() { + return httpClient() + .get( + properties.get(URI), + null, + LoadCredentialsResponse.class, + OAuth2Util.authHeaders(properties.get(OAuth2Properties.TOKEN)), + ErrorHandlers.defaultErrorHandler()); + } + + private void checkCredential(Credential credential, String property) { + Preconditions.checkState( + credential.config().containsKey(property), + "Invalid ADLS Credentials: %s not set", + property); + } + + @Override + public void close() { + IOUtils.closeQuietly(client); + shutdownRefreshExecutor(); + } + + private void shutdownRefreshExecutor() { + if (refreshExecutor != null) { + ScheduledExecutorService service = refreshExecutor; + this.refreshExecutor = null; + + List tasks = service.shutdownNow(); + tasks.forEach( + task -> { + if (task instanceof Future) { + ((Future) task).cancel(true); + } + }); + + try { + if (!service.awaitTermination(1, TimeUnit.MINUTES)) { + LOG.warn("Timed out waiting for refresh executor to terminate"); + } + } catch (InterruptedException e) { + LOG.warn("Interrupted while waiting for refresh executor to terminate", e); + Thread.currentThread().interrupt(); + } + } + } +} diff --git a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java index 4f032d7ab125..9ca61f550345 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java @@ -24,18 +24,24 @@ import static org.apache.iceberg.azure.AzureProperties.ADLS_SHARED_KEY_ACCOUNT_KEY; import static org.apache.iceberg.azure.AzureProperties.ADLS_SHARED_KEY_ACCOUNT_NAME; import static org.apache.iceberg.azure.AzureProperties.ADLS_WRITE_BLOCK_SIZE; +import static org.apache.iceberg.azure.AzureProperties.REFRESH_CREDENTIALS_ENABLED; +import static org.apache.iceberg.azure.AzureProperties.REFRESH_CREDENTIALS_ENDPOINT; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockConstruction; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import com.azure.core.credential.AzureSasCredential; import com.azure.core.credential.TokenCredential; import com.azure.storage.common.StorageSharedKeyCredential; import com.azure.storage.file.datalake.DataLakeFileSystemClientBuilder; import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.azure.adlsv2.VendedAzureSasCredentialProvider; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; @@ -71,6 +77,43 @@ public void testWithSasToken() { verify(clientBuilder, never()).credential(any(StorageSharedKeyCredential.class)); } + @Test + public void testWithRefreshCredentialsEndpoint() { + try (var providerMockedConstruction = + mockConstruction(VendedAzureSasCredentialProvider.class)) { + AzureProperties props = + new AzureProperties(ImmutableMap.of(REFRESH_CREDENTIALS_ENDPOINT, "endpoint")); + assertThat(providerMockedConstruction.constructed()).hasSize(1); + var providerMock = providerMockedConstruction.constructed().get(0); + AzureSasCredential azureSasCredential = mock(AzureSasCredential.class); + when(providerMock.getCredential("account1")).thenReturn(azureSasCredential); + DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); + props.applyClientConfiguration("account1", clientBuilder); + + verify(clientBuilder, times(1)).credential(azureSasCredential); + verify(clientBuilder, never()).sasToken(any()); + verify(clientBuilder, never()).credential(any(StorageSharedKeyCredential.class)); + } + } + + @Test + public void testWithRefreshCredentialsEndpointDisabled() { + try (var providerMockedConstruction = + mockConstruction(VendedAzureSasCredentialProvider.class)) { + AzureProperties props = + new AzureProperties( + ImmutableMap.of( + REFRESH_CREDENTIALS_ENDPOINT, "endpoint", REFRESH_CREDENTIALS_ENABLED, "false")); + assertThat(providerMockedConstruction.constructed()).hasSize(0); + + DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); + props.applyClientConfiguration("account1", clientBuilder); + verify(clientBuilder, times(0)).sasToken(any()); + verify(clientBuilder).credential(any(TokenCredential.class)); + verify(clientBuilder, never()).credential(any(StorageSharedKeyCredential.class)); + } + } + @Test public void testNoMatchingSasToken() { AzureProperties props = diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java new file mode 100644 index 000000000000..7711349ccde4 --- /dev/null +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java @@ -0,0 +1,323 @@ +/* + * 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.apache.iceberg.azure.AzureProperties.ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX; +import static org.apache.iceberg.azure.AzureProperties.ADLS_SAS_TOKEN_PREFIX; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockserver.integration.ClientAndServer.startClientAndServer; +import static org.mockserver.model.HttpRequest.request; +import static org.mockserver.model.HttpResponse.response; + +import com.azure.core.credential.AzureSasCredential; +import com.azure.core.http.HttpMethod; +import java.io.IOException; +import java.time.Instant; +import java.time.temporal.ChronoUnit; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.exceptions.RESTException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.credentials.Credential; +import org.apache.iceberg.rest.credentials.ImmutableCredential; +import org.apache.iceberg.rest.responses.ImmutableLoadCredentialsResponse; +import org.apache.iceberg.rest.responses.LoadCredentialsResponse; +import org.apache.iceberg.rest.responses.LoadCredentialsResponseParser; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockserver.integration.ClientAndServer; +import org.mockserver.model.HttpRequest; +import org.mockserver.model.HttpResponse; +import org.mockserver.verify.VerificationTimes; + +public class VendedAzureSasCredentialProviderTest { + private static final int PORT = 3232; + private static final String URI = String.format("http://127.0.0.1:%d/v1/credentials", PORT); + private static ClientAndServer mockServer; + private static final String STORAGE_ACCOUNT = "account1"; + private static final String CREDENTIAL_PREFIX = + "abfs://container@account1.dfs.core.windows.net/dir"; + private static final String STORAGE_ACCOUNT_2 = "account2"; + private static final String CREDENTIAL_PREFIX_2 = + "abfs://container@account2.dfs.core.windows.net/dir"; + + @BeforeAll + public static void beforeAll() { + mockServer = startClientAndServer(PORT); + } + + @AfterAll + public static void stopServer() { + mockServer.stop(); + } + + @BeforeEach + public void before() { + mockServer.reset(); + } + + @Test + public void invalidOrMissingUri() { + assertThatThrownBy(() -> new VendedAzureSasCredentialProvider(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid properties: null"); + assertThatThrownBy(() -> new VendedAzureSasCredentialProvider(ImmutableMap.of())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid URI: null"); + + try (VendedAzureSasCredentialProvider provider = + new VendedAzureSasCredentialProvider( + ImmutableMap.of(VendedAzureSasCredentialProvider.URI, "invalid uri"))) { + assertThatThrownBy(() -> provider.getCredential(STORAGE_ACCOUNT)) + .isInstanceOf(RESTException.class) + .hasMessageStartingWith("Failed to create request URI from base invalid uri"); + } + } + + @Test + public void noADLSCredentials() { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + + HttpResponse mockResponse = + response( + LoadCredentialsResponseParser.toJson( + ImmutableLoadCredentialsResponse.builder().build())) + .withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedAzureSasCredentialProvider provider = + new VendedAzureSasCredentialProvider( + ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { + assertThatThrownBy(() -> provider.getCredential(STORAGE_ACCOUNT)) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid ADLS Credentials for storage-account account1: empty"); + } + } + + @Test + public void expirationNotSet() { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder() + .addCredentials( + ImmutableCredential.builder() + .prefix(CREDENTIAL_PREFIX) + .config( + ImmutableMap.of(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT, "randomSasToken")) + .build()) + .build(); + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedAzureSasCredentialProvider provider = + new VendedAzureSasCredentialProvider( + ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { + assertThatThrownBy(() -> provider.getCredential(STORAGE_ACCOUNT)) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid ADLS Credentials: adls.sas-token-expire-at-ms.account1 not set"); + } + } + + @Test + public void nonExpiredSasToken() { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + Credential credential = + ImmutableCredential.builder() + .prefix(CREDENTIAL_PREFIX) + .config( + ImmutableMap.of( + ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT, + "randomSasToken", + ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + STORAGE_ACCOUNT, + Long.toString(Instant.now().plus(1, ChronoUnit.HOURS).toEpochMilli()))) + .build(); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder().addCredentials(credential).build(); + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedAzureSasCredentialProvider provider = + new VendedAzureSasCredentialProvider( + ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { + AzureSasCredential azureSasCredential = provider.getCredential(STORAGE_ACCOUNT); + assertThat(azureSasCredential.getSignature()) + .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); + + for (int i = 0; i < 5; i++) { + // resolving credentials multiple times should not hit the credentials endpoint again + assertThat(provider.getCredential(STORAGE_ACCOUNT)).isSameAs(azureSasCredential); + } + } + mockServer.verify(mockRequest, VerificationTimes.once()); + } + + @Test + public void expiredSasToken() throws InterruptedException { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + Credential credential = + ImmutableCredential.builder() + .prefix(CREDENTIAL_PREFIX) + .config( + ImmutableMap.of( + ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT, + "randomSasToken", + ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + STORAGE_ACCOUNT, + Long.toString(Instant.now().minus(1, ChronoUnit.MINUTES).toEpochMilli()))) + .build(); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder().addCredentials(credential).build(); + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedAzureSasCredentialProvider provider = + new VendedAzureSasCredentialProvider( + ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { + AzureSasCredential azureSasCredential = provider.getCredential(STORAGE_ACCOUNT); + assertThat(azureSasCredential.getSignature()) + .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); + + Thread.sleep(10); + // Since expiration time past to current time, the refresh will fall back at minimum 10ms + // interval + } + mockServer.verify(mockRequest, VerificationTimes.atLeast(2)); + } + + @Test + public void multipleADLSCredentialsPerStorageAccount() { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + Credential credential1 = + ImmutableCredential.builder() + .prefix(CREDENTIAL_PREFIX) + .config( + ImmutableMap.of( + ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT, + "randomSasToken1", + ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + STORAGE_ACCOUNT, + Long.toString(Instant.now().plus(1, ChronoUnit.HOURS).toEpochMilli()))) + .build(); + Credential credential2 = + ImmutableCredential.builder() + .prefix(CREDENTIAL_PREFIX + "/dir2") + .config( + ImmutableMap.of( + ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT, + "randomSasToken2", + ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + STORAGE_ACCOUNT, + Long.toString(Instant.now().plus(1, ChronoUnit.HOURS).toEpochMilli()))) + .build(); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder().addCredentials(credential1, credential2).build(); + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedAzureSasCredentialProvider provider = + new VendedAzureSasCredentialProvider( + ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { + assertThatThrownBy(() -> provider.getCredential(STORAGE_ACCOUNT)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid ADLS Credentials: only one ADLS credential should exist per storage-account"); + } + } + + @Test + public void multipleStorageAccounts() { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + Credential credential1 = + ImmutableCredential.builder() + .prefix(CREDENTIAL_PREFIX) + .config( + ImmutableMap.of( + ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT, + "randomSasToken1", + ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + STORAGE_ACCOUNT, + Long.toString(Instant.now().plus(1, ChronoUnit.HOURS).toEpochMilli()))) + .build(); + Credential credential2 = + ImmutableCredential.builder() + .prefix(CREDENTIAL_PREFIX_2) + .config( + ImmutableMap.of( + ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT_2, + "randomSasToken2", + ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + STORAGE_ACCOUNT_2, + Long.toString(Instant.now().plus(1, ChronoUnit.HOURS).toEpochMilli()))) + .build(); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder().addCredentials(credential1, credential2).build(); + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedAzureSasCredentialProvider provider = + new VendedAzureSasCredentialProvider( + ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { + AzureSasCredential azureSasCredential1 = provider.getCredential(STORAGE_ACCOUNT); + AzureSasCredential azureSasCredential2 = provider.getCredential(STORAGE_ACCOUNT_2); + assertThat(azureSasCredential1).isNotSameAs(azureSasCredential2); + assertThat(azureSasCredential1.getSignature()) + .isEqualTo(credential1.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); + assertThat(azureSasCredential2.getSignature()) + .isEqualTo(credential2.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT_2)); + } + } + + @Test + public void serializableTest() throws IOException, ClassNotFoundException { + HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); + Credential credential = + ImmutableCredential.builder() + .prefix(CREDENTIAL_PREFIX) + .config( + ImmutableMap.of( + ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT, + "randomSasToken", + ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + STORAGE_ACCOUNT, + Long.toString(Instant.now().plus(1, ChronoUnit.HOURS).toEpochMilli()))) + .build(); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder().addCredentials(credential).build(); + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + try (VendedAzureSasCredentialProvider provider = + new VendedAzureSasCredentialProvider( + ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { + AzureSasCredential azureSasCredential = provider.getCredential(STORAGE_ACCOUNT); + assertThat(azureSasCredential.getSignature()) + .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); + + VendedAzureSasCredentialProvider deserializedProvider = + TestHelpers.roundTripSerialize(provider); + AzureSasCredential reGeneratedAzureSasCredential = + deserializedProvider.getCredential(STORAGE_ACCOUNT); + + assertThat(azureSasCredential).isNotSameAs(reGeneratedAzureSasCredential); + } + mockServer.verify(mockRequest, VerificationTimes.exactly(2)); + } +} diff --git a/build.gradle b/build.gradle index 81daf14a357f..57ac0ecb92b7 100644 --- a/build.gradle +++ b/build.gradle @@ -546,6 +546,8 @@ project(':iceberg-azure') { testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') testImplementation libs.esotericsoftware.kryo testImplementation libs.testcontainers + testImplementation libs.mockserver.netty + testImplementation libs.mockserver.client.java } } From 7cb7e72b0d58cc60619e2e466f7b6100c3884aa4 Mon Sep 17 00:00:00 2001 From: Mohan Vamsi Date: Mon, 18 Nov 2024 17:16:44 +0530 Subject: [PATCH 02/21] Fix checkstyle violations. --- .../apache/iceberg/azure/AzureProperties.java | 8 +++---- .../VendedAzureSasCredentialProvider.java | 22 ++++++++++--------- .../VendedAzureSasCredentialProviderTest.java | 2 +- 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java index c56a9a3df2a3..f56e9b121d25 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java +++ b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java @@ -18,14 +18,11 @@ */ package org.apache.iceberg.azure; -import static org.apache.iceberg.azure.adlsv2.VendedAzureSasCredentialProvider.URI; - import com.azure.identity.DefaultAzureCredentialBuilder; import com.azure.storage.common.StorageSharedKeyCredential; import com.azure.storage.file.datalake.DataLakeFileSystemClientBuilder; import java.io.Serializable; import java.util.Collections; -import java.util.HashMap; import java.util.Map; import java.util.Optional; import org.apache.iceberg.azure.adlsv2.VendedAzureSasCredentialProvider; @@ -89,9 +86,10 @@ public AzureProperties(Map properties) { this.refreshCredentialsEndpoint = properties.get(REFRESH_CREDENTIALS_ENDPOINT); this.refreshCredentialsEnabled = PropertyUtil.propertyAsBoolean(properties, REFRESH_CREDENTIALS_ENABLED, true); - Map credentialProviderProperties = new HashMap<>(properties); + Map credentialProviderProperties = Maps.newHashMap(properties); if (refreshCredentialsEnabled && !Strings.isNullOrEmpty(refreshCredentialsEndpoint)) { - credentialProviderProperties.put(URI, refreshCredentialsEndpoint); + credentialProviderProperties.put( + VendedAzureSasCredentialProvider.URI, refreshCredentialsEndpoint); this.vendedAzureSasCredentialProvider = new VendedAzureSasCredentialProvider(credentialProviderProperties); } diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java index 6ebdd923aaf3..2ce770d319e1 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java @@ -18,12 +18,8 @@ */ package org.apache.iceberg.azure.adlsv2; -import static org.apache.iceberg.azure.AzureProperties.ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX; -import static org.apache.iceberg.azure.AzureProperties.ADLS_SAS_TOKEN_PREFIX; - import com.azure.core.credential.AzureSasCredential; import java.io.Serializable; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.Future; @@ -32,7 +28,9 @@ import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.tuple.Pair; +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; @@ -61,7 +59,7 @@ public VendedAzureSasCredentialProvider(Map properties) { Preconditions.checkArgument(null != properties, "Invalid properties: null"); Preconditions.checkArgument(null != properties.get(URI), "Invalid URI: null"); this.properties = SerializableMap.copyOf(properties); - azureSasCredentialRefresherMap = new HashMap<>(); + azureSasCredentialRefresherMap = Maps.newHashMap(); } public AzureSasCredential getCredential(String storageAccount) { @@ -91,12 +89,16 @@ private Pair getSasTokenWithExpiration(String storageAccount) { "Invalid ADLS Credentials: only one ADLS credential should exist per storage-account"); Credential adlsCredential = adlsCredentials.get(0); - checkCredential(adlsCredential, ADLS_SAS_TOKEN_PREFIX + storageAccount); - checkCredential(adlsCredential, ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + storageAccount); + checkCredential(adlsCredential, AzureProperties.ADLS_SAS_TOKEN_PREFIX + storageAccount); + checkCredential( + adlsCredential, AzureProperties.ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + storageAccount); - String updatedSasToken = adlsCredential.config().get(ADLS_SAS_TOKEN_PREFIX + storageAccount); + String updatedSasToken = + adlsCredential.config().get(AzureProperties.ADLS_SAS_TOKEN_PREFIX + storageAccount); String tokenExpiresAtMillis = - adlsCredential.config().get(ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + storageAccount); + adlsCredential + .config() + .get(AzureProperties.ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + storageAccount); Long expiresAtMs = Long.parseLong(tokenExpiresAtMillis); @@ -107,7 +109,7 @@ private Map azureSasCredentialRefresherMap( if (this.azureSasCredentialRefresherMap == null) { synchronized (this) { if (this.azureSasCredentialRefresherMap == null) { - this.azureSasCredentialRefresherMap = new HashMap<>(); + this.azureSasCredentialRefresherMap = Maps.newHashMap(); } } } diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java index 7711349ccde4..df6e7d6d1e00 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java @@ -197,7 +197,7 @@ public void expiredSasToken() throws InterruptedException { assertThat(azureSasCredential.getSignature()) .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); - Thread.sleep(10); + Thread.sleep(20); // Since expiration time past to current time, the refresh will fall back at minimum 10ms // interval } From e19f7e87a3b103b8f64993dd9e403e39e06bedcd Mon Sep 17 00:00:00 2001 From: Mohan Vamsi Date: Mon, 18 Nov 2024 21:53:33 +0530 Subject: [PATCH 03/21] Allow passing additional headers from properties while refreshing credentials. --- .../adlsv2/VendedAzureSasCredentialProvider.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java index 2ce770d319e1..82a7cae2d842 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java @@ -34,6 +34,7 @@ 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; @@ -140,15 +141,23 @@ private RESTClient httpClient() { } private LoadCredentialsResponse fetchCredentials() { + Map headers = + RESTUtil.merge( + configHeaders(properties), + OAuth2Util.authHeaders(properties.get(OAuth2Properties.TOKEN))); return httpClient() .get( properties.get(URI), null, LoadCredentialsResponse.class, - OAuth2Util.authHeaders(properties.get(OAuth2Properties.TOKEN)), + headers, ErrorHandlers.defaultErrorHandler()); } + private Map configHeaders(Map props) { + return RESTUtil.extractPrefixMap(props, "header."); + } + private void checkCredential(Credential credential, String property) { Preconditions.checkState( credential.config().containsKey(property), From 4b869c50e745d4533f5dd6eebbb66cce58570319 Mon Sep 17 00:00:00 2001 From: Mohan Vamsi Date: Thu, 28 Nov 2024 23:46:02 +0530 Subject: [PATCH 04/21] refactor as per iceberg coding practices. --- .../apache/iceberg/azure/AzureProperties.java | 2 +- .../adlsv2/AzureSasCredentialRefresher.java | 28 ++++++++------- .../VendedAzureSasCredentialProvider.java | 36 +++++++++---------- .../iceberg/azure/AzurePropertiesTest.java | 2 +- .../VendedAzureSasCredentialProviderTest.java | 22 ++++++------ 5 files changed, 46 insertions(+), 44 deletions(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java index f56e9b121d25..bcf41f3d06c1 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java +++ b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java @@ -117,7 +117,7 @@ public Optional adlsWriteBlockSize() { public void applyClientConfiguration(String account, DataLakeFileSystemClientBuilder builder) { String sasToken = adlsSasTokens.get(account); if (refreshCredentialsEnabled && !Strings.isNullOrEmpty(refreshCredentialsEndpoint)) { - builder.credential(vendedAzureSasCredentialProvider.getCredential(account)); + builder.credential(vendedAzureSasCredentialProvider.credentialForAccount(account)); } else if (sasToken != null && !sasToken.isEmpty()) { builder.sasToken(sasToken); } else if (namedKeyCreds != null) { diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java index 8a9046b8d6d4..5a9f9d145be1 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java @@ -27,7 +27,6 @@ public class AzureSasCredentialRefresher { private final Supplier> sasTokenWithExpirationSupplier; private final ScheduledExecutorService refreshExecutor; - private final AzureSasCredential azureSasCredential; private static final long MAX_REFRESH_WINDOW_MILLIS = 300_000; // 5 minutes; @@ -40,30 +39,33 @@ public AzureSasCredentialRefresher( this.refreshExecutor = refreshExecutor; Pair sasTokenWithExpiration = sasTokenWithExpirationSupplier.get(); this.azureSasCredential = new AzureSasCredential(sasTokenWithExpiration.getLeft()); - scheduleRefresh(System.currentTimeMillis(), sasTokenWithExpiration.getRight()); + scheduleRefresh(sasTokenWithExpiration.getRight()); } - public AzureSasCredential get() { + public AzureSasCredential azureSasCredential() { return this.azureSasCredential; } - private void scheduleRefresh(long startTimeMillis, Long expireAtMillis) { - - long expireInMillis = expireAtMillis - startTimeMillis; - long refreshWindowMillis = Math.min(expireInMillis / 10, MAX_REFRESH_WINDOW_MILLIS); - long waitIntervalMillis = expireInMillis - refreshWindowMillis; - long elapsedMillis = System.currentTimeMillis() - startTimeMillis; - long timeToWait = Math.max(waitIntervalMillis - elapsedMillis, MIN_REFRESH_WAIT_MILLIS); + private void scheduleRefresh(Long expireAtMillis) { this.refreshExecutor.schedule( () -> { - long refreshStartTime = System.currentTimeMillis(); Pair sasTokenWithExpiration = sasTokenWithExpirationSupplier.get(); azureSasCredential.update(sasTokenWithExpiration.getLeft()); if (sasTokenWithExpiration.getRight() != null) { - this.scheduleRefresh(refreshStartTime, sasTokenWithExpiration.getRight()); + this.scheduleRefresh(sasTokenWithExpiration.getRight()); } }, - timeToWait, + refreshDelay(expireAtMillis), TimeUnit.MILLISECONDS); } + + private long refreshDelay(Long expireAtMillis) { + long expireInMillis = expireAtMillis - System.currentTimeMillis(); + // how much ahead of time to start the request to allow it to complete + long refreshWindowMillis = Math.min(expireInMillis / 10, MAX_REFRESH_WINDOW_MILLIS); + // how much time to wait before expiration + long waitIntervalMillis = expireInMillis - refreshWindowMillis; + // how much time to actually wait + return Math.max(waitIntervalMillis, MIN_REFRESH_WAIT_MILLIS); + } } diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java index 82a7cae2d842..9457f6378315 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java @@ -47,15 +47,15 @@ public class VendedAzureSasCredentialProvider implements Serializable, AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(VendedAzureSasCredentialProvider.class); + private static final String THREAD_PREFIX = "adls-fileio-credential-refresh"; + public static final String URI = "credentials.uri"; + private final SerializableMap properties; private transient volatile Map azureSasCredentialRefresherMap; private transient volatile RESTClient client; private transient volatile ScheduledExecutorService refreshExecutor; - public static final String URI = "credentials.uri"; - private static final String THREAD_PREFIX = "adls-fileio-credential-refresh"; - public VendedAzureSasCredentialProvider(Map properties) { Preconditions.checkArgument(null != properties, "Invalid properties: null"); Preconditions.checkArgument(null != properties.get(URI), "Invalid URI: null"); @@ -63,20 +63,21 @@ public VendedAzureSasCredentialProvider(Map properties) { azureSasCredentialRefresherMap = Maps.newHashMap(); } - public AzureSasCredential getCredential(String storageAccount) { - Map refresherMap = azureSasCredentialRefresherMap(); - if (refresherMap.containsKey(storageAccount)) { - return refresherMap.get(storageAccount).get(); + public AzureSasCredential credentialForAccount(String storageAccount) { + Map refresherForAccountMap = + azureSasCredentialRefresherMap(); + if (refresherForAccountMap.containsKey(storageAccount)) { + return refresherForAccountMap.get(storageAccount).azureSasCredential(); } else { AzureSasCredentialRefresher azureSasCredentialRefresher = new AzureSasCredentialRefresher( - () -> this.getSasTokenWithExpiration(storageAccount), credentialRefreshExecutor()); - refresherMap.put(storageAccount, azureSasCredentialRefresher); - return azureSasCredentialRefresher.get(); + () -> this.sasTokenWithExpiration(storageAccount), credentialRefreshExecutor()); + refresherForAccountMap.put(storageAccount, azureSasCredentialRefresher); + return azureSasCredentialRefresher.azureSasCredential(); } } - private Pair getSasTokenWithExpiration(String storageAccount) { + private Pair sasTokenWithExpiration(String storageAccount) { LoadCredentialsResponse response = fetchCredentials(); List adlsCredentials = response.credentials().stream() @@ -96,14 +97,13 @@ private Pair getSasTokenWithExpiration(String storageAccount) { String updatedSasToken = adlsCredential.config().get(AzureProperties.ADLS_SAS_TOKEN_PREFIX + storageAccount); - String tokenExpiresAtMillis = - adlsCredential - .config() - .get(AzureProperties.ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + storageAccount); - - Long expiresAtMs = Long.parseLong(tokenExpiresAtMillis); + Long tokenExpiresAtMillis = + Long.parseLong( + adlsCredential + .config() + .get(AzureProperties.ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + storageAccount)); - return Pair.of(updatedSasToken, expiresAtMs); + return Pair.of(updatedSasToken, tokenExpiresAtMillis); } private Map azureSasCredentialRefresherMap() { diff --git a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java index 9ca61f550345..3ae491c3e650 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java @@ -86,7 +86,7 @@ public void testWithRefreshCredentialsEndpoint() { assertThat(providerMockedConstruction.constructed()).hasSize(1); var providerMock = providerMockedConstruction.constructed().get(0); AzureSasCredential azureSasCredential = mock(AzureSasCredential.class); - when(providerMock.getCredential("account1")).thenReturn(azureSasCredential); + when(providerMock.credentialForAccount("account1")).thenReturn(azureSasCredential); DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); props.applyClientConfiguration("account1", clientBuilder); diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java index df6e7d6d1e00..9a2bda23c6e6 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java @@ -86,7 +86,7 @@ public void invalidOrMissingUri() { try (VendedAzureSasCredentialProvider provider = new VendedAzureSasCredentialProvider( ImmutableMap.of(VendedAzureSasCredentialProvider.URI, "invalid uri"))) { - assertThatThrownBy(() -> provider.getCredential(STORAGE_ACCOUNT)) + assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) .isInstanceOf(RESTException.class) .hasMessageStartingWith("Failed to create request URI from base invalid uri"); } @@ -106,7 +106,7 @@ public void noADLSCredentials() { try (VendedAzureSasCredentialProvider provider = new VendedAzureSasCredentialProvider( ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { - assertThatThrownBy(() -> provider.getCredential(STORAGE_ACCOUNT)) + assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) .isInstanceOf(IllegalStateException.class) .hasMessage("Invalid ADLS Credentials for storage-account account1: empty"); } @@ -131,7 +131,7 @@ public void expirationNotSet() { try (VendedAzureSasCredentialProvider provider = new VendedAzureSasCredentialProvider( ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { - assertThatThrownBy(() -> provider.getCredential(STORAGE_ACCOUNT)) + assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) .isInstanceOf(IllegalStateException.class) .hasMessage("Invalid ADLS Credentials: adls.sas-token-expire-at-ms.account1 not set"); } @@ -159,13 +159,13 @@ public void nonExpiredSasToken() { try (VendedAzureSasCredentialProvider provider = new VendedAzureSasCredentialProvider( ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { - AzureSasCredential azureSasCredential = provider.getCredential(STORAGE_ACCOUNT); + AzureSasCredential azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); assertThat(azureSasCredential.getSignature()) .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); for (int i = 0; i < 5; i++) { // resolving credentials multiple times should not hit the credentials endpoint again - assertThat(provider.getCredential(STORAGE_ACCOUNT)).isSameAs(azureSasCredential); + assertThat(provider.credentialForAccount(STORAGE_ACCOUNT)).isSameAs(azureSasCredential); } } mockServer.verify(mockRequest, VerificationTimes.once()); @@ -193,7 +193,7 @@ public void expiredSasToken() throws InterruptedException { try (VendedAzureSasCredentialProvider provider = new VendedAzureSasCredentialProvider( ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { - AzureSasCredential azureSasCredential = provider.getCredential(STORAGE_ACCOUNT); + AzureSasCredential azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); assertThat(azureSasCredential.getSignature()) .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); @@ -236,7 +236,7 @@ public void multipleADLSCredentialsPerStorageAccount() { try (VendedAzureSasCredentialProvider provider = new VendedAzureSasCredentialProvider( ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { - assertThatThrownBy(() -> provider.getCredential(STORAGE_ACCOUNT)) + assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) .isInstanceOf(IllegalStateException.class) .hasMessage( "Invalid ADLS Credentials: only one ADLS credential should exist per storage-account"); @@ -275,8 +275,8 @@ public void multipleStorageAccounts() { try (VendedAzureSasCredentialProvider provider = new VendedAzureSasCredentialProvider( ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { - AzureSasCredential azureSasCredential1 = provider.getCredential(STORAGE_ACCOUNT); - AzureSasCredential azureSasCredential2 = provider.getCredential(STORAGE_ACCOUNT_2); + AzureSasCredential azureSasCredential1 = provider.credentialForAccount(STORAGE_ACCOUNT); + AzureSasCredential azureSasCredential2 = provider.credentialForAccount(STORAGE_ACCOUNT_2); assertThat(azureSasCredential1).isNotSameAs(azureSasCredential2); assertThat(azureSasCredential1.getSignature()) .isEqualTo(credential1.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); @@ -307,14 +307,14 @@ public void serializableTest() throws IOException, ClassNotFoundException { try (VendedAzureSasCredentialProvider provider = new VendedAzureSasCredentialProvider( ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { - AzureSasCredential azureSasCredential = provider.getCredential(STORAGE_ACCOUNT); + AzureSasCredential azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); assertThat(azureSasCredential.getSignature()) .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); VendedAzureSasCredentialProvider deserializedProvider = TestHelpers.roundTripSerialize(provider); AzureSasCredential reGeneratedAzureSasCredential = - deserializedProvider.getCredential(STORAGE_ACCOUNT); + deserializedProvider.credentialForAccount(STORAGE_ACCOUNT); assertThat(azureSasCredential).isNotSameAs(reGeneratedAzureSasCredential); } From 6b0f859bba4ac9d62c03d5ef04fafd49bfbe21e2 Mon Sep 17 00:00:00 2001 From: ChaladiMohanVamsi Date: Fri, 6 Dec 2024 13:53:28 +0530 Subject: [PATCH 05/21] Apply suggestions from code review Co-authored-by: Eduard Tudenhoefner --- .../java/org/apache/iceberg/azure/AzureProperties.java | 10 +++++----- .../azure/adlsv2/AzureSasCredentialRefresher.java | 2 +- .../azure/adlsv2/VendedAzureSasCredentialProvider.java | 4 ++-- .../adlsv2/VendedAzureSasCredentialProviderTest.java | 2 +- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java index bcf41f3d06c1..267983605718 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java +++ b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java @@ -33,7 +33,7 @@ public class AzureProperties implements Serializable { public static final String ADLS_SAS_TOKEN_PREFIX = "adls.sas-token."; - public static final String ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX = "adls.sas-token-expire-at-ms."; + public static final String ADLS_SAS_TOKEN_EXPIRES_AT_MS_PREFIX = "adls.sas-token-expires-at-ms."; public static final String ADLS_CONNECTION_STRING_PREFIX = "adls.connection-string."; public static final String ADLS_READ_BLOCK_SIZE = "adls.read.block-size-bytes"; public static final String ADLS_WRITE_BLOCK_SIZE = "adls.write.block-size-bytes"; @@ -44,10 +44,10 @@ public class AzureProperties implements Serializable { * When set, the {@link org.apache.iceberg.azure.adlsv2.VendedAzureSasCredentialProvider} will be * used to fetch and refresh vended credentials from this endpoint. */ - public static final String REFRESH_CREDENTIALS_ENDPOINT = "client.refresh-credentials-endpoint"; + public static final String ADLS_REFRESH_CREDENTIALS_ENDPOINT = "adls.refresh-credentials-endpoint"; /** Controls whether vended credentials should be refreshed or not. Defaults to true. */ - public static final String REFRESH_CREDENTIALS_ENABLED = "client.refresh-credentials-enabled"; + public static final String ADLS_REFRESH_CREDENTIALS_ENABLED = "adls.refresh-credentials-enabled"; private Map adlsSasTokens = Collections.emptyMap(); private Map adlsConnectionStrings = Collections.emptyMap(); @@ -56,8 +56,8 @@ public class AzureProperties implements Serializable { private Long adlsWriteBlockSize; private VendedAzureSasCredentialProvider vendedAzureSasCredentialProvider; - private String refreshCredentialsEndpoint; - private boolean refreshCredentialsEnabled; + private String adlsRefreshCredentialsEndpoint; + private boolean adlsRefreshCredentialsEnabled; public AzureProperties() {} diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java index 5a9f9d145be1..d110fee2115a 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java @@ -59,7 +59,7 @@ private void scheduleRefresh(Long expireAtMillis) { TimeUnit.MILLISECONDS); } - private long refreshDelay(Long expireAtMillis) { + private long refreshDelayMillis(Long expireAtMillis) { long expireInMillis = expireAtMillis - System.currentTimeMillis(); // how much ahead of time to start the request to allow it to complete long refreshWindowMillis = Math.min(expireInMillis / 10, MAX_REFRESH_WINDOW_MILLIS); diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java index 9457f6378315..fc16d4d2d96a 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java @@ -44,10 +44,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class VendedAzureSasCredentialProvider implements Serializable, AutoCloseable { +public class VendedAdlsCredentialProvider implements Serializable, AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(VendedAzureSasCredentialProvider.class); - private static final String THREAD_PREFIX = "adls-fileio-credential-refresh"; + private static final String THREAD_PREFIX = "adls-vended-credential-refresh"; public static final String URI = "credentials.uri"; private final SerializableMap properties; diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java index 9a2bda23c6e6..89ffb22914aa 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java @@ -48,7 +48,7 @@ import org.mockserver.model.HttpResponse; import org.mockserver.verify.VerificationTimes; -public class VendedAzureSasCredentialProviderTest { +public class VendedAdlsCredentialProviderTest { private static final int PORT = 3232; private static final String URI = String.format("http://127.0.0.1:%d/v1/credentials", PORT); private static ClientAndServer mockServer; From 54112b9738985846d374593ca09fdd9564d88365 Mon Sep 17 00:00:00 2001 From: Mohan Vamsi Date: Fri, 6 Dec 2024 15:12:35 +0530 Subject: [PATCH 06/21] Apply suggestions from code review. --- .../apache/iceberg/azure/AzureProperties.java | 31 ++++----- .../adlsv2/AzureSasCredentialRefresher.java | 6 +- ...java => VendedAdlsCredentialProvider.java} | 8 +-- .../iceberg/azure/AzurePropertiesTest.java | 19 +++--- ... => VendedAdlsCredentialProviderTest.java} | 66 ++++++++----------- 5 files changed, 62 insertions(+), 68 deletions(-) rename azure/src/main/java/org/apache/iceberg/azure/adlsv2/{VendedAzureSasCredentialProvider.java => VendedAdlsCredentialProvider.java} (96%) rename azure/src/test/java/org/apache/iceberg/azure/adlsv2/{VendedAzureSasCredentialProviderTest.java => VendedAdlsCredentialProviderTest.java} (84%) diff --git a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java index 267983605718..04982ce3df8c 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java +++ b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java @@ -25,7 +25,7 @@ import java.util.Collections; import java.util.Map; import java.util.Optional; -import org.apache.iceberg.azure.adlsv2.VendedAzureSasCredentialProvider; +import org.apache.iceberg.azure.adlsv2.VendedAdlsCredentialProvider; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -41,10 +41,11 @@ public class AzureProperties implements Serializable { public static final String ADLS_SHARED_KEY_ACCOUNT_KEY = "adls.auth.shared-key.account.key"; /** - * When set, the {@link org.apache.iceberg.azure.adlsv2.VendedAzureSasCredentialProvider} will be - * used to fetch and refresh vended credentials from this endpoint. + * When set, the {@link VendedAdlsCredentialProvider} will be used to fetch and refresh vended + * credentials from this endpoint. */ - public static final String ADLS_REFRESH_CREDENTIALS_ENDPOINT = "adls.refresh-credentials-endpoint"; + public static final String ADLS_REFRESH_CREDENTIALS_ENDPOINT = + "adls.refresh-credentials-endpoint"; /** Controls whether vended credentials should be refreshed or not. Defaults to true. */ public static final String ADLS_REFRESH_CREDENTIALS_ENABLED = "adls.refresh-credentials-enabled"; @@ -55,7 +56,7 @@ public class AzureProperties implements Serializable { private Integer adlsReadBlockSize; private Long adlsWriteBlockSize; - private VendedAzureSasCredentialProvider vendedAzureSasCredentialProvider; + private VendedAdlsCredentialProvider vendedAdlsCredentialProvider; private String adlsRefreshCredentialsEndpoint; private boolean adlsRefreshCredentialsEnabled; @@ -83,15 +84,15 @@ public AzureProperties(Map properties) { if (properties.containsKey(ADLS_WRITE_BLOCK_SIZE)) { this.adlsWriteBlockSize = Long.parseLong(properties.get(ADLS_WRITE_BLOCK_SIZE)); } - this.refreshCredentialsEndpoint = properties.get(REFRESH_CREDENTIALS_ENDPOINT); - this.refreshCredentialsEnabled = - PropertyUtil.propertyAsBoolean(properties, REFRESH_CREDENTIALS_ENABLED, true); - Map credentialProviderProperties = Maps.newHashMap(properties); - if (refreshCredentialsEnabled && !Strings.isNullOrEmpty(refreshCredentialsEndpoint)) { + this.adlsRefreshCredentialsEndpoint = properties.get(ADLS_REFRESH_CREDENTIALS_ENDPOINT); + this.adlsRefreshCredentialsEnabled = + PropertyUtil.propertyAsBoolean(properties, ADLS_REFRESH_CREDENTIALS_ENABLED, true); + if (adlsRefreshCredentialsEnabled && !Strings.isNullOrEmpty(adlsRefreshCredentialsEndpoint)) { + Map credentialProviderProperties = Maps.newHashMap(properties); credentialProviderProperties.put( - VendedAzureSasCredentialProvider.URI, refreshCredentialsEndpoint); - this.vendedAzureSasCredentialProvider = - new VendedAzureSasCredentialProvider(credentialProviderProperties); + VendedAdlsCredentialProvider.URI, adlsRefreshCredentialsEndpoint); + this.vendedAdlsCredentialProvider = + new VendedAdlsCredentialProvider(credentialProviderProperties); } } @@ -116,8 +117,8 @@ public Optional adlsWriteBlockSize() { */ public void applyClientConfiguration(String account, DataLakeFileSystemClientBuilder builder) { String sasToken = adlsSasTokens.get(account); - if (refreshCredentialsEnabled && !Strings.isNullOrEmpty(refreshCredentialsEndpoint)) { - builder.credential(vendedAzureSasCredentialProvider.credentialForAccount(account)); + if (adlsRefreshCredentialsEnabled && !Strings.isNullOrEmpty(adlsRefreshCredentialsEndpoint)) { + builder.credential(vendedAdlsCredentialProvider.credentialForAccount(account)); } else if (sasToken != null && !sasToken.isEmpty()) { builder.sasToken(sasToken); } else if (namedKeyCreds != null) { diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java index d110fee2115a..0a3e1b932568 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java @@ -24,7 +24,7 @@ import java.util.function.Supplier; import org.apache.commons.lang3.tuple.Pair; -public class AzureSasCredentialRefresher { +class AzureSasCredentialRefresher { private final Supplier> sasTokenWithExpirationSupplier; private final ScheduledExecutorService refreshExecutor; private final AzureSasCredential azureSasCredential; @@ -32,7 +32,7 @@ public class AzureSasCredentialRefresher { private static final long MAX_REFRESH_WINDOW_MILLIS = 300_000; // 5 minutes; private static final long MIN_REFRESH_WAIT_MILLIS = 10; - public AzureSasCredentialRefresher( + AzureSasCredentialRefresher( Supplier> sasTokenWithExpirationSupplier, ScheduledExecutorService refreshExecutor) { this.sasTokenWithExpirationSupplier = sasTokenWithExpirationSupplier; @@ -55,7 +55,7 @@ private void scheduleRefresh(Long expireAtMillis) { this.scheduleRefresh(sasTokenWithExpiration.getRight()); } }, - refreshDelay(expireAtMillis), + refreshDelayMillis(expireAtMillis), TimeUnit.MILLISECONDS); } diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java similarity index 96% rename from azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java rename to azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java index fc16d4d2d96a..89d7e0709f75 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProvider.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java @@ -45,7 +45,7 @@ import org.slf4j.LoggerFactory; public class VendedAdlsCredentialProvider implements Serializable, AutoCloseable { - private static final Logger LOG = LoggerFactory.getLogger(VendedAzureSasCredentialProvider.class); + 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"; @@ -56,7 +56,7 @@ public class VendedAdlsCredentialProvider implements Serializable, AutoCloseable private transient volatile RESTClient client; private transient volatile ScheduledExecutorService refreshExecutor; - public VendedAzureSasCredentialProvider(Map properties) { + public VendedAdlsCredentialProvider(Map properties) { Preconditions.checkArgument(null != properties, "Invalid properties: null"); Preconditions.checkArgument(null != properties.get(URI), "Invalid URI: null"); this.properties = SerializableMap.copyOf(properties); @@ -93,7 +93,7 @@ private Pair sasTokenWithExpiration(String storageAccount) { Credential adlsCredential = adlsCredentials.get(0); checkCredential(adlsCredential, AzureProperties.ADLS_SAS_TOKEN_PREFIX + storageAccount); checkCredential( - adlsCredential, AzureProperties.ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + storageAccount); + adlsCredential, AzureProperties.ADLS_SAS_TOKEN_EXPIRES_AT_MS_PREFIX + storageAccount); String updatedSasToken = adlsCredential.config().get(AzureProperties.ADLS_SAS_TOKEN_PREFIX + storageAccount); @@ -101,7 +101,7 @@ private Pair sasTokenWithExpiration(String storageAccount) { Long.parseLong( adlsCredential .config() - .get(AzureProperties.ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + storageAccount)); + .get(AzureProperties.ADLS_SAS_TOKEN_EXPIRES_AT_MS_PREFIX + storageAccount)); return Pair.of(updatedSasToken, tokenExpiresAtMillis); } diff --git a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java index 3ae491c3e650..f3b977d2f5bc 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java @@ -20,12 +20,12 @@ import static org.apache.iceberg.azure.AzureProperties.ADLS_CONNECTION_STRING_PREFIX; import static org.apache.iceberg.azure.AzureProperties.ADLS_READ_BLOCK_SIZE; +import static org.apache.iceberg.azure.AzureProperties.ADLS_REFRESH_CREDENTIALS_ENABLED; +import static org.apache.iceberg.azure.AzureProperties.ADLS_REFRESH_CREDENTIALS_ENDPOINT; import static org.apache.iceberg.azure.AzureProperties.ADLS_SAS_TOKEN_PREFIX; import static org.apache.iceberg.azure.AzureProperties.ADLS_SHARED_KEY_ACCOUNT_KEY; import static org.apache.iceberg.azure.AzureProperties.ADLS_SHARED_KEY_ACCOUNT_NAME; import static org.apache.iceberg.azure.AzureProperties.ADLS_WRITE_BLOCK_SIZE; -import static org.apache.iceberg.azure.AzureProperties.REFRESH_CREDENTIALS_ENABLED; -import static org.apache.iceberg.azure.AzureProperties.REFRESH_CREDENTIALS_ENDPOINT; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; import static org.mockito.ArgumentMatchers.any; @@ -41,7 +41,7 @@ import com.azure.storage.common.StorageSharedKeyCredential; import com.azure.storage.file.datalake.DataLakeFileSystemClientBuilder; import org.apache.iceberg.TestHelpers; -import org.apache.iceberg.azure.adlsv2.VendedAzureSasCredentialProvider; +import org.apache.iceberg.azure.adlsv2.VendedAdlsCredentialProvider; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; @@ -79,10 +79,9 @@ public void testWithSasToken() { @Test public void testWithRefreshCredentialsEndpoint() { - try (var providerMockedConstruction = - mockConstruction(VendedAzureSasCredentialProvider.class)) { + try (var providerMockedConstruction = mockConstruction(VendedAdlsCredentialProvider.class)) { AzureProperties props = - new AzureProperties(ImmutableMap.of(REFRESH_CREDENTIALS_ENDPOINT, "endpoint")); + new AzureProperties(ImmutableMap.of(ADLS_REFRESH_CREDENTIALS_ENDPOINT, "endpoint")); assertThat(providerMockedConstruction.constructed()).hasSize(1); var providerMock = providerMockedConstruction.constructed().get(0); AzureSasCredential azureSasCredential = mock(AzureSasCredential.class); @@ -98,12 +97,14 @@ public void testWithRefreshCredentialsEndpoint() { @Test public void testWithRefreshCredentialsEndpointDisabled() { - try (var providerMockedConstruction = - mockConstruction(VendedAzureSasCredentialProvider.class)) { + try (var providerMockedConstruction = mockConstruction(VendedAdlsCredentialProvider.class)) { AzureProperties props = new AzureProperties( ImmutableMap.of( - REFRESH_CREDENTIALS_ENDPOINT, "endpoint", REFRESH_CREDENTIALS_ENABLED, "false")); + ADLS_REFRESH_CREDENTIALS_ENDPOINT, + "endpoint", + ADLS_REFRESH_CREDENTIALS_ENABLED, + "false")); assertThat(providerMockedConstruction.constructed()).hasSize(0); DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java similarity index 84% rename from azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java rename to azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java index 89ffb22914aa..0ddfecb41af2 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialProviderTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.azure.adlsv2; -import static org.apache.iceberg.azure.AzureProperties.ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX; +import static org.apache.iceberg.azure.AzureProperties.ADLS_SAS_TOKEN_EXPIRES_AT_MS_PREFIX; import static org.apache.iceberg.azure.AzureProperties.ADLS_SAS_TOKEN_PREFIX; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -76,16 +76,16 @@ public void before() { @Test public void invalidOrMissingUri() { - assertThatThrownBy(() -> new VendedAzureSasCredentialProvider(null)) + assertThatThrownBy(() -> new VendedAdlsCredentialProvider(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid properties: null"); - assertThatThrownBy(() -> new VendedAzureSasCredentialProvider(ImmutableMap.of())) + assertThatThrownBy(() -> new VendedAdlsCredentialProvider(ImmutableMap.of())) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid URI: null"); - try (VendedAzureSasCredentialProvider provider = - new VendedAzureSasCredentialProvider( - ImmutableMap.of(VendedAzureSasCredentialProvider.URI, "invalid uri"))) { + try (VendedAdlsCredentialProvider provider = + new VendedAdlsCredentialProvider( + ImmutableMap.of(VendedAdlsCredentialProvider.URI, "invalid uri"))) { assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) .isInstanceOf(RESTException.class) .hasMessageStartingWith("Failed to create request URI from base invalid uri"); @@ -103,9 +103,8 @@ public void noADLSCredentials() { .withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAzureSasCredentialProvider provider = - new VendedAzureSasCredentialProvider( - ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { + try (VendedAdlsCredentialProvider provider = + new VendedAdlsCredentialProvider(ImmutableMap.of(VendedAdlsCredentialProvider.URI, URI))) { assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) .isInstanceOf(IllegalStateException.class) .hasMessage("Invalid ADLS Credentials for storage-account account1: empty"); @@ -128,12 +127,11 @@ public void expirationNotSet() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAzureSasCredentialProvider provider = - new VendedAzureSasCredentialProvider( - ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { + try (VendedAdlsCredentialProvider provider = + new VendedAdlsCredentialProvider(ImmutableMap.of(VendedAdlsCredentialProvider.URI, URI))) { assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) .isInstanceOf(IllegalStateException.class) - .hasMessage("Invalid ADLS Credentials: adls.sas-token-expire-at-ms.account1 not set"); + .hasMessage("Invalid ADLS Credentials: adls.sas-token-expires-at-ms.account1 not set"); } } @@ -147,7 +145,7 @@ public void nonExpiredSasToken() { ImmutableMap.of( ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT, "randomSasToken", - ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + STORAGE_ACCOUNT, + ADLS_SAS_TOKEN_EXPIRES_AT_MS_PREFIX + STORAGE_ACCOUNT, Long.toString(Instant.now().plus(1, ChronoUnit.HOURS).toEpochMilli()))) .build(); LoadCredentialsResponse response = @@ -156,9 +154,8 @@ public void nonExpiredSasToken() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAzureSasCredentialProvider provider = - new VendedAzureSasCredentialProvider( - ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { + try (VendedAdlsCredentialProvider provider = + new VendedAdlsCredentialProvider(ImmutableMap.of(VendedAdlsCredentialProvider.URI, URI))) { AzureSasCredential azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); assertThat(azureSasCredential.getSignature()) .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); @@ -181,7 +178,7 @@ public void expiredSasToken() throws InterruptedException { ImmutableMap.of( ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT, "randomSasToken", - ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + STORAGE_ACCOUNT, + ADLS_SAS_TOKEN_EXPIRES_AT_MS_PREFIX + STORAGE_ACCOUNT, Long.toString(Instant.now().minus(1, ChronoUnit.MINUTES).toEpochMilli()))) .build(); LoadCredentialsResponse response = @@ -190,9 +187,8 @@ public void expiredSasToken() throws InterruptedException { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAzureSasCredentialProvider provider = - new VendedAzureSasCredentialProvider( - ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { + try (VendedAdlsCredentialProvider provider = + new VendedAdlsCredentialProvider(ImmutableMap.of(VendedAdlsCredentialProvider.URI, URI))) { AzureSasCredential azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); assertThat(azureSasCredential.getSignature()) .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); @@ -214,7 +210,7 @@ public void multipleADLSCredentialsPerStorageAccount() { ImmutableMap.of( ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT, "randomSasToken1", - ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + STORAGE_ACCOUNT, + ADLS_SAS_TOKEN_EXPIRES_AT_MS_PREFIX + STORAGE_ACCOUNT, Long.toString(Instant.now().plus(1, ChronoUnit.HOURS).toEpochMilli()))) .build(); Credential credential2 = @@ -224,7 +220,7 @@ public void multipleADLSCredentialsPerStorageAccount() { ImmutableMap.of( ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT, "randomSasToken2", - ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + STORAGE_ACCOUNT, + ADLS_SAS_TOKEN_EXPIRES_AT_MS_PREFIX + STORAGE_ACCOUNT, Long.toString(Instant.now().plus(1, ChronoUnit.HOURS).toEpochMilli()))) .build(); LoadCredentialsResponse response = @@ -233,9 +229,8 @@ public void multipleADLSCredentialsPerStorageAccount() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAzureSasCredentialProvider provider = - new VendedAzureSasCredentialProvider( - ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { + try (VendedAdlsCredentialProvider provider = + new VendedAdlsCredentialProvider(ImmutableMap.of(VendedAdlsCredentialProvider.URI, URI))) { assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) .isInstanceOf(IllegalStateException.class) .hasMessage( @@ -253,7 +248,7 @@ public void multipleStorageAccounts() { ImmutableMap.of( ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT, "randomSasToken1", - ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + STORAGE_ACCOUNT, + ADLS_SAS_TOKEN_EXPIRES_AT_MS_PREFIX + STORAGE_ACCOUNT, Long.toString(Instant.now().plus(1, ChronoUnit.HOURS).toEpochMilli()))) .build(); Credential credential2 = @@ -263,7 +258,7 @@ public void multipleStorageAccounts() { ImmutableMap.of( ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT_2, "randomSasToken2", - ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + STORAGE_ACCOUNT_2, + ADLS_SAS_TOKEN_EXPIRES_AT_MS_PREFIX + STORAGE_ACCOUNT_2, Long.toString(Instant.now().plus(1, ChronoUnit.HOURS).toEpochMilli()))) .build(); LoadCredentialsResponse response = @@ -272,9 +267,8 @@ public void multipleStorageAccounts() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAzureSasCredentialProvider provider = - new VendedAzureSasCredentialProvider( - ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { + try (VendedAdlsCredentialProvider provider = + new VendedAdlsCredentialProvider(ImmutableMap.of(VendedAdlsCredentialProvider.URI, URI))) { AzureSasCredential azureSasCredential1 = provider.credentialForAccount(STORAGE_ACCOUNT); AzureSasCredential azureSasCredential2 = provider.credentialForAccount(STORAGE_ACCOUNT_2); assertThat(azureSasCredential1).isNotSameAs(azureSasCredential2); @@ -295,7 +289,7 @@ public void serializableTest() throws IOException, ClassNotFoundException { ImmutableMap.of( ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT, "randomSasToken", - ADLS_SAS_TOKEN_EXPIRE_AT_MS_PREFIX + STORAGE_ACCOUNT, + ADLS_SAS_TOKEN_EXPIRES_AT_MS_PREFIX + STORAGE_ACCOUNT, Long.toString(Instant.now().plus(1, ChronoUnit.HOURS).toEpochMilli()))) .build(); LoadCredentialsResponse response = @@ -304,15 +298,13 @@ public void serializableTest() throws IOException, ClassNotFoundException { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAzureSasCredentialProvider provider = - new VendedAzureSasCredentialProvider( - ImmutableMap.of(VendedAzureSasCredentialProvider.URI, URI))) { + try (VendedAdlsCredentialProvider provider = + new VendedAdlsCredentialProvider(ImmutableMap.of(VendedAdlsCredentialProvider.URI, URI))) { AzureSasCredential azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); assertThat(azureSasCredential.getSignature()) .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); - VendedAzureSasCredentialProvider deserializedProvider = - TestHelpers.roundTripSerialize(provider); + VendedAdlsCredentialProvider deserializedProvider = TestHelpers.roundTripSerialize(provider); AzureSasCredential reGeneratedAzureSasCredential = deserializedProvider.credentialForAccount(STORAGE_ACCOUNT); From c92479d15e68828fcbcf99a77546bd03fa3552bc Mon Sep 17 00:00:00 2001 From: ChaladiMohanVamsi Date: Sat, 14 Dec 2024 20:59:22 +0530 Subject: [PATCH 07/21] Apply suggestions from code review Co-authored-by: Eduard Tudenhoefner --- .../iceberg/azure/adlsv2/AzureSasCredentialRefresher.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java index 0a3e1b932568..c4d07f3b22c2 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java @@ -29,7 +29,7 @@ class AzureSasCredentialRefresher { private final ScheduledExecutorService refreshExecutor; private final AzureSasCredential azureSasCredential; - private static final long MAX_REFRESH_WINDOW_MILLIS = 300_000; // 5 minutes; + private static final long MAX_REFRESH_WINDOW_MILLIS = 300_000; // 5 minutes private static final long MIN_REFRESH_WAIT_MILLIS = 10; AzureSasCredentialRefresher( @@ -46,7 +46,7 @@ public AzureSasCredential azureSasCredential() { return this.azureSasCredential; } - private void scheduleRefresh(Long expireAtMillis) { + private void scheduleRefresh(Long expiresAtMillis) { this.refreshExecutor.schedule( () -> { Pair sasTokenWithExpiration = sasTokenWithExpirationSupplier.get(); @@ -59,7 +59,7 @@ private void scheduleRefresh(Long expireAtMillis) { TimeUnit.MILLISECONDS); } - private long refreshDelayMillis(Long expireAtMillis) { + private long refreshDelayMillis(Long expiresAtMillis) { long expireInMillis = expireAtMillis - System.currentTimeMillis(); // how much ahead of time to start the request to allow it to complete long refreshWindowMillis = Math.min(expireInMillis / 10, MAX_REFRESH_WINDOW_MILLIS); From bc08205b510bd5c0b688f4fd5addcf10e5afbe7e Mon Sep 17 00:00:00 2001 From: Mohan Vamsi Date: Sat, 14 Dec 2024 21:44:45 +0530 Subject: [PATCH 08/21] Apply suggestions from code review. --- .../apache/iceberg/azure/AzureProperties.java | 2 +- .../adlsv2/AzureSasCredentialRefresher.java | 20 +++++++++---------- .../adlsv2/VendedAdlsCredentialProvider.java | 2 +- 3 files changed, 11 insertions(+), 13 deletions(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java index 04982ce3df8c..1dad90c4f472 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java +++ b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java @@ -117,7 +117,7 @@ public Optional adlsWriteBlockSize() { */ public void applyClientConfiguration(String account, DataLakeFileSystemClientBuilder builder) { String sasToken = adlsSasTokens.get(account); - if (adlsRefreshCredentialsEnabled && !Strings.isNullOrEmpty(adlsRefreshCredentialsEndpoint)) { + if (vendedAdlsCredentialProvider != null) { builder.credential(vendedAdlsCredentialProvider.credentialForAccount(account)); } else if (sasToken != null && !sasToken.isEmpty()) { builder.sasToken(sasToken); diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java index c4d07f3b22c2..1b9908f77133 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java @@ -22,7 +22,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; -import org.apache.commons.lang3.tuple.Pair; +import org.apache.iceberg.util.Pair; class AzureSasCredentialRefresher { private final Supplier> sasTokenWithExpirationSupplier; @@ -38,8 +38,8 @@ class AzureSasCredentialRefresher { this.sasTokenWithExpirationSupplier = sasTokenWithExpirationSupplier; this.refreshExecutor = refreshExecutor; Pair sasTokenWithExpiration = sasTokenWithExpirationSupplier.get(); - this.azureSasCredential = new AzureSasCredential(sasTokenWithExpiration.getLeft()); - scheduleRefresh(sasTokenWithExpiration.getRight()); + this.azureSasCredential = new AzureSasCredential(sasTokenWithExpiration.first()); + scheduleRefresh(sasTokenWithExpiration.second()); } public AzureSasCredential azureSasCredential() { @@ -50,21 +50,19 @@ private void scheduleRefresh(Long expiresAtMillis) { this.refreshExecutor.schedule( () -> { Pair sasTokenWithExpiration = sasTokenWithExpirationSupplier.get(); - azureSasCredential.update(sasTokenWithExpiration.getLeft()); - if (sasTokenWithExpiration.getRight() != null) { - this.scheduleRefresh(sasTokenWithExpiration.getRight()); - } + azureSasCredential.update(sasTokenWithExpiration.first()); + this.scheduleRefresh(sasTokenWithExpiration.second()); }, - refreshDelayMillis(expireAtMillis), + refreshDelayMillis(expiresAtMillis), TimeUnit.MILLISECONDS); } private long refreshDelayMillis(Long expiresAtMillis) { - long expireInMillis = expireAtMillis - System.currentTimeMillis(); + long expiresInMillis = expiresAtMillis - System.currentTimeMillis(); // how much ahead of time to start the request to allow it to complete - long refreshWindowMillis = Math.min(expireInMillis / 10, MAX_REFRESH_WINDOW_MILLIS); + long refreshWindowMillis = Math.min(expiresInMillis / 10, MAX_REFRESH_WINDOW_MILLIS); // how much time to wait before expiration - long waitIntervalMillis = expireInMillis - refreshWindowMillis; + long waitIntervalMillis = expiresInMillis - refreshWindowMillis; // how much time to actually wait return Math.max(waitIntervalMillis, MIN_REFRESH_WAIT_MILLIS); } diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java index 89d7e0709f75..145e236cc62c 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java @@ -27,7 +27,6 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.tuple.Pair; 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; @@ -39,6 +38,7 @@ 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; From 5913fd168bd7fc56a6c940685304f1260479f67a Mon Sep 17 00:00:00 2001 From: Mohan Vamsi Date: Mon, 16 Dec 2024 20:39:13 +0530 Subject: [PATCH 09/21] Remove config headers while fetching vended credentials from credential endpoint. --- .../azure/adlsv2/VendedAdlsCredentialProvider.java | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java index 145e236cc62c..4293549d87db 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java @@ -33,7 +33,6 @@ 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; @@ -141,23 +140,15 @@ private RESTClient httpClient() { } private LoadCredentialsResponse fetchCredentials() { - Map headers = - RESTUtil.merge( - configHeaders(properties), - OAuth2Util.authHeaders(properties.get(OAuth2Properties.TOKEN))); return httpClient() .get( properties.get(URI), null, LoadCredentialsResponse.class, - headers, + OAuth2Util.authHeaders(properties.get(OAuth2Properties.TOKEN)), ErrorHandlers.defaultErrorHandler()); } - private Map configHeaders(Map props) { - return RESTUtil.extractPrefixMap(props, "header."); - } - private void checkCredential(Credential credential, String property) { Preconditions.checkState( credential.config().containsKey(property), From 016347df07bcc74bd6c529f8311264f81aaeef03 Mon Sep 17 00:00:00 2001 From: Mohan Vamsi Date: Sun, 9 Mar 2025 00:40:18 +0530 Subject: [PATCH 10/21] Implement sasToken refresh using SimpleTokenRefresh. --- .../apache/iceberg/azure/AzureProperties.java | 14 ++- .../adlsv2/AzureSasCredentialRefresher.java | 69 ----------- .../adlsv2/VendedAdlsCredentialProvider.java | 107 +++++++----------- .../VendedAzureSasCredentialPolicy.java | 67 +++++++++++ .../iceberg/azure/AzurePropertiesTest.java | 14 ++- .../adlsv2/BaseVendedCredentialsTest.java | 47 ++++++++ .../VendedAdlsCredentialProviderTest.java | 79 ++++++------- .../VendedAzureSasCredentialPolicyTest.java | 96 ++++++++++++++++ 8 files changed, 299 insertions(+), 194 deletions(-) delete mode 100644 azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java create mode 100644 azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicy.java create mode 100644 azure/src/test/java/org/apache/iceberg/azure/adlsv2/BaseVendedCredentialsTest.java create mode 100644 azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicyTest.java diff --git a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java index be316cd22935..c932e04e4c5e 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java +++ b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java @@ -25,10 +25,13 @@ import java.util.Collections; import java.util.Map; import java.util.Optional; +import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.azure.adlsv2.VendedAdlsCredentialProvider; +import org.apache.iceberg.azure.adlsv2.VendedAzureSasCredentialPolicy; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.rest.RESTUtil; import org.apache.iceberg.util.PropertyUtil; public class AzureProperties implements Serializable { @@ -57,8 +60,6 @@ public class AzureProperties implements Serializable { private Long adlsWriteBlockSize; private VendedAdlsCredentialProvider vendedAdlsCredentialProvider; - private String adlsRefreshCredentialsEndpoint; - private boolean adlsRefreshCredentialsEnabled; public AzureProperties() {} @@ -84,8 +85,11 @@ public AzureProperties(Map properties) { if (properties.containsKey(ADLS_WRITE_BLOCK_SIZE)) { this.adlsWriteBlockSize = Long.parseLong(properties.get(ADLS_WRITE_BLOCK_SIZE)); } - this.adlsRefreshCredentialsEndpoint = properties.get(ADLS_REFRESH_CREDENTIALS_ENDPOINT); - this.adlsRefreshCredentialsEnabled = + String adlsRefreshCredentialsEndpoint = + RESTUtil.resolveEndpoint( + properties.get(CatalogProperties.URI), + properties.get(ADLS_REFRESH_CREDENTIALS_ENDPOINT)); + boolean adlsRefreshCredentialsEnabled = PropertyUtil.propertyAsBoolean(properties, ADLS_REFRESH_CREDENTIALS_ENABLED, true); if (adlsRefreshCredentialsEnabled && !Strings.isNullOrEmpty(adlsRefreshCredentialsEndpoint)) { Map credentialProviderProperties = Maps.newHashMap(properties); @@ -116,7 +120,7 @@ public Optional adlsWriteBlockSize() { public void applyClientConfiguration(String account, DataLakeFileSystemClientBuilder builder) { String sasToken = adlsSasTokens.get(account); if (vendedAdlsCredentialProvider != null) { - builder.credential(vendedAdlsCredentialProvider.credentialForAccount(account)); + builder.addPolicy(new VendedAzureSasCredentialPolicy(account, vendedAdlsCredentialProvider)); } else if (sasToken != null && !sasToken.isEmpty()) { builder.sasToken(sasToken); } else if (namedKeyCreds != null) { diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java deleted file mode 100644 index 1b9908f77133..000000000000 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/AzureSasCredentialRefresher.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.function.Supplier; -import org.apache.iceberg.util.Pair; - -class AzureSasCredentialRefresher { - private final Supplier> sasTokenWithExpirationSupplier; - private final ScheduledExecutorService refreshExecutor; - private final AzureSasCredential azureSasCredential; - - private static final long MAX_REFRESH_WINDOW_MILLIS = 300_000; // 5 minutes - private static final long MIN_REFRESH_WAIT_MILLIS = 10; - - AzureSasCredentialRefresher( - Supplier> sasTokenWithExpirationSupplier, - ScheduledExecutorService refreshExecutor) { - this.sasTokenWithExpirationSupplier = sasTokenWithExpirationSupplier; - this.refreshExecutor = refreshExecutor; - Pair sasTokenWithExpiration = sasTokenWithExpirationSupplier.get(); - this.azureSasCredential = new AzureSasCredential(sasTokenWithExpiration.first()); - scheduleRefresh(sasTokenWithExpiration.second()); - } - - public AzureSasCredential azureSasCredential() { - return this.azureSasCredential; - } - - private void scheduleRefresh(Long expiresAtMillis) { - this.refreshExecutor.schedule( - () -> { - Pair sasTokenWithExpiration = sasTokenWithExpirationSupplier.get(); - azureSasCredential.update(sasTokenWithExpiration.first()); - this.scheduleRefresh(sasTokenWithExpiration.second()); - }, - refreshDelayMillis(expiresAtMillis), - TimeUnit.MILLISECONDS); - } - - private long refreshDelayMillis(Long expiresAtMillis) { - long expiresInMillis = expiresAtMillis - System.currentTimeMillis(); - // how much ahead of time to start the request to allow it to complete - long refreshWindowMillis = Math.min(expiresInMillis / 10, MAX_REFRESH_WINDOW_MILLIS); - // how much time to wait before expiration - long waitIntervalMillis = expiresInMillis - refreshWindowMillis; - // how much time to actually wait - return Math.max(waitIntervalMillis, MIN_REFRESH_WAIT_MILLIS); - } -} diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java index 4293549d87db..094202af2040 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java @@ -18,13 +18,13 @@ */ package org.apache.iceberg.azure.adlsv2; -import com.azure.core.credential.AzureSasCredential; +import com.azure.core.credential.AccessToken; +import com.azure.core.credential.SimpleTokenCache; import java.io.Serializable; +import java.time.Instant; +import java.time.ZoneOffset; 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; @@ -32,51 +32,48 @@ 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.HTTPHeaders; import org.apache.iceberg.rest.RESTClient; +import org.apache.iceberg.rest.auth.DefaultAuthSession; 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; +import reactor.core.publisher.Mono; 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 properties; - private transient volatile Map - azureSasCredentialRefresherMap; + private transient volatile Map azureSasCredentialMap; private transient volatile RESTClient client; - private transient volatile ScheduledExecutorService refreshExecutor; public VendedAdlsCredentialProvider(Map 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(); + azureSasCredentialMap = Maps.newHashMap(); } - public AzureSasCredential credentialForAccount(String storageAccount) { - Map refresherForAccountMap = - azureSasCredentialRefresherMap(); - if (refresherForAccountMap.containsKey(storageAccount)) { - return refresherForAccountMap.get(storageAccount).azureSasCredential(); + public String credentialForAccount(String storageAccount) { + Map tokenCacheForAccountMap = azureSasCredentialMap(); + if (tokenCacheForAccountMap.containsKey(storageAccount)) { + return tokenFromCache(tokenCacheForAccountMap.get(storageAccount)); } else { - AzureSasCredentialRefresher azureSasCredentialRefresher = - new AzureSasCredentialRefresher( - () -> this.sasTokenWithExpiration(storageAccount), credentialRefreshExecutor()); - refresherForAccountMap.put(storageAccount, azureSasCredentialRefresher); - return azureSasCredentialRefresher.azureSasCredential(); + SimpleTokenCache tokenCache = + new SimpleTokenCache(() -> Mono.fromSupplier(() -> sasTokenSupplier(storageAccount))); + tokenCacheForAccountMap.put(storageAccount, tokenCache); + return tokenFromCache(tokenCache); } } - private Pair sasTokenWithExpiration(String storageAccount) { + private String tokenFromCache(SimpleTokenCache simpleTokenCache) { + return simpleTokenCache.getToken().map(AccessToken::getToken).block(); + } + + private AccessToken sasTokenSupplier(String storageAccount) { LoadCredentialsResponse response = fetchCredentials(); List adlsCredentials = response.credentials().stream() @@ -102,36 +99,33 @@ private Pair sasTokenWithExpiration(String storageAccount) { .config() .get(AzureProperties.ADLS_SAS_TOKEN_EXPIRES_AT_MS_PREFIX + storageAccount)); - return Pair.of(updatedSasToken, tokenExpiresAtMillis); + return new AccessToken( + updatedSasToken, Instant.ofEpochMilli(tokenExpiresAtMillis).atOffset(ZoneOffset.UTC)); } - private Map azureSasCredentialRefresherMap() { - if (this.azureSasCredentialRefresherMap == null) { + private Map azureSasCredentialMap() { + if (this.azureSasCredentialMap == null) { synchronized (this) { - if (this.azureSasCredentialRefresherMap == null) { - this.azureSasCredentialRefresherMap = Maps.newHashMap(); + if (this.azureSasCredentialMap == null) { + this.azureSasCredentialMap = 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; + return this.azureSasCredentialMap; } private RESTClient httpClient() { if (null == client) { synchronized (this) { if (null == client) { - client = HTTPClient.builder(properties).uri(properties.get(URI)).build(); + DefaultAuthSession authSession = + DefaultAuthSession.of( + HTTPHeaders.of(OAuth2Util.authHeaders(properties.get(OAuth2Properties.TOKEN)))); + client = + HTTPClient.builder(properties) + .uri(properties.get(URI)) + .withAuthSession(authSession) + .build(); } } } @@ -145,7 +139,7 @@ private LoadCredentialsResponse fetchCredentials() { properties.get(URI), null, LoadCredentialsResponse.class, - OAuth2Util.authHeaders(properties.get(OAuth2Properties.TOKEN)), + Map.of(), ErrorHandlers.defaultErrorHandler()); } @@ -159,30 +153,5 @@ private void checkCredential(Credential credential, String property) { @Override public void close() { IOUtils.closeQuietly(client); - shutdownRefreshExecutor(); - } - - private void shutdownRefreshExecutor() { - if (refreshExecutor != null) { - ScheduledExecutorService service = refreshExecutor; - this.refreshExecutor = null; - - List tasks = service.shutdownNow(); - tasks.forEach( - task -> { - if (task instanceof Future) { - ((Future) task).cancel(true); - } - }); - - try { - if (!service.awaitTermination(1, TimeUnit.MINUTES)) { - LOG.warn("Timed out waiting for refresh executor to terminate"); - } - } catch (InterruptedException e) { - LOG.warn("Interrupted while waiting for refresh executor to terminate", e); - Thread.currentThread().interrupt(); - } - } } } diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicy.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicy.java new file mode 100644 index 000000000000..b2804c2c6693 --- /dev/null +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicy.java @@ -0,0 +1,67 @@ +/* + * 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 com.azure.core.http.HttpPipelineCallContext; +import com.azure.core.http.HttpPipelineNextPolicy; +import com.azure.core.http.HttpPipelineNextSyncPolicy; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.policy.AzureSasCredentialPolicy; +import com.azure.core.http.policy.HttpPipelinePolicy; +import java.util.Objects; +import reactor.core.publisher.Mono; + +public class VendedAzureSasCredentialPolicy implements HttpPipelinePolicy { + private final String account; + private final VendedAdlsCredentialProvider vendedAdlsCredentialProvider; + private AzureSasCredential azureSasCredential; + private AzureSasCredentialPolicy azureSasCredentialPolicy; + + public VendedAzureSasCredentialPolicy( + String account, VendedAdlsCredentialProvider vendedAdlsCredentialProvider) { + this.account = account; + this.vendedAdlsCredentialProvider = vendedAdlsCredentialProvider; + } + + @Override + public Mono process( + HttpPipelineCallContext httpPipelineCallContext, + HttpPipelineNextPolicy httpPipelineNextPolicy) { + updateAzureSasCredential(); + return azureSasCredentialPolicy.process(httpPipelineCallContext, httpPipelineNextPolicy); + } + + @Override + public HttpResponse processSync( + HttpPipelineCallContext context, HttpPipelineNextSyncPolicy next) { + updateAzureSasCredential(); + return azureSasCredentialPolicy.processSync(context, next); + } + + private void updateAzureSasCredential() { + String sasToken = vendedAdlsCredentialProvider.credentialForAccount(account); + if (Objects.isNull(azureSasCredential)) { + azureSasCredential = new AzureSasCredential(sasToken); + azureSasCredentialPolicy = new AzureSasCredentialPolicy(azureSasCredential, false); + } else { + azureSasCredential.update(sasToken); + } + } +} diff --git a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java index a01d21dc2d81..5fe63b93f283 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java @@ -42,6 +42,7 @@ import com.azure.storage.file.datalake.DataLakeFileSystemClientBuilder; import org.apache.iceberg.TestHelpers; import org.apache.iceberg.azure.adlsv2.VendedAdlsCredentialProvider; +import org.apache.iceberg.azure.adlsv2.VendedAzureSasCredentialPolicy; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; @@ -79,19 +80,24 @@ public void testWithSasToken() { @Test public void testWithRefreshCredentialsEndpoint() { - try (var providerMockedConstruction = mockConstruction(VendedAdlsCredentialProvider.class)) { + try (var providerMockedConstruction = mockConstruction(VendedAdlsCredentialProvider.class); + var policyMockedConstruction = mockConstruction(VendedAzureSasCredentialPolicy.class)) { AzureProperties props = new AzureProperties(ImmutableMap.of(ADLS_REFRESH_CREDENTIALS_ENDPOINT, "endpoint")); assertThat(providerMockedConstruction.constructed()).hasSize(1); var providerMock = providerMockedConstruction.constructed().get(0); - AzureSasCredential azureSasCredential = mock(AzureSasCredential.class); - when(providerMock.credentialForAccount("account1")).thenReturn(azureSasCredential); + String sasToken = "random-token"; + + when(providerMock.credentialForAccount("account1")).thenReturn(sasToken); DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); props.applyClientConfiguration("account1", clientBuilder); - verify(clientBuilder, times(1)).credential(azureSasCredential); + var policyMock = policyMockedConstruction.constructed().get(0); + + verify(clientBuilder, never()).credential(any(AzureSasCredential.class)); verify(clientBuilder, never()).sasToken(any()); verify(clientBuilder, never()).credential(any(StorageSharedKeyCredential.class)); + verify(clientBuilder, times(1)).addPolicy(policyMock); } } diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/BaseVendedCredentialsTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/BaseVendedCredentialsTest.java new file mode 100644 index 000000000000..1aacf87cbc5f --- /dev/null +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/BaseVendedCredentialsTest.java @@ -0,0 +1,47 @@ +/* + * 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.mockserver.integration.ClientAndServer.startClientAndServer; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.mockserver.integration.ClientAndServer; + +public class BaseVendedCredentialsTest { + protected static final int PORT = 3232; + protected static final String BASE_URI = String.format("http://127.0.0.1:%d", PORT); + protected static ClientAndServer mockServer; + + @BeforeAll + public static void beforeAll() { + mockServer = startClientAndServer(PORT); + } + + @AfterAll + public static void stopServer() { + mockServer.stop(); + } + + @BeforeEach + public void before() { + mockServer.reset(); + } +} diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java index 0ddfecb41af2..1e18d64359f0 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java @@ -22,11 +22,9 @@ import static org.apache.iceberg.azure.AzureProperties.ADLS_SAS_TOKEN_PREFIX; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.mockserver.integration.ClientAndServer.startClientAndServer; import static org.mockserver.model.HttpRequest.request; import static org.mockserver.model.HttpResponse.response; -import com.azure.core.credential.AzureSasCredential; import com.azure.core.http.HttpMethod; import java.io.IOException; import java.time.Instant; @@ -39,19 +37,13 @@ import org.apache.iceberg.rest.responses.ImmutableLoadCredentialsResponse; import org.apache.iceberg.rest.responses.LoadCredentialsResponse; import org.apache.iceberg.rest.responses.LoadCredentialsResponseParser; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.mockserver.integration.ClientAndServer; import org.mockserver.model.HttpRequest; import org.mockserver.model.HttpResponse; import org.mockserver.verify.VerificationTimes; -public class VendedAdlsCredentialProviderTest { - private static final int PORT = 3232; - private static final String URI = String.format("http://127.0.0.1:%d/v1/credentials", PORT); - private static ClientAndServer mockServer; +public class VendedAdlsCredentialProviderTest extends BaseVendedCredentialsTest { + private static final String CREDENTIALS_URI = String.format("%s%s", BASE_URI, "/v1/credentials"); private static final String STORAGE_ACCOUNT = "account1"; private static final String CREDENTIAL_PREFIX = "abfs://container@account1.dfs.core.windows.net/dir"; @@ -59,21 +51,6 @@ public class VendedAdlsCredentialProviderTest { private static final String CREDENTIAL_PREFIX_2 = "abfs://container@account2.dfs.core.windows.net/dir"; - @BeforeAll - public static void beforeAll() { - mockServer = startClientAndServer(PORT); - } - - @AfterAll - public static void stopServer() { - mockServer.stop(); - } - - @BeforeEach - public void before() { - mockServer.reset(); - } - @Test public void invalidOrMissingUri() { assertThatThrownBy(() -> new VendedAdlsCredentialProvider(null)) @@ -104,7 +81,8 @@ public void noADLSCredentials() { mockServer.when(mockRequest).respond(mockResponse); try (VendedAdlsCredentialProvider provider = - new VendedAdlsCredentialProvider(ImmutableMap.of(VendedAdlsCredentialProvider.URI, URI))) { + new VendedAdlsCredentialProvider( + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) .isInstanceOf(IllegalStateException.class) .hasMessage("Invalid ADLS Credentials for storage-account account1: empty"); @@ -128,7 +106,8 @@ public void expirationNotSet() { mockServer.when(mockRequest).respond(mockResponse); try (VendedAdlsCredentialProvider provider = - new VendedAdlsCredentialProvider(ImmutableMap.of(VendedAdlsCredentialProvider.URI, URI))) { + new VendedAdlsCredentialProvider( + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) .isInstanceOf(IllegalStateException.class) .hasMessage("Invalid ADLS Credentials: adls.sas-token-expires-at-ms.account1 not set"); @@ -155,9 +134,10 @@ public void nonExpiredSasToken() { mockServer.when(mockRequest).respond(mockResponse); try (VendedAdlsCredentialProvider provider = - new VendedAdlsCredentialProvider(ImmutableMap.of(VendedAdlsCredentialProvider.URI, URI))) { - AzureSasCredential azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); - assertThat(azureSasCredential.getSignature()) + new VendedAdlsCredentialProvider( + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); + assertThat(azureSasCredential) .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); for (int i = 0; i < 5; i++) { @@ -188,16 +168,18 @@ public void expiredSasToken() throws InterruptedException { mockServer.when(mockRequest).respond(mockResponse); try (VendedAdlsCredentialProvider provider = - new VendedAdlsCredentialProvider(ImmutableMap.of(VendedAdlsCredentialProvider.URI, URI))) { - AzureSasCredential azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); - assertThat(azureSasCredential.getSignature()) + new VendedAdlsCredentialProvider( + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); + assertThat(azureSasCredential) .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); - Thread.sleep(20); - // Since expiration time past to current time, the refresh will fall back at minimum 10ms - // interval + // resolving credentials multiple times should hit the credentials endpoint again + String refreshedAzureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); + assertThat(refreshedAzureSasCredential) + .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); } - mockServer.verify(mockRequest, VerificationTimes.atLeast(2)); + mockServer.verify(mockRequest, VerificationTimes.exactly(2)); } @Test @@ -230,7 +212,8 @@ public void multipleADLSCredentialsPerStorageAccount() { mockServer.when(mockRequest).respond(mockResponse); try (VendedAdlsCredentialProvider provider = - new VendedAdlsCredentialProvider(ImmutableMap.of(VendedAdlsCredentialProvider.URI, URI))) { + new VendedAdlsCredentialProvider( + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) .isInstanceOf(IllegalStateException.class) .hasMessage( @@ -268,13 +251,14 @@ public void multipleStorageAccounts() { mockServer.when(mockRequest).respond(mockResponse); try (VendedAdlsCredentialProvider provider = - new VendedAdlsCredentialProvider(ImmutableMap.of(VendedAdlsCredentialProvider.URI, URI))) { - AzureSasCredential azureSasCredential1 = provider.credentialForAccount(STORAGE_ACCOUNT); - AzureSasCredential azureSasCredential2 = provider.credentialForAccount(STORAGE_ACCOUNT_2); + new VendedAdlsCredentialProvider( + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + String azureSasCredential1 = provider.credentialForAccount(STORAGE_ACCOUNT); + String azureSasCredential2 = provider.credentialForAccount(STORAGE_ACCOUNT_2); assertThat(azureSasCredential1).isNotSameAs(azureSasCredential2); - assertThat(azureSasCredential1.getSignature()) + assertThat(azureSasCredential1) .isEqualTo(credential1.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); - assertThat(azureSasCredential2.getSignature()) + assertThat(azureSasCredential2) .isEqualTo(credential2.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT_2)); } } @@ -299,13 +283,14 @@ public void serializableTest() throws IOException, ClassNotFoundException { mockServer.when(mockRequest).respond(mockResponse); try (VendedAdlsCredentialProvider provider = - new VendedAdlsCredentialProvider(ImmutableMap.of(VendedAdlsCredentialProvider.URI, URI))) { - AzureSasCredential azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); - assertThat(azureSasCredential.getSignature()) + new VendedAdlsCredentialProvider( + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); + assertThat(azureSasCredential) .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); VendedAdlsCredentialProvider deserializedProvider = TestHelpers.roundTripSerialize(provider); - AzureSasCredential reGeneratedAzureSasCredential = + String reGeneratedAzureSasCredential = deserializedProvider.credentialForAccount(STORAGE_ACCOUNT); assertThat(azureSasCredential).isNotSameAs(reGeneratedAzureSasCredential); diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicyTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicyTest.java new file mode 100644 index 000000000000..ec76f9671f9d --- /dev/null +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicyTest.java @@ -0,0 +1,96 @@ +/* + * 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.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockserver.model.HttpRequest.request; + +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpMethod; +import com.azure.storage.file.datalake.DataLakeFileSystemClient; +import com.azure.storage.file.datalake.DataLakeFileSystemClientBuilder; +import com.azure.storage.file.datalake.models.DataLakeStorageException; +import org.junit.jupiter.api.Test; +import org.mockserver.model.HttpRequest; +import org.mockserver.model.HttpResponse; +import org.mockserver.verify.VerificationTimes; + +public class VendedAzureSasCredentialPolicyTest extends BaseVendedCredentialsTest { + + private static final String STORAGE_ACCOUNT = "account1"; + private static final String ACCOUNT_ENDPOINT = String.format("%s/%s", BASE_URI, STORAGE_ACCOUNT); + + @Test + public void vendedSasTokenAsRequestQueryParameters() { + String filePath = "file1"; + String container = "container1"; + String validSasToken = "tokenInstance=1"; + String expiredSasToken = "tokenInstance=2"; + + VendedAdlsCredentialProvider vendedAdlsCredentialProvider = + mock(VendedAdlsCredentialProvider.class); + VendedAzureSasCredentialPolicy vendedAzureSasCredentialPolicy = + new VendedAzureSasCredentialPolicy(STORAGE_ACCOUNT, vendedAdlsCredentialProvider); + + DataLakeFileSystemClient client = + new DataLakeFileSystemClientBuilder() + .httpClient(HttpClient.createDefault()) + .addPolicy(vendedAzureSasCredentialPolicy) + .fileSystemName(container) + .endpoint(ACCOUNT_ENDPOINT) + .buildClient(); + + String requestPath = String.format("/%s/%s/%s", STORAGE_ACCOUNT, container, filePath); + + HttpRequest mockRequestWithValidSasToken = + request(requestPath) + .withMethod(HttpMethod.HEAD.name()) + .withQueryStringParameter("tokenInstance", "1"); + mockServer + .when(mockRequestWithValidSasToken) + .respond(HttpResponse.response().withStatusCode(200)); + + HttpRequest mockRequestWithExpiredSasToken = + request(requestPath) + .withMethod(HttpMethod.HEAD.name()) + .withQueryStringParameter("tokenInstance", "2"); + mockServer + .when(mockRequestWithExpiredSasToken) + .respond(HttpResponse.response().withStatusCode(403)); + + when(vendedAdlsCredentialProvider.credentialForAccount(STORAGE_ACCOUNT)) + .thenReturn(validSasToken); + assertThat(client.getFileClient(filePath).exists()).isTrue(); + mockServer.verify(mockRequestWithValidSasToken, VerificationTimes.exactly(1)); + + when(vendedAdlsCredentialProvider.credentialForAccount(STORAGE_ACCOUNT)) + .thenReturn(expiredSasToken); + + // Every new request of the same client fetches latest SasToken credentials from + // VendedAdlsCredentialProvider to build http request query parameters. + assertThatThrownBy(() -> client.getFileClient(filePath).exists()) + .isInstanceOf(DataLakeStorageException.class) + .hasMessageContaining( + "If you are using a SAS token, and the server returned an error message that says 'Signature did not match'"); + mockServer.verify(mockRequestWithExpiredSasToken, VerificationTimes.atLeast(1)); + } +} From a135ca0cf7ffb8c4e462125ee5fb0105a452e62f Mon Sep 17 00:00:00 2001 From: ChaladiMohanVamsi Date: Fri, 14 Mar 2025 23:25:26 +0530 Subject: [PATCH 11/21] Apply suggestions from code review Co-authored-by: Eduard Tudenhoefner --- .../azure/adlsv2/VendedAdlsCredentialProvider.java | 6 +++--- .../azure/adlsv2/VendedAzureSasCredentialPolicy.java | 8 ++++---- .../azure/adlsv2/VendedAdlsCredentialProviderTest.java | 2 +- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java index 094202af2040..f5e04f66e238 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java @@ -47,7 +47,7 @@ public class VendedAdlsCredentialProvider implements Serializable, AutoCloseable public static final String URI = "credentials.uri"; private final SerializableMap properties; - private transient volatile Map azureSasCredentialMap; + private transient volatile Map sasCredentialByAccount; private transient volatile RESTClient client; public VendedAdlsCredentialProvider(Map properties) { @@ -91,7 +91,7 @@ private AccessToken sasTokenSupplier(String storageAccount) { checkCredential( adlsCredential, AzureProperties.ADLS_SAS_TOKEN_EXPIRES_AT_MS_PREFIX + storageAccount); - String updatedSasToken = + String sasToken = adlsCredential.config().get(AzureProperties.ADLS_SAS_TOKEN_PREFIX + storageAccount); Long tokenExpiresAtMillis = Long.parseLong( @@ -103,7 +103,7 @@ private AccessToken sasTokenSupplier(String storageAccount) { updatedSasToken, Instant.ofEpochMilli(tokenExpiresAtMillis).atOffset(ZoneOffset.UTC)); } - private Map azureSasCredentialMap() { + private Map sasCredentialByAccount() { if (this.azureSasCredentialMap == null) { synchronized (this) { if (this.azureSasCredentialMap == null) { diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicy.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicy.java index b2804c2c6693..e4b05c3998af 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicy.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicy.java @@ -55,11 +55,11 @@ public HttpResponse processSync( return azureSasCredentialPolicy.processSync(context, next); } - private void updateAzureSasCredential() { + private void maybeUpdateCredential() { String sasToken = vendedAdlsCredentialProvider.credentialForAccount(account); - if (Objects.isNull(azureSasCredential)) { - azureSasCredential = new AzureSasCredential(sasToken); - azureSasCredentialPolicy = new AzureSasCredentialPolicy(azureSasCredential, false); + if (azureSasCredential == null) { + this.azureSasCredential = new AzureSasCredential(sasToken); + this.azureSasCredentialPolicy = new AzureSasCredentialPolicy(azureSasCredential, false); } else { azureSasCredential.update(sasToken); } diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java index 1e18d64359f0..e30fbef8d590 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java @@ -149,7 +149,7 @@ public void nonExpiredSasToken() { } @Test - public void expiredSasToken() throws InterruptedException { + public void expiredSasToken() { HttpRequest mockRequest = request("/v1/credentials").withMethod(HttpMethod.GET.name()); Credential credential = ImmutableCredential.builder() From b97ee0c88763afd6952a33d377f5f04367f39c53 Mon Sep 17 00:00:00 2001 From: Mohan Vamsi Date: Tue, 11 Mar 2025 21:32:35 +0530 Subject: [PATCH 12/21] Fix mock server port conflict with dynamic port allocation. --- .../iceberg/azure/adlsv2/BaseVendedCredentialsTest.java | 8 +++++--- .../azure/adlsv2/VendedAdlsCredentialProviderTest.java | 2 +- .../azure/adlsv2/VendedAzureSasCredentialPolicyTest.java | 2 +- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/BaseVendedCredentialsTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/BaseVendedCredentialsTest.java index 1aacf87cbc5f..3fe969d00d9b 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/BaseVendedCredentialsTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/BaseVendedCredentialsTest.java @@ -26,13 +26,15 @@ import org.mockserver.integration.ClientAndServer; public class BaseVendedCredentialsTest { - protected static final int PORT = 3232; - protected static final String BASE_URI = String.format("http://127.0.0.1:%d", PORT); + protected static String baseUri; protected static ClientAndServer mockServer; @BeforeAll public static void beforeAll() { - mockServer = startClientAndServer(PORT); + // Allocate port dynamically as there could be parallel test executions. + mockServer = startClientAndServer(0); + int mockServerPort = mockServer.getPort(); + baseUri = String.format("http://127.0.0.1:%d", mockServerPort); } @AfterAll diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java index e30fbef8d590..7e2ade5fe601 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java @@ -43,7 +43,7 @@ import org.mockserver.verify.VerificationTimes; public class VendedAdlsCredentialProviderTest extends BaseVendedCredentialsTest { - private static final String CREDENTIALS_URI = String.format("%s%s", BASE_URI, "/v1/credentials"); + private static final String CREDENTIALS_URI = String.format("%s%s", baseUri, "/v1/credentials"); private static final String STORAGE_ACCOUNT = "account1"; private static final String CREDENTIAL_PREFIX = "abfs://container@account1.dfs.core.windows.net/dir"; diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicyTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicyTest.java index ec76f9671f9d..8aa2369c8270 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicyTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicyTest.java @@ -37,7 +37,7 @@ public class VendedAzureSasCredentialPolicyTest extends BaseVendedCredentialsTest { private static final String STORAGE_ACCOUNT = "account1"; - private static final String ACCOUNT_ENDPOINT = String.format("%s/%s", BASE_URI, STORAGE_ACCOUNT); + private static final String ACCOUNT_ENDPOINT = String.format("%s/%s", baseUri, STORAGE_ACCOUNT); @Test public void vendedSasTokenAsRequestQueryParameters() { From 0a2f094781d9e36e0166fd29565cf20fb82533aa Mon Sep 17 00:00:00 2001 From: Mohan Vamsi Date: Sat, 15 Mar 2025 00:40:23 +0530 Subject: [PATCH 13/21] Apply suggestions from code review. --- .../adlsv2/VendedAdlsCredentialProvider.java | 102 ++++++++------- .../VendedAzureSasCredentialPolicy.java | 5 +- .../iceberg/azure/AzurePropertiesTest.java | 59 +++------ .../VendedAdlsCredentialProviderTest.java | 120 ++++++++---------- 4 files changed, 129 insertions(+), 157 deletions(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java index f5e04f66e238..e6d37e57a38a 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java @@ -20,13 +20,13 @@ import com.azure.core.credential.AccessToken; import com.azure.core.credential.SimpleTokenCache; +import java.io.IOException; import java.io.Serializable; import java.time.Instant; import java.time.ZoneOffset; import java.util.List; import java.util.Map; 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; @@ -34,6 +34,7 @@ import org.apache.iceberg.rest.HTTPClient; import org.apache.iceberg.rest.HTTPHeaders; import org.apache.iceberg.rest.RESTClient; +import org.apache.iceberg.rest.auth.AuthSession; import org.apache.iceberg.rest.auth.DefaultAuthSession; import org.apache.iceberg.rest.auth.OAuth2Properties; import org.apache.iceberg.rest.auth.OAuth2Util; @@ -42,38 +43,33 @@ import org.apache.iceberg.util.SerializableMap; import reactor.core.publisher.Mono; -public class VendedAdlsCredentialProvider implements Serializable, AutoCloseable { +public class VendedAdlsCredentialProvider implements Serializable { public static final String URI = "credentials.uri"; private final SerializableMap properties; private transient volatile Map sasCredentialByAccount; - private transient volatile RESTClient client; + private transient volatile AuthSession authSession; public VendedAdlsCredentialProvider(Map properties) { Preconditions.checkArgument(null != properties, "Invalid properties: null"); Preconditions.checkArgument(null != properties.get(URI), "Invalid URI: null"); this.properties = SerializableMap.copyOf(properties); - azureSasCredentialMap = Maps.newHashMap(); } public String credentialForAccount(String storageAccount) { - Map tokenCacheForAccountMap = azureSasCredentialMap(); - if (tokenCacheForAccountMap.containsKey(storageAccount)) { - return tokenFromCache(tokenCacheForAccountMap.get(storageAccount)); - } else { - SimpleTokenCache tokenCache = - new SimpleTokenCache(() -> Mono.fromSupplier(() -> sasTokenSupplier(storageAccount))); - tokenCacheForAccountMap.put(storageAccount, tokenCache); - return tokenFromCache(tokenCache); - } - } - - private String tokenFromCache(SimpleTokenCache simpleTokenCache) { - return simpleTokenCache.getToken().map(AccessToken::getToken).block(); + return sasCredentialByAccount() + .computeIfAbsent( + storageAccount, + ignored -> + new SimpleTokenCache( + () -> Mono.fromSupplier(() -> sasTokenForAccount(storageAccount)))) + .getToken() + .map(AccessToken::getToken) + .block(); } - private AccessToken sasTokenSupplier(String storageAccount) { + private AccessToken sasTokenForAccount(String storageAccount) { LoadCredentialsResponse response = fetchCredentials(); List adlsCredentials = response.credentials().stream() @@ -93,54 +89,61 @@ private AccessToken sasTokenSupplier(String storageAccount) { String sasToken = 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 new AccessToken( - updatedSasToken, Instant.ofEpochMilli(tokenExpiresAtMillis).atOffset(ZoneOffset.UTC)); + Instant tokenExpiresAt = + Instant.ofEpochMilli( + Long.parseLong( + adlsCredential + .config() + .get(AzureProperties.ADLS_SAS_TOKEN_EXPIRES_AT_MS_PREFIX + storageAccount))); + + return new AccessToken(sasToken, tokenExpiresAt.atOffset(ZoneOffset.UTC)); } private Map sasCredentialByAccount() { - if (this.azureSasCredentialMap == null) { + if (this.sasCredentialByAccount == null) { synchronized (this) { - if (this.azureSasCredentialMap == null) { - this.azureSasCredentialMap = Maps.newHashMap(); + if (this.sasCredentialByAccount == null) { + this.sasCredentialByAccount = Maps.newHashMap(); } } } - return this.azureSasCredentialMap; + return this.sasCredentialByAccount; } private RESTClient httpClient() { - if (null == client) { + return HTTPClient.builder(properties) + .uri(properties.get(URI)) + .withAuthSession(authSession()) + .build(); + } + + private AuthSession authSession() { + if (this.authSession == null) { synchronized (this) { - if (null == client) { - DefaultAuthSession authSession = + if (this.authSession == null) { + this.authSession = DefaultAuthSession.of( HTTPHeaders.of(OAuth2Util.authHeaders(properties.get(OAuth2Properties.TOKEN)))); - client = - HTTPClient.builder(properties) - .uri(properties.get(URI)) - .withAuthSession(authSession) - .build(); } } } - - return client; + return this.authSession; } private LoadCredentialsResponse fetchCredentials() { - return httpClient() - .get( - properties.get(URI), - null, - LoadCredentialsResponse.class, - Map.of(), - ErrorHandlers.defaultErrorHandler()); + LoadCredentialsResponse response; + try (RESTClient client = httpClient()) { + response = + client.get( + properties.get(URI), + null, + LoadCredentialsResponse.class, + Map.of(), + ErrorHandlers.defaultErrorHandler()); + } catch (IOException e) { + throw new RuntimeException(e); + } + return response; } private void checkCredential(Credential credential, String property) { @@ -149,9 +152,4 @@ private void checkCredential(Credential credential, String property) { "Invalid ADLS Credentials: %s not set", property); } - - @Override - public void close() { - IOUtils.closeQuietly(client); - } } diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicy.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicy.java index e4b05c3998af..3e6c72c37b3d 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicy.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicy.java @@ -25,7 +25,6 @@ import com.azure.core.http.HttpResponse; import com.azure.core.http.policy.AzureSasCredentialPolicy; import com.azure.core.http.policy.HttpPipelinePolicy; -import java.util.Objects; import reactor.core.publisher.Mono; public class VendedAzureSasCredentialPolicy implements HttpPipelinePolicy { @@ -44,14 +43,14 @@ public VendedAzureSasCredentialPolicy( public Mono process( HttpPipelineCallContext httpPipelineCallContext, HttpPipelineNextPolicy httpPipelineNextPolicy) { - updateAzureSasCredential(); + maybeUpdateCredential(); return azureSasCredentialPolicy.process(httpPipelineCallContext, httpPipelineNextPolicy); } @Override public HttpResponse processSync( HttpPipelineCallContext context, HttpPipelineNextSyncPolicy next) { - updateAzureSasCredential(); + maybeUpdateCredential(); return azureSasCredentialPolicy.processSync(context, next); } diff --git a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java index 5fe63b93f283..4e0c4ea39782 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java @@ -30,18 +30,15 @@ import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.mockConstruction; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; import com.azure.core.credential.AzureSasCredential; import com.azure.core.credential.TokenCredential; import com.azure.storage.common.StorageSharedKeyCredential; import com.azure.storage.file.datalake.DataLakeFileSystemClientBuilder; import org.apache.iceberg.TestHelpers; -import org.apache.iceberg.azure.adlsv2.VendedAdlsCredentialProvider; import org.apache.iceberg.azure.adlsv2.VendedAzureSasCredentialPolicy; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; @@ -80,45 +77,31 @@ public void testWithSasToken() { @Test public void testWithRefreshCredentialsEndpoint() { - try (var providerMockedConstruction = mockConstruction(VendedAdlsCredentialProvider.class); - var policyMockedConstruction = mockConstruction(VendedAzureSasCredentialPolicy.class)) { - AzureProperties props = - new AzureProperties(ImmutableMap.of(ADLS_REFRESH_CREDENTIALS_ENDPOINT, "endpoint")); - assertThat(providerMockedConstruction.constructed()).hasSize(1); - var providerMock = providerMockedConstruction.constructed().get(0); - String sasToken = "random-token"; - - when(providerMock.credentialForAccount("account1")).thenReturn(sasToken); - DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); - props.applyClientConfiguration("account1", clientBuilder); - - var policyMock = policyMockedConstruction.constructed().get(0); - - verify(clientBuilder, never()).credential(any(AzureSasCredential.class)); - verify(clientBuilder, never()).sasToken(any()); - verify(clientBuilder, never()).credential(any(StorageSharedKeyCredential.class)); - verify(clientBuilder, times(1)).addPolicy(policyMock); - } + AzureProperties props = + new AzureProperties(ImmutableMap.of(ADLS_REFRESH_CREDENTIALS_ENDPOINT, "endpoint")); + + DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); + props.applyClientConfiguration("account1", clientBuilder); + + verify(clientBuilder, never()).credential(any(AzureSasCredential.class)); + verify(clientBuilder, never()).sasToken(any()); + verify(clientBuilder, never()).credential(any(StorageSharedKeyCredential.class)); + verify(clientBuilder, times(1)).addPolicy(any(VendedAzureSasCredentialPolicy.class)); } @Test public void testWithRefreshCredentialsEndpointDisabled() { - try (var providerMockedConstruction = mockConstruction(VendedAdlsCredentialProvider.class)) { - AzureProperties props = - new AzureProperties( - ImmutableMap.of( - ADLS_REFRESH_CREDENTIALS_ENDPOINT, - "endpoint", - ADLS_REFRESH_CREDENTIALS_ENABLED, - "false")); - assertThat(providerMockedConstruction.constructed()).hasSize(0); - - DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); - props.applyClientConfiguration("account1", clientBuilder); - verify(clientBuilder, times(0)).sasToken(any()); - verify(clientBuilder).credential(any(TokenCredential.class)); - verify(clientBuilder, never()).credential(any(StorageSharedKeyCredential.class)); - } + AzureProperties props = + new AzureProperties( + ImmutableMap.of( + ADLS_REFRESH_CREDENTIALS_ENDPOINT, + "endpoint", + ADLS_REFRESH_CREDENTIALS_ENABLED, + "false")); + + DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); + props.applyClientConfiguration("account1", clientBuilder); + verify(clientBuilder, never()).addPolicy(any(VendedAzureSasCredentialPolicy.class)); } @Test diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java index 7e2ade5fe601..7465138f39b8 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java @@ -60,13 +60,12 @@ public void invalidOrMissingUri() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid URI: null"); - try (VendedAdlsCredentialProvider provider = + VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, "invalid uri"))) { - assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) - .isInstanceOf(RESTException.class) - .hasMessageStartingWith("Failed to create request URI from base invalid uri"); - } + ImmutableMap.of(VendedAdlsCredentialProvider.URI, "invalid uri")); + assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) + .isInstanceOf(RESTException.class) + .hasMessageStartingWith("Failed to create request URI from base invalid uri"); } @Test @@ -80,13 +79,12 @@ public void noADLSCredentials() { .withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAdlsCredentialProvider provider = + VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { - assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) - .isInstanceOf(IllegalStateException.class) - .hasMessage("Invalid ADLS Credentials for storage-account account1: empty"); - } + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI)); + assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid ADLS Credentials for storage-account account1: empty"); } @Test @@ -105,13 +103,12 @@ public void expirationNotSet() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAdlsCredentialProvider provider = + VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { - assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) - .isInstanceOf(IllegalStateException.class) - .hasMessage("Invalid ADLS Credentials: adls.sas-token-expires-at-ms.account1 not set"); - } + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI)); + assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid ADLS Credentials: adls.sas-token-expires-at-ms.account1 not set"); } @Test @@ -133,17 +130,16 @@ public void nonExpiredSasToken() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAdlsCredentialProvider provider = + VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { - String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); - assertThat(azureSasCredential) - .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI)); + String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); + assertThat(azureSasCredential) + .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); - for (int i = 0; i < 5; i++) { - // resolving credentials multiple times should not hit the credentials endpoint again - assertThat(provider.credentialForAccount(STORAGE_ACCOUNT)).isSameAs(azureSasCredential); - } + for (int i = 0; i < 5; i++) { + // resolving credentials multiple times should not hit the credentials endpoint again + assertThat(provider.credentialForAccount(STORAGE_ACCOUNT)).isSameAs(azureSasCredential); } mockServer.verify(mockRequest, VerificationTimes.once()); } @@ -167,18 +163,17 @@ public void expiredSasToken() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAdlsCredentialProvider provider = + VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { - String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); - assertThat(azureSasCredential) - .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI)); + String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); + assertThat(azureSasCredential) + .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); - // resolving credentials multiple times should hit the credentials endpoint again - String refreshedAzureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); - assertThat(refreshedAzureSasCredential) - .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); - } + // resolving credentials multiple times should hit the credentials endpoint again + String refreshedAzureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); + assertThat(refreshedAzureSasCredential) + .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); mockServer.verify(mockRequest, VerificationTimes.exactly(2)); } @@ -211,14 +206,13 @@ public void multipleADLSCredentialsPerStorageAccount() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAdlsCredentialProvider provider = + VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { - assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "Invalid ADLS Credentials: only one ADLS credential should exist per storage-account"); - } + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI)); + assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid ADLS Credentials: only one ADLS credential should exist per storage-account"); } @Test @@ -250,17 +244,16 @@ public void multipleStorageAccounts() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAdlsCredentialProvider provider = + VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { - String azureSasCredential1 = provider.credentialForAccount(STORAGE_ACCOUNT); - String azureSasCredential2 = provider.credentialForAccount(STORAGE_ACCOUNT_2); - assertThat(azureSasCredential1).isNotSameAs(azureSasCredential2); - assertThat(azureSasCredential1) - .isEqualTo(credential1.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); - assertThat(azureSasCredential2) - .isEqualTo(credential2.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT_2)); - } + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI)); + String azureSasCredential1 = provider.credentialForAccount(STORAGE_ACCOUNT); + String azureSasCredential2 = provider.credentialForAccount(STORAGE_ACCOUNT_2); + assertThat(azureSasCredential1).isNotSameAs(azureSasCredential2); + assertThat(azureSasCredential1) + .isEqualTo(credential1.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); + assertThat(azureSasCredential2) + .isEqualTo(credential2.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT_2)); } @Test @@ -282,19 +275,18 @@ public void serializableTest() throws IOException, ClassNotFoundException { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAdlsCredentialProvider provider = + VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { - String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); - assertThat(azureSasCredential) - .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI)); + String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); + assertThat(azureSasCredential) + .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); - VendedAdlsCredentialProvider deserializedProvider = TestHelpers.roundTripSerialize(provider); - String reGeneratedAzureSasCredential = - deserializedProvider.credentialForAccount(STORAGE_ACCOUNT); + VendedAdlsCredentialProvider deserializedProvider = TestHelpers.roundTripSerialize(provider); + String reGeneratedAzureSasCredential = + deserializedProvider.credentialForAccount(STORAGE_ACCOUNT); - assertThat(azureSasCredential).isNotSameAs(reGeneratedAzureSasCredential); - } + assertThat(azureSasCredential).isNotSameAs(reGeneratedAzureSasCredential); mockServer.verify(mockRequest, VerificationTimes.exactly(2)); } } From 9fc4f237ef19d37740976665b7d00d8c22d8a2da Mon Sep 17 00:00:00 2001 From: Mohan Vamsi Date: Thu, 27 Mar 2025 01:08:28 +0530 Subject: [PATCH 14/21] address review comments. --- .../apache/iceberg/azure/AzureProperties.java | 50 +++++++++++-------- .../iceberg/azure/adlsv2/ADLSFileIO.java | 7 +++ .../iceberg/azure/AzurePropertiesTest.java | 13 +++-- 3 files changed, 45 insertions(+), 25 deletions(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java index c932e04e4c5e..f03c28ab6a75 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java +++ b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java @@ -27,12 +27,12 @@ import java.util.Optional; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.azure.adlsv2.VendedAdlsCredentialProvider; -import org.apache.iceberg.azure.adlsv2.VendedAzureSasCredentialPolicy; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.rest.RESTUtil; import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.SerializableMap; public class AzureProperties implements Serializable { public static final String ADLS_SAS_TOKEN_PREFIX = "adls.sas-token."; @@ -58,8 +58,9 @@ public class AzureProperties implements Serializable { private Map.Entry namedKeyCreds; private Integer adlsReadBlockSize; private Long adlsWriteBlockSize; - - private VendedAdlsCredentialProvider vendedAdlsCredentialProvider; + private String adlsRefreshCredentialsEndpoint; + private boolean adlsRefreshCredentialsEnabled; + private Map allProperties; public AzureProperties() {} @@ -85,19 +86,13 @@ public AzureProperties(Map properties) { if (properties.containsKey(ADLS_WRITE_BLOCK_SIZE)) { this.adlsWriteBlockSize = Long.parseLong(properties.get(ADLS_WRITE_BLOCK_SIZE)); } - String adlsRefreshCredentialsEndpoint = + this.adlsRefreshCredentialsEndpoint = RESTUtil.resolveEndpoint( properties.get(CatalogProperties.URI), properties.get(ADLS_REFRESH_CREDENTIALS_ENDPOINT)); - boolean adlsRefreshCredentialsEnabled = + this.adlsRefreshCredentialsEnabled = PropertyUtil.propertyAsBoolean(properties, ADLS_REFRESH_CREDENTIALS_ENABLED, true); - if (adlsRefreshCredentialsEnabled && !Strings.isNullOrEmpty(adlsRefreshCredentialsEndpoint)) { - Map credentialProviderProperties = Maps.newHashMap(properties); - credentialProviderProperties.put( - VendedAdlsCredentialProvider.URI, adlsRefreshCredentialsEndpoint); - this.vendedAdlsCredentialProvider = - new VendedAdlsCredentialProvider(credentialProviderProperties); - } + this.allProperties = SerializableMap.copyOf(properties); } public Optional adlsReadBlockSize() { @@ -108,6 +103,17 @@ public Optional adlsWriteBlockSize() { return Optional.ofNullable(adlsWriteBlockSize); } + public Optional vendedAdlsCredentialProvider() { + if (adlsRefreshCredentialsEnabled && !Strings.isNullOrEmpty(adlsRefreshCredentialsEndpoint)) { + Map credentialProviderProperties = Maps.newHashMap(allProperties); + credentialProviderProperties.put( + VendedAdlsCredentialProvider.URI, adlsRefreshCredentialsEndpoint); + return Optional.of(new VendedAdlsCredentialProvider(credentialProviderProperties)); + } else { + return Optional.empty(); + } + } + /** * Applies configuration to the {@link DataLakeFileSystemClientBuilder} to provide the endpoint * and credentials required to create an instance of the client. @@ -118,16 +124,16 @@ public Optional adlsWriteBlockSize() { * @param builder the builder instance */ public void applyClientConfiguration(String account, DataLakeFileSystemClientBuilder builder) { - String sasToken = adlsSasTokens.get(account); - if (vendedAdlsCredentialProvider != null) { - builder.addPolicy(new VendedAzureSasCredentialPolicy(account, vendedAdlsCredentialProvider)); - } else if (sasToken != null && !sasToken.isEmpty()) { - builder.sasToken(sasToken); - } else if (namedKeyCreds != null) { - builder.credential( - new StorageSharedKeyCredential(namedKeyCreds.getKey(), namedKeyCreds.getValue())); - } else { - builder.credential(new DefaultAzureCredentialBuilder().build()); + if (!adlsRefreshCredentialsEnabled || Strings.isNullOrEmpty(adlsRefreshCredentialsEndpoint)) { + String sasToken = adlsSasTokens.get(account); + if (sasToken != null && !sasToken.isEmpty()) { + builder.sasToken(sasToken); + } else if (namedKeyCreds != null) { + builder.credential( + new StorageSharedKeyCredential(namedKeyCreds.getKey(), namedKeyCreds.getValue())); + } else { + builder.credential(new DefaultAzureCredentialBuilder().build()); + } } // apply connection string last so its parameters take precedence, e.g. SAS token diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java index e1bf21f69dc8..d2352c248a80 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java @@ -27,6 +27,7 @@ import com.azure.storage.file.datalake.models.ListPathsOptions; import java.util.Collections; import java.util.Map; +import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.azure.AzureProperties; import org.apache.iceberg.common.DynConstructors; @@ -55,6 +56,7 @@ public class ADLSFileIO implements DelegateFileIO { private AzureProperties azureProperties; private MetricsContext metrics = MetricsContext.nullMetrics(); private SerializableMap properties; + private VendedAdlsCredentialProvider vendedAdlsCredentialProvider; /** * No-arg constructor to load the FileIO dynamically. @@ -111,6 +113,9 @@ DataLakeFileSystemClient client(ADLSLocation location) { new DataLakeFileSystemClientBuilder().httpClient(HTTP); location.container().ifPresent(clientBuilder::fileSystemName); + Optional.ofNullable(vendedAdlsCredentialProvider) + .map(p -> new VendedAzureSasCredentialPolicy(location.host(), p)) + .ifPresent(clientBuilder::addPolicy); azureProperties.applyClientConfiguration(location.host(), clientBuilder); return clientBuilder.buildClient(); @@ -126,6 +131,8 @@ public void initialize(Map props) { this.properties = SerializableMap.copyOf(props); this.azureProperties = new AzureProperties(properties); initMetrics(properties); + this.vendedAdlsCredentialProvider = + this.azureProperties.vendedAdlsCredentialProvider().orElse(null); } @SuppressWarnings("CatchBlockLogException") diff --git a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java index 4e0c4ea39782..77a1a4d1981c 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java @@ -36,10 +36,12 @@ import com.azure.core.credential.AzureSasCredential; import com.azure.core.credential.TokenCredential; +import com.azure.identity.DefaultAzureCredential; import com.azure.storage.common.StorageSharedKeyCredential; import com.azure.storage.file.datalake.DataLakeFileSystemClientBuilder; +import java.util.Optional; import org.apache.iceberg.TestHelpers; -import org.apache.iceberg.azure.adlsv2.VendedAzureSasCredentialPolicy; +import org.apache.iceberg.azure.adlsv2.VendedAdlsCredentialProvider; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; @@ -82,11 +84,13 @@ public void testWithRefreshCredentialsEndpoint() { DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); props.applyClientConfiguration("account1", clientBuilder); + Optional vendedAdlsCredentialProvider = + props.vendedAdlsCredentialProvider(); verify(clientBuilder, never()).credential(any(AzureSasCredential.class)); verify(clientBuilder, never()).sasToken(any()); verify(clientBuilder, never()).credential(any(StorageSharedKeyCredential.class)); - verify(clientBuilder, times(1)).addPolicy(any(VendedAzureSasCredentialPolicy.class)); + assertThat(vendedAdlsCredentialProvider).isPresent(); } @Test @@ -101,7 +105,10 @@ public void testWithRefreshCredentialsEndpointDisabled() { DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); props.applyClientConfiguration("account1", clientBuilder); - verify(clientBuilder, never()).addPolicy(any(VendedAzureSasCredentialPolicy.class)); + Optional vendedAdlsCredentialProvider = + props.vendedAdlsCredentialProvider(); + verify(clientBuilder).credential(any(DefaultAzureCredential.class)); + assertThat(vendedAdlsCredentialProvider).isEmpty(); } @Test From b398961184bb163f57b1fbc096e9724f0aeb01ec Mon Sep 17 00:00:00 2001 From: Mohan Vamsi Date: Thu, 27 Mar 2025 01:44:11 +0530 Subject: [PATCH 15/21] ensure OAuth token in present in properties. --- .../main/java/org/apache/iceberg/azure/AzureProperties.java | 4 ++++ .../iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java index f03c28ab6a75..6e74046bcdbd 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java +++ b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java @@ -31,6 +31,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.rest.RESTUtil; +import org.apache.iceberg.rest.auth.OAuth2Properties; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.SerializableMap; @@ -108,6 +109,9 @@ public Optional vendedAdlsCredentialProvider() { Map credentialProviderProperties = Maps.newHashMap(allProperties); credentialProviderProperties.put( VendedAdlsCredentialProvider.URI, adlsRefreshCredentialsEndpoint); + Optional.ofNullable(allProperties.get(OAuth2Properties.TOKEN)) + .ifPresent( + token -> credentialProviderProperties.putIfAbsent(OAuth2Properties.TOKEN, token)); return Optional.of(new VendedAdlsCredentialProvider(credentialProviderProperties)); } else { return Optional.empty(); diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java index e6d37e57a38a..2415cbf05d6d 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java @@ -57,7 +57,7 @@ public VendedAdlsCredentialProvider(Map properties) { this.properties = SerializableMap.copyOf(properties); } - public String credentialForAccount(String storageAccount) { + String credentialForAccount(String storageAccount) { return sasCredentialByAccount() .computeIfAbsent( storageAccount, From 3f33808f7dd86a280deb544008464b15bd9236fd Mon Sep 17 00:00:00 2001 From: Mohan Vamsi Date: Thu, 27 Mar 2025 02:03:06 +0530 Subject: [PATCH 16/21] Handle Auth manager for ADLS fileio. --- .../iceberg/azure/adlsv2/ADLSFileIO.java | 8 +++ .../adlsv2/VendedAdlsCredentialProvider.java | 69 ++++++++++--------- 2 files changed, 45 insertions(+), 32 deletions(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java index d2352c248a80..90daacf2b64b 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java @@ -219,4 +219,12 @@ public void deletePrefix(String prefix) { } } } + + @Override + public void close() { + if (vendedAdlsCredentialProvider != null) { + vendedAdlsCredentialProvider.close(); + } + DelegateFileIO.super.close(); + } } diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java index 2415cbf05d6d..dd725c48d9d0 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java @@ -22,34 +22,36 @@ import com.azure.core.credential.SimpleTokenCache; import java.io.IOException; import java.io.Serializable; +import java.io.UncheckedIOException; import java.time.Instant; import java.time.ZoneOffset; import java.util.List; import java.util.Map; import java.util.stream.Collectors; import org.apache.iceberg.azure.AzureProperties; +import org.apache.iceberg.io.CloseableGroup; 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.HTTPHeaders; import org.apache.iceberg.rest.RESTClient; +import org.apache.iceberg.rest.auth.AuthManager; +import org.apache.iceberg.rest.auth.AuthManagers; import org.apache.iceberg.rest.auth.AuthSession; -import org.apache.iceberg.rest.auth.DefaultAuthSession; -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.SerializableMap; import reactor.core.publisher.Mono; -public class VendedAdlsCredentialProvider implements Serializable { +public class VendedAdlsCredentialProvider implements Serializable, AutoCloseable { public static final String URI = "credentials.uri"; private final SerializableMap properties; private transient volatile Map sasCredentialByAccount; - private transient volatile AuthSession authSession; + private transient volatile HTTPClient client; + private transient AuthManager authManager; + private transient AuthSession authSession; public VendedAdlsCredentialProvider(Map properties) { Preconditions.checkArgument(null != properties, "Invalid properties: null"); @@ -111,39 +113,28 @@ private Map sasCredentialByAccount() { } private RESTClient httpClient() { - return HTTPClient.builder(properties) - .uri(properties.get(URI)) - .withAuthSession(authSession()) - .build(); - } - - private AuthSession authSession() { - if (this.authSession == null) { + if (null == client) { synchronized (this) { - if (this.authSession == null) { - this.authSession = - DefaultAuthSession.of( - HTTPHeaders.of(OAuth2Util.authHeaders(properties.get(OAuth2Properties.TOKEN)))); + if (null == client) { + authManager = AuthManagers.loadAuthManager("adls-credentials-refresh", properties); + HTTPClient httpClient = HTTPClient.builder(properties).uri(properties.get(URI)).build(); + authSession = authManager.catalogSession(httpClient, properties); + client = httpClient.withAuthSession(authSession); } } } - return this.authSession; + + return client; } private LoadCredentialsResponse fetchCredentials() { - LoadCredentialsResponse response; - try (RESTClient client = httpClient()) { - response = - client.get( - properties.get(URI), - null, - LoadCredentialsResponse.class, - Map.of(), - ErrorHandlers.defaultErrorHandler()); - } catch (IOException e) { - throw new RuntimeException(e); - } - return response; + return httpClient() + .get( + properties.get(URI), + null, + LoadCredentialsResponse.class, + Map.of(), + ErrorHandlers.defaultErrorHandler()); } private void checkCredential(Credential credential, String property) { @@ -152,4 +143,18 @@ private void checkCredential(Credential credential, String property) { "Invalid ADLS Credentials: %s not set", property); } + + @Override + public void close() { + CloseableGroup closeableGroup = new CloseableGroup(); + closeableGroup.addCloseable(authSession); + closeableGroup.addCloseable(authManager); + closeableGroup.addCloseable(client); + closeableGroup.setSuppressCloseFailure(true); + try { + closeableGroup.close(); + } catch (IOException e) { + throw new UncheckedIOException("Failed to close the VendedAdlsCredentialProvider", e); + } + } } From 0f491c8e681fb4da6f771d2d9cb2cfa31fda4453 Mon Sep 17 00:00:00 2001 From: Mohan Vamsi Date: Thu, 27 Mar 2025 02:11:41 +0530 Subject: [PATCH 17/21] Update VendedAdlsCredentialProviderTest with try with resource to handle autoclosable. --- .../VendedAdlsCredentialProviderTest.java | 123 ++++++++++-------- 1 file changed, 67 insertions(+), 56 deletions(-) diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java index 7465138f39b8..ee3edb6ae6aa 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java @@ -60,12 +60,13 @@ public void invalidOrMissingUri() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid URI: null"); - VendedAdlsCredentialProvider provider = + try (VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, "invalid uri")); - assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) - .isInstanceOf(RESTException.class) - .hasMessageStartingWith("Failed to create request URI from base invalid uri"); + ImmutableMap.of(VendedAdlsCredentialProvider.URI, "invalid uri"))) { + assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) + .isInstanceOf(RESTException.class) + .hasMessageStartingWith("Failed to create request URI from base invalid uri"); + } } @Test @@ -79,12 +80,13 @@ public void noADLSCredentials() { .withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - VendedAdlsCredentialProvider provider = + try (VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI)); - assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) - .isInstanceOf(IllegalStateException.class) - .hasMessage("Invalid ADLS Credentials for storage-account account1: empty"); + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid ADLS Credentials for storage-account account1: empty"); + } } @Test @@ -103,12 +105,13 @@ public void expirationNotSet() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - VendedAdlsCredentialProvider provider = + try (VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI)); - assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) - .isInstanceOf(IllegalStateException.class) - .hasMessage("Invalid ADLS Credentials: adls.sas-token-expires-at-ms.account1 not set"); + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) + .isInstanceOf(IllegalStateException.class) + .hasMessage("Invalid ADLS Credentials: adls.sas-token-expires-at-ms.account1 not set"); + } } @Test @@ -130,17 +133,19 @@ public void nonExpiredSasToken() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - VendedAdlsCredentialProvider provider = + try (VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI)); - String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); - assertThat(azureSasCredential) - .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); + assertThat(azureSasCredential) + .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); - for (int i = 0; i < 5; i++) { - // resolving credentials multiple times should not hit the credentials endpoint again - assertThat(provider.credentialForAccount(STORAGE_ACCOUNT)).isSameAs(azureSasCredential); + for (int i = 0; i < 5; i++) { + // resolving credentials multiple times should not hit the credentials endpoint again + assertThat(provider.credentialForAccount(STORAGE_ACCOUNT)).isSameAs(azureSasCredential); + } } + mockServer.verify(mockRequest, VerificationTimes.once()); } @@ -163,17 +168,19 @@ public void expiredSasToken() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - VendedAdlsCredentialProvider provider = + try (VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI)); - String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); - assertThat(azureSasCredential) - .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); + assertThat(azureSasCredential) + .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); + + // resolving credentials multiple times should hit the credentials endpoint again + String refreshedAzureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); + assertThat(refreshedAzureSasCredential) + .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); + } - // resolving credentials multiple times should hit the credentials endpoint again - String refreshedAzureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); - assertThat(refreshedAzureSasCredential) - .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); mockServer.verify(mockRequest, VerificationTimes.exactly(2)); } @@ -206,13 +213,14 @@ public void multipleADLSCredentialsPerStorageAccount() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - VendedAdlsCredentialProvider provider = + try (VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI)); - assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "Invalid ADLS Credentials: only one ADLS credential should exist per storage-account"); + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid ADLS Credentials: only one ADLS credential should exist per storage-account"); + } } @Test @@ -244,16 +252,17 @@ public void multipleStorageAccounts() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - VendedAdlsCredentialProvider provider = + try (VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI)); - String azureSasCredential1 = provider.credentialForAccount(STORAGE_ACCOUNT); - String azureSasCredential2 = provider.credentialForAccount(STORAGE_ACCOUNT_2); - assertThat(azureSasCredential1).isNotSameAs(azureSasCredential2); - assertThat(azureSasCredential1) - .isEqualTo(credential1.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); - assertThat(azureSasCredential2) - .isEqualTo(credential2.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT_2)); + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + String azureSasCredential1 = provider.credentialForAccount(STORAGE_ACCOUNT); + String azureSasCredential2 = provider.credentialForAccount(STORAGE_ACCOUNT_2); + assertThat(azureSasCredential1).isNotSameAs(azureSasCredential2); + assertThat(azureSasCredential1) + .isEqualTo(credential1.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); + assertThat(azureSasCredential2) + .isEqualTo(credential2.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT_2)); + } } @Test @@ -275,18 +284,20 @@ public void serializableTest() throws IOException, ClassNotFoundException { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - VendedAdlsCredentialProvider provider = + try (VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI)); - String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); - assertThat(azureSasCredential) - .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); + assertThat(azureSasCredential) + .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); - VendedAdlsCredentialProvider deserializedProvider = TestHelpers.roundTripSerialize(provider); - String reGeneratedAzureSasCredential = - deserializedProvider.credentialForAccount(STORAGE_ACCOUNT); + VendedAdlsCredentialProvider deserializedProvider = TestHelpers.roundTripSerialize(provider); + String reGeneratedAzureSasCredential = + deserializedProvider.credentialForAccount(STORAGE_ACCOUNT); + + assertThat(azureSasCredential).isNotSameAs(reGeneratedAzureSasCredential); + } - assertThat(azureSasCredential).isNotSameAs(reGeneratedAzureSasCredential); mockServer.verify(mockRequest, VerificationTimes.exactly(2)); } } From 28e7ba6ff792480757a2666b0cfdaca3fd348099 Mon Sep 17 00:00:00 2001 From: Mohan Vamsi Date: Thu, 27 Mar 2025 16:48:48 +0530 Subject: [PATCH 18/21] Apply review comments. --- .../apache/iceberg/azure/AzureProperties.java | 4 -- .../iceberg/azure/adlsv2/ADLSFileIO.java | 1 + .../adlsv2/VendedAdlsCredentialProvider.java | 13 ++++- .../VendedAzureSasCredentialPolicy.java | 4 +- .../VendedAdlsCredentialProviderTest.java | 56 ++++++++++--------- 5 files changed, 44 insertions(+), 34 deletions(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java index 6e74046bcdbd..f03c28ab6a75 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java +++ b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java @@ -31,7 +31,6 @@ import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.rest.RESTUtil; -import org.apache.iceberg.rest.auth.OAuth2Properties; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.SerializableMap; @@ -109,9 +108,6 @@ public Optional vendedAdlsCredentialProvider() { Map credentialProviderProperties = Maps.newHashMap(allProperties); credentialProviderProperties.put( VendedAdlsCredentialProvider.URI, adlsRefreshCredentialsEndpoint); - Optional.ofNullable(allProperties.get(OAuth2Properties.TOKEN)) - .ifPresent( - token -> credentialProviderProperties.putIfAbsent(OAuth2Properties.TOKEN, token)); return Optional.of(new VendedAdlsCredentialProvider(credentialProviderProperties)); } else { return Optional.empty(); diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java index 90daacf2b64b..234f8434b13e 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java @@ -225,6 +225,7 @@ public void close() { if (vendedAdlsCredentialProvider != null) { vendedAdlsCredentialProvider.close(); } + DelegateFileIO.super.close(); } } diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java index dd725c48d9d0..3a03a5824c58 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.azure.AzureProperties; import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -48,6 +49,8 @@ public class VendedAdlsCredentialProvider implements Serializable, AutoCloseable public static final String URI = "credentials.uri"; private final SerializableMap properties; + private final String credentialsEndpoint; + private final String catalogEndpoint; private transient volatile Map sasCredentialByAccount; private transient volatile HTTPClient client; private transient AuthManager authManager; @@ -55,8 +58,12 @@ public class VendedAdlsCredentialProvider implements Serializable, AutoCloseable public VendedAdlsCredentialProvider(Map properties) { Preconditions.checkArgument(null != properties, "Invalid properties: null"); - Preconditions.checkArgument(null != properties.get(URI), "Invalid URI: null"); + Preconditions.checkArgument(null != properties.get(URI), "Invalid credentials endpoint: null"); + Preconditions.checkArgument( + null != properties.get(CatalogProperties.URI), "Invalid catalog endpoint: null"); this.properties = SerializableMap.copyOf(properties); + this.credentialsEndpoint = properties.get(URI); + this.catalogEndpoint = properties.get(CatalogProperties.URI); } String credentialForAccount(String storageAccount) { @@ -117,7 +124,7 @@ private RESTClient httpClient() { synchronized (this) { if (null == client) { authManager = AuthManagers.loadAuthManager("adls-credentials-refresh", properties); - HTTPClient httpClient = HTTPClient.builder(properties).uri(properties.get(URI)).build(); + HTTPClient httpClient = HTTPClient.builder(properties).uri(catalogEndpoint).build(); authSession = authManager.catalogSession(httpClient, properties); client = httpClient.withAuthSession(authSession); } @@ -130,7 +137,7 @@ private RESTClient httpClient() { private LoadCredentialsResponse fetchCredentials() { return httpClient() .get( - properties.get(URI), + credentialsEndpoint, null, LoadCredentialsResponse.class, Map.of(), diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicy.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicy.java index 3e6c72c37b3d..b5ac3d9c405d 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicy.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAzureSasCredentialPolicy.java @@ -27,13 +27,13 @@ import com.azure.core.http.policy.HttpPipelinePolicy; import reactor.core.publisher.Mono; -public class VendedAzureSasCredentialPolicy implements HttpPipelinePolicy { +class VendedAzureSasCredentialPolicy implements HttpPipelinePolicy { private final String account; private final VendedAdlsCredentialProvider vendedAdlsCredentialProvider; private AzureSasCredential azureSasCredential; private AzureSasCredentialPolicy azureSasCredentialPolicy; - public VendedAzureSasCredentialPolicy( + VendedAzureSasCredentialPolicy( String account, VendedAdlsCredentialProvider vendedAdlsCredentialProvider) { this.account = account; this.vendedAdlsCredentialProvider = vendedAdlsCredentialProvider; diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java index ee3edb6ae6aa..f17b1ea5a685 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProviderTest.java @@ -29,6 +29,8 @@ import java.io.IOException; import java.time.Instant; import java.time.temporal.ChronoUnit; +import java.util.Map; +import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.TestHelpers; import org.apache.iceberg.exceptions.RESTException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -44,28 +46,46 @@ public class VendedAdlsCredentialProviderTest extends BaseVendedCredentialsTest { private static final String CREDENTIALS_URI = String.format("%s%s", baseUri, "/v1/credentials"); + private static final String CATALOG_URI = String.format("%s%s", baseUri, "/v1/"); private static final String STORAGE_ACCOUNT = "account1"; private static final String CREDENTIAL_PREFIX = "abfs://container@account1.dfs.core.windows.net/dir"; private static final String STORAGE_ACCOUNT_2 = "account2"; private static final String CREDENTIAL_PREFIX_2 = "abfs://container@account2.dfs.core.windows.net/dir"; + private static final Map PROPERTIES = + ImmutableMap.of( + VendedAdlsCredentialProvider.URI, CREDENTIALS_URI, CatalogProperties.URI, CATALOG_URI); @Test public void invalidOrMissingUri() { assertThatThrownBy(() -> new VendedAdlsCredentialProvider(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid properties: null"); - assertThatThrownBy(() -> new VendedAdlsCredentialProvider(ImmutableMap.of())) + assertThatThrownBy( + () -> + new VendedAdlsCredentialProvider( + ImmutableMap.of(CatalogProperties.URI, CATALOG_URI))) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid URI: null"); + .hasMessage("Invalid credentials endpoint: null"); + assertThatThrownBy( + () -> + new VendedAdlsCredentialProvider( + ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid catalog endpoint: null"); try (VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, "invalid uri"))) { + ImmutableMap.of( + VendedAdlsCredentialProvider.URI, + "invalid uri", + CatalogProperties.URI, + CATALOG_URI))) { assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) .isInstanceOf(RESTException.class) - .hasMessageStartingWith("Failed to create request URI from base invalid uri"); + .hasMessageStartingWith( + "Failed to create request URI from base %sinvalid uri", CATALOG_URI); } } @@ -80,9 +100,7 @@ public void noADLSCredentials() { .withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAdlsCredentialProvider provider = - new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + try (VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider(PROPERTIES)) { assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) .isInstanceOf(IllegalStateException.class) .hasMessage("Invalid ADLS Credentials for storage-account account1: empty"); @@ -105,9 +123,7 @@ public void expirationNotSet() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAdlsCredentialProvider provider = - new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + try (VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider(PROPERTIES)) { assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) .isInstanceOf(IllegalStateException.class) .hasMessage("Invalid ADLS Credentials: adls.sas-token-expires-at-ms.account1 not set"); @@ -133,9 +149,7 @@ public void nonExpiredSasToken() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAdlsCredentialProvider provider = - new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + try (VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider(PROPERTIES)) { String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); assertThat(azureSasCredential) .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); @@ -168,9 +182,7 @@ public void expiredSasToken() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAdlsCredentialProvider provider = - new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + try (VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider(PROPERTIES)) { String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); assertThat(azureSasCredential) .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); @@ -213,9 +225,7 @@ public void multipleADLSCredentialsPerStorageAccount() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAdlsCredentialProvider provider = - new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + try (VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider(PROPERTIES)) { assertThatThrownBy(() -> provider.credentialForAccount(STORAGE_ACCOUNT)) .isInstanceOf(IllegalStateException.class) .hasMessage( @@ -252,9 +262,7 @@ public void multipleStorageAccounts() { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAdlsCredentialProvider provider = - new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + try (VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider(PROPERTIES)) { String azureSasCredential1 = provider.credentialForAccount(STORAGE_ACCOUNT); String azureSasCredential2 = provider.credentialForAccount(STORAGE_ACCOUNT_2); assertThat(azureSasCredential1).isNotSameAs(azureSasCredential2); @@ -284,9 +292,7 @@ public void serializableTest() throws IOException, ClassNotFoundException { response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); mockServer.when(mockRequest).respond(mockResponse); - try (VendedAdlsCredentialProvider provider = - new VendedAdlsCredentialProvider( - ImmutableMap.of(VendedAdlsCredentialProvider.URI, CREDENTIALS_URI))) { + try (VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider(PROPERTIES)) { String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT); assertThat(azureSasCredential) .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); From d419ecb3c75f19a211c179f0e7ef18a0152df03b Mon Sep 17 00:00:00 2001 From: Mohan Vamsi Date: Thu, 27 Mar 2025 16:56:43 +0530 Subject: [PATCH 19/21] Fix test case. --- .../org/apache/iceberg/azure/AzurePropertiesTest.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java index 77a1a4d1981c..153c088c4f84 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java @@ -40,6 +40,7 @@ import com.azure.storage.common.StorageSharedKeyCredential; import com.azure.storage.file.datalake.DataLakeFileSystemClientBuilder; import java.util.Optional; +import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.TestHelpers; import org.apache.iceberg.azure.adlsv2.VendedAdlsCredentialProvider; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -80,7 +81,12 @@ public void testWithSasToken() { @Test public void testWithRefreshCredentialsEndpoint() { AzureProperties props = - new AzureProperties(ImmutableMap.of(ADLS_REFRESH_CREDENTIALS_ENDPOINT, "endpoint")); + new AzureProperties( + ImmutableMap.of( + ADLS_REFRESH_CREDENTIALS_ENDPOINT, + "endpoint", + CatalogProperties.URI, + "catalog-endpoint")); DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); props.applyClientConfiguration("account1", clientBuilder); From da31d42d083ae62897645c819ffd7fafd30bd232 Mon Sep 17 00:00:00 2001 From: ChaladiMohanVamsi Date: Thu, 27 Mar 2025 21:57:57 +0530 Subject: [PATCH 20/21] Update azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java Co-authored-by: Daniel Weeks --- .../main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java index 234f8434b13e..9396a1ff695f 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java @@ -131,8 +131,8 @@ public void initialize(Map props) { this.properties = SerializableMap.copyOf(props); this.azureProperties = new AzureProperties(properties); initMetrics(properties); - this.vendedAdlsCredentialProvider = - this.azureProperties.vendedAdlsCredentialProvider().orElse(null); + this.azureProperties.vendedAdlsCredentialProvider() + .ifPresent((provider -> this.vendedAdlsCredentialProvider = provider)); } @SuppressWarnings("CatchBlockLogException") From ec3e6799e0b50d38d6113ae347cad1986dc14af4 Mon Sep 17 00:00:00 2001 From: Mohan Vamsi Date: Thu, 27 Mar 2025 21:59:55 +0530 Subject: [PATCH 21/21] Fix spotless violations. --- .../main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java index 9396a1ff695f..2e7f77aca90e 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSFileIO.java @@ -131,7 +131,8 @@ public void initialize(Map props) { this.properties = SerializableMap.copyOf(props); this.azureProperties = new AzureProperties(properties); initMetrics(properties); - this.azureProperties.vendedAdlsCredentialProvider() + this.azureProperties + .vendedAdlsCredentialProvider() .ifPresent((provider -> this.vendedAdlsCredentialProvider = provider)); }