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

etudenhoefner 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 c88e9422bb Azure: Make AzureProperties w/ shared-key creds 
serializable (#10045)
c88e9422bb is described below

commit c88e9422bb5a2a27ff6ee71c26740f20dd00ea29
Author: Robert Stupp <[email protected]>
AuthorDate: Wed Jun 26 15:41:14 2024 +0200

    Azure: Make AzureProperties w/ shared-key creds serializable (#10045)
    
    As `StorageSharedKeyCredential` is not serializable, shared key auth 
doesn't work with Spark.
---
 .../org/apache/iceberg/azure/AzureProperties.java  |  9 ++++----
 .../apache/iceberg/azure/AzurePropertiesTest.java  | 24 ++++++++++++++++++++++
 2 files changed, 29 insertions(+), 4 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 8fcb0a098f..2d363cbc52 100644
--- a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java
+++ b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java
@@ -26,6 +26,7 @@ import java.util.Collections;
 import java.util.Map;
 import java.util.Optional;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.util.PropertyUtil;
 
 public class AzureProperties implements Serializable {
@@ -38,7 +39,7 @@ public class AzureProperties implements Serializable {
 
   private Map<String, String> adlsSasTokens = Collections.emptyMap();
   private Map<String, String> adlsConnectionStrings = Collections.emptyMap();
-  private StorageSharedKeyCredential namedKeyCreds;
+  private Map.Entry<String, String> namedKeyCreds;
   private Integer adlsReadBlockSize;
   private Long adlsWriteBlockSize;
 
@@ -57,8 +58,7 @@ public class AzureProperties implements Serializable {
           "Azure authentication: shared-key requires both %s and %s",
           ADLS_SHARED_KEY_ACCOUNT_NAME,
           ADLS_SHARED_KEY_ACCOUNT_KEY);
-      this.namedKeyCreds =
-          new StorageSharedKeyCredential(sharedKeyAccountName, 
sharedKeyAccountKey);
+      this.namedKeyCreds = Maps.immutableEntry(sharedKeyAccountName, 
sharedKeyAccountKey);
     }
 
     if (properties.containsKey(ADLS_READ_BLOCK_SIZE)) {
@@ -82,7 +82,8 @@ public class AzureProperties implements Serializable {
     if (sasToken != null && !sasToken.isEmpty()) {
       builder.sasToken(sasToken);
     } else if (namedKeyCreds != null) {
-      builder.credential(namedKeyCreds);
+      builder.credential(
+          new StorageSharedKeyCredential(namedKeyCreds.getKey(), 
namedKeyCreds.getValue()));
     } else {
       builder.credential(new DefaultAzureCredentialBuilder().build());
     }
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 32f491119e..6b8287c44e 100644
--- a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java
+++ b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java
@@ -18,8 +18,13 @@
  */
 package org.apache.iceberg.azure;
 
+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_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.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;
@@ -30,11 +35,30 @@ import static org.mockito.Mockito.verify;
 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.relocated.com.google.common.collect.ImmutableMap;
 import org.junit.jupiter.api.Test;
 
 public class AzurePropertiesTest {
 
+  @Test
+  public void testSerializable() throws Exception {
+    AzureProperties props =
+        new AzureProperties(
+            ImmutableMap.<String, String>builder()
+                .put(ADLS_SAS_TOKEN_PREFIX + "foo", "bar")
+                .put(ADLS_CONNECTION_STRING_PREFIX + "foo", "bar")
+                .put(ADLS_READ_BLOCK_SIZE, "42")
+                .put(ADLS_WRITE_BLOCK_SIZE, "42")
+                .put(ADLS_SHARED_KEY_ACCOUNT_NAME, "me")
+                .put(ADLS_SHARED_KEY_ACCOUNT_KEY, "secret")
+                .build());
+
+    AzureProperties serdedProps = TestHelpers.roundTripSerialize(props);
+    
assertThat(serdedProps.adlsReadBlockSize()).isEqualTo(props.adlsReadBlockSize());
+    
assertThat(serdedProps.adlsWriteBlockSize()).isEqualTo(props.adlsWriteBlockSize());
+  }
+
   @Test
   public void testWithSasToken() {
     AzureProperties props =

Reply via email to