kevinjqliu commented on code in PR #13186:
URL: https://github.com/apache/iceberg/pull/13186#discussion_r2566119844
##########
azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java:
##########
@@ -48,6 +49,8 @@ public class AzureProperties implements Serializable {
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";
public static final String ADLS_TOKEN = "adls.token";
+ public static final String KEYVAULT_URI = "azure.keyvault.uri";
Review Comment:
```suggestion
public static final String AZURE_KEYVAULT_URL = "azure.keyvault.url";
```
nit: can we use url instead of uri here since the Azure library uses
`vaultUrl`?
https://learn.microsoft.com/en-us/java/api/overview/azure/security-keyvault-keys-readme?view=azure-java-stable#create-key-client
and also use the azure prefix for the variable name
##########
azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java:
##########
@@ -124,6 +129,14 @@ public AzureProperties(Map<String, String> properties) {
PropertyUtil.propertyAsBoolean(properties,
ADLS_REFRESH_CREDENTIALS_ENABLED, true);
this.token = properties.get(ADLS_TOKEN);
this.allProperties = SerializableMap.copyOf(properties);
+ if (properties.containsKey(KEYVAULT_URI)) {
+ this.keyVaultUri = properties.get(KEYVAULT_URI);
+ }
+
+ this.keyWrapAlgorithm =
+ properties.getOrDefault(
+ AzureProperties.KEYVAULT_KEY_WRAPPING_ALGORITHM,
+ KeyWrapAlgorithm.RSA_OAEP_256.getValue());
Review Comment:
> Microsoft recommends using RSA_OAEP_256 or stronger algorithms for
enhanced security.
From
https://learn.microsoft.com/en-us/azure/key-vault/keys/about-keys-details#wrapkeyunwrapkey-encryptdecrypt
👍
##########
azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java:
##########
@@ -189,4 +202,12 @@ public Mono<AccessToken> getToken(TokenRequestContext
request) {
builder.endpoint("https://" + account);
}
}
+
+ public KeyWrapAlgorithm keyWrapAlgorithm() {
+ return KeyWrapAlgorithm.fromString(this.keyWrapAlgorithm);
+ }
+
+ public String keyVaultUri() {
+ return this.keyVaultUri;
+ }
Review Comment:
wdyt about making this return Optional<String>, similar to what we do for
the `adlsReadBlockSize` and `adlsWriteBlockSize` functions above?
##########
azure/src/integration/java/org/apache/iceberg/azure/keymanagement/TestAzureKeyManagementClient.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.keymanagement;
+
+import static org.apache.iceberg.azure.AzureProperties.KEYVAULT_URI;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.security.keyvault.keys.KeyClient;
+import com.azure.security.keyvault.keys.KeyClientBuilder;
+import com.azure.security.keyvault.keys.models.KeyType;
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import org.apache.iceberg.encryption.KeyManagementClient;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.condition.EnabledIfEnvironmentVariable;
+import org.junit.jupiter.api.condition.EnabledIfEnvironmentVariables;
+
+@EnabledIfEnvironmentVariables({
+ @EnabledIfEnvironmentVariable(named = "AZURE_CLIENT_ID", matches = ".*"),
+ @EnabledIfEnvironmentVariable(named = "AZURE_CLIENT_SECRET", matches = ".*"),
+ @EnabledIfEnvironmentVariable(named = "AZURE_TENANT_ID", matches = ".*"),
+ @EnabledIfEnvironmentVariable(named = KEYVAULT_URI, matches = ".*")
Review Comment:
```suggestion
@EnabledIfEnvironmentVariable(named = "azure.keyvault.url", matches = ".*")
```
##########
azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java:
##########
@@ -48,6 +49,8 @@ public class AzureProperties implements Serializable {
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";
public static final String ADLS_TOKEN = "adls.token";
+ public static final String KEYVAULT_URI = "azure.keyvault.uri";
+ public static final String KEYVAULT_KEY_WRAPPING_ALGORITHM =
"keyvault.key-wrapping-algorithm";
Review Comment:
```suggestion
public static final String AZURE_KEYVAULT_KEY_WRAP_ALGORITHM =
"azure.keyvault.key-wrap-algorithm";
```
nit: can we use "key wrap" here since thats what the Azure library uses?
and also use the azure prefix for the variable name and properties string
##########
azure/src/main/java/org/apache/iceberg/azure/keymanagement/AzureKeyManagementClient.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.keymanagement;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.security.keyvault.keys.KeyClient;
+import com.azure.security.keyvault.keys.KeyClientBuilder;
+import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm;
+import com.azure.security.keyvault.keys.cryptography.models.UnwrapResult;
+import com.azure.security.keyvault.keys.cryptography.models.WrapResult;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import org.apache.iceberg.azure.AzureProperties;
+import org.apache.iceberg.encryption.KeyManagementClient;
+import org.apache.iceberg.util.ByteBuffers;
+
+/** Azure key management client which connects to Azure Key Vault. */
+public class AzureKeyManagementClient implements KeyManagementClient {
+ private KeyClient keyClient;
+ private KeyWrapAlgorithm keyWrapAlgorithm;
+
+ @Override
+ public void initialize(Map<String, String> properties) {
+ AzureProperties azureProperties = new AzureProperties(properties);
+
+ String vaultUrl = azureProperties.keyVaultUri();
+ this.keyWrapAlgorithm = azureProperties.keyWrapAlgorithm();
+ this.keyClient =
+ new KeyClientBuilder()
+ .vaultUrl(vaultUrl)
+ .credential(new DefaultAzureCredentialBuilder().build())
Review Comment:
perhaps we can use `AdlsTokenCredentialProviders.from(properties)` here to
support other auth types
https://github.com/apache/iceberg/blob/main/azure/src/main/java/org/apache/iceberg/azure/AdlsTokenCredentialProviders.java
##########
azure/src/integration/java/org/apache/iceberg/azure/keymanagement/TestAzureKeyManagementClient.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.keymanagement;
+
+import static org.apache.iceberg.azure.AzureProperties.KEYVAULT_URI;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import com.azure.identity.DefaultAzureCredentialBuilder;
+import com.azure.security.keyvault.keys.KeyClient;
+import com.azure.security.keyvault.keys.KeyClientBuilder;
+import com.azure.security.keyvault.keys.models.KeyType;
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import org.apache.iceberg.encryption.KeyManagementClient;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.condition.EnabledIfEnvironmentVariable;
+import org.junit.jupiter.api.condition.EnabledIfEnvironmentVariables;
+
+@EnabledIfEnvironmentVariables({
+ @EnabledIfEnvironmentVariable(named = "AZURE_CLIENT_ID", matches = ".*"),
+ @EnabledIfEnvironmentVariable(named = "AZURE_CLIENT_SECRET", matches = ".*"),
+ @EnabledIfEnvironmentVariable(named = "AZURE_TENANT_ID", matches = ".*"),
+ @EnabledIfEnvironmentVariable(named = KEYVAULT_URI, matches = ".*")
+})
+public class TestAzureKeyManagementClient {
+ private static final String ICEBERG_TEST_KEY_NAME = "iceberg-test-key";
+
+ private static KeyClient keyClient;
+
+ private static KeyManagementClient azureKeyManagementClient;
+
+ @BeforeAll
+ public static void beforeClass() {
+ String keyVaultUri = System.getenv(KEYVAULT_URI);
+ keyClient =
+ new KeyClientBuilder()
+ .vaultUrl(keyVaultUri)
+ .credential(new DefaultAzureCredentialBuilder().build())
+ .buildClient();
+ keyClient.createKey(ICEBERG_TEST_KEY_NAME, KeyType.RSA);
+ azureKeyManagementClient = new AzureKeyManagementClient();
+ azureKeyManagementClient.initialize(ImmutableMap.of(KEYVAULT_URI,
keyVaultUri));
+ }
+
+ @AfterAll
+ public static void afterClass() {
+ if (keyClient != null) {
+
keyClient.beginDeleteKey(ICEBERG_TEST_KEY_NAME).waitForCompletion(Duration.ofMinutes(5));
Review Comment:
nit: is this a blocking call? 5 minutes is a long time to wait for test
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]