This is an automated email from the ASF dual-hosted git repository.

kevinjqliu pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/main by this push:
     new e7a5a87f26 Azure: Avoid depending on KeyWrapAlgorithm in 
AzureProperties (#16186)
e7a5a87f26 is described below

commit e7a5a87f26f9de5b200254155aa037368b13a29c
Author: Yuya Ebihara <[email protected]>
AuthorDate: Sat May 9 03:39:59 2026 +0900

    Azure: Avoid depending on KeyWrapAlgorithm in AzureProperties (#16186)
    
    * Azure: Avoid depending on KeyWrapAlgorithm in AzureProperties
    
    * fixup! Azure: Avoid depending on KeyWrapAlgorithm in AzureProperties
---
 .../main/java/org/apache/iceberg/azure/AzureProperties.java   | 11 ++++++-----
 .../iceberg/azure/keymanagement/AzureKeyManagementClient.java |  3 ++-
 2 files changed, 8 insertions(+), 6 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 73e99e0292..383bec3011 100644
--- a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java
+++ b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java
@@ -21,7 +21,6 @@ package org.apache.iceberg.azure;
 import com.azure.core.credential.AccessToken;
 import com.azure.core.credential.TokenCredential;
 import com.azure.core.credential.TokenRequestContext;
-import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
 import com.azure.storage.common.StorageSharedKeyCredential;
 import com.azure.storage.file.datalake.DataLakeFileSystemClientBuilder;
 import java.io.Serializable;
@@ -53,6 +52,9 @@ public class AzureProperties implements Serializable {
   public static final String AZURE_KEYVAULT_KEY_WRAP_ALGORITHM =
       "azure.keyvault.key-wrap-algorithm";
 
+  // Must match KeyWrapAlgorithm.RSA_OAEP_256.getValue() from 
azure-security-keyvault-keys
+  private static final String DEFAULT_KEY_WRAP_ALGORITHM = "RSA-OAEP-256";
+
   /**
    * Configure the ADLS token credential provider used to get {@link 
TokenCredential}. A fully
    * qualified concrete class with package that implements the {@link 
AdlsTokenCredentialProvider}
@@ -136,8 +138,7 @@ public class AzureProperties implements Serializable {
 
     this.keyWrapAlgorithm =
         properties.getOrDefault(
-            AzureProperties.AZURE_KEYVAULT_KEY_WRAP_ALGORITHM,
-            KeyWrapAlgorithm.RSA_OAEP_256.getValue());
+            AzureProperties.AZURE_KEYVAULT_KEY_WRAP_ALGORITHM, 
DEFAULT_KEY_WRAP_ALGORITHM);
   }
 
   public Optional<Integer> adlsReadBlockSize() {
@@ -204,8 +205,8 @@ public class AzureProperties implements Serializable {
     }
   }
 
-  public KeyWrapAlgorithm keyWrapAlgorithm() {
-    return KeyWrapAlgorithm.fromString(this.keyWrapAlgorithm);
+  public String keyWrapAlgorithm() {
+    return this.keyWrapAlgorithm;
   }
 
   public Optional<String> keyVaultUrl() {
diff --git 
a/azure/src/main/java/org/apache/iceberg/azure/keymanagement/AzureKeyManagementClient.java
 
b/azure/src/main/java/org/apache/iceberg/azure/keymanagement/AzureKeyManagementClient.java
index 66bf0678bc..498c432212 100644
--- 
a/azure/src/main/java/org/apache/iceberg/azure/keymanagement/AzureKeyManagementClient.java
+++ 
b/azure/src/main/java/org/apache/iceberg/azure/keymanagement/AzureKeyManagementClient.java
@@ -80,7 +80,8 @@ public class AzureKeyManagementClient implements 
KeyManagementClient {
               keyClientBuilder
                   
.credential(AdlsTokenCredentialProviders.from(allProperties).credential())
                   .buildClient();
-          KeyWrapAlgorithm keyWrapAlgorithm = 
azureProperties.keyWrapAlgorithm();
+          KeyWrapAlgorithm keyWrapAlgorithm =
+              KeyWrapAlgorithm.fromString(azureProperties.keyWrapAlgorithm());
           state = new ClientState(keyClient, keyWrapAlgorithm);
         }
       }

Reply via email to