You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by pa...@apache.org on 2021/01/26 02:12:28 UTC

[beam] branch master updated: Revert "Merge pull request #12647 from [BEAM-10378] Deserializing Azure Credentials"

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

pabloem pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new efc35c5  Revert "Merge pull request #12647 from [BEAM-10378] Deserializing Azure Credentials"
     new 15e2ad8  Merge pull request #13807 from pabloem/fix-azure
efc35c5 is described below

commit efc35c589f3788e59d0445827266245810a218d6
Author: Pablo Estrada <pa...@apache.org>
AuthorDate: Mon Jan 25 17:21:58 2021 -0800

    Revert "Merge pull request #12647 from [BEAM-10378] Deserializing Azure Credentials"
    
    This reverts commit e1b42bfb432d14682fe9e3d61d3f30a3e050a65b.
---
 .../DefaultBlobstoreClientBuilderFactory.java      |   8 +-
 .../beam/sdk/io/azure/options/AzureModule.java     | 165 ---------------------
 .../sdk/io/azure/options/BlobstoreOptions.java     |  24 +--
 3 files changed, 18 insertions(+), 179 deletions(-)

diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java
index b68fcc2..707e73b 100644
--- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java
+++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java
@@ -37,8 +37,12 @@ public class DefaultBlobstoreClientBuilderFactory implements BlobstoreClientBuil
       builder = builder.connectionString(blobstoreOptions.getAzureConnectionString());
     }
 
-    if (blobstoreOptions.getAzureCredentialsProvider() != null) {
-      builder = builder.credential(blobstoreOptions.getAzureCredentialsProvider());
+    if (blobstoreOptions.getSharedKeyCredential() != null) {
+      builder = builder.credential(blobstoreOptions.getSharedKeyCredential());
+    }
+
+    if (blobstoreOptions.getTokenCredential() != null) {
+      builder = builder.credential(blobstoreOptions.getTokenCredential());
     }
 
     if (!Strings.isNullOrEmpty(blobstoreOptions.getSasToken())) {
diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureModule.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureModule.java
deleted file mode 100644
index 9dccf5c..0000000
--- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureModule.java
+++ /dev/null
@@ -1,165 +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.beam.sdk.io.azure.options;
-
-import com.azure.core.credential.TokenCredential;
-import com.azure.identity.ClientCertificateCredential;
-import com.azure.identity.ClientCertificateCredentialBuilder;
-import com.azure.identity.ClientSecretCredential;
-import com.azure.identity.ClientSecretCredentialBuilder;
-import com.azure.identity.DefaultAzureCredential;
-import com.azure.identity.DefaultAzureCredentialBuilder;
-import com.azure.identity.EnvironmentCredential;
-import com.azure.identity.EnvironmentCredentialBuilder;
-import com.azure.identity.ManagedIdentityCredential;
-import com.azure.identity.ManagedIdentityCredentialBuilder;
-import com.azure.identity.UsernamePasswordCredential;
-import com.azure.identity.UsernamePasswordCredentialBuilder;
-import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import com.fasterxml.jackson.core.JsonGenerator;
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.DeserializationContext;
-import com.fasterxml.jackson.databind.JsonDeserializer;
-import com.fasterxml.jackson.databind.JsonSerializer;
-import com.fasterxml.jackson.databind.Module;
-import com.fasterxml.jackson.databind.SerializerProvider;
-import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
-import com.fasterxml.jackson.databind.annotation.JsonSerialize;
-import com.fasterxml.jackson.databind.jsontype.TypeDeserializer;
-import com.fasterxml.jackson.databind.jsontype.TypeSerializer;
-import com.fasterxml.jackson.databind.module.SimpleModule;
-import com.google.auto.service.AutoService;
-import java.io.IOException;
-import java.util.Map;
-import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
-
-/**
- * A Jackson {@link Module} that registers a {@link JsonSerializer} and {@link JsonDeserializer} for
- * Azure credential providers. The serialized form is a JSON map.
- */
-@AutoService(Module.class)
-public class AzureModule extends SimpleModule {
-
-  private static final String AZURE_CLIENT_ID = "azureClientId";
-  private static final String AZURE_TENANT_ID = "azureTenantId";
-  private static final String AZURE_CLIENT_SECRET = "azureClientSecret";
-  private static final String AZURE_CLIENT_CERTIFICATE_PATH = "azureClientCertificatePath";
-  private static final String AZURE_USERNAME = "azureUsername";
-  private static final String AZURE_PASSWORD = "azurePassword";
-  private static final String AZURE_PFX_CERTIFICATE_PATH = "azurePfxCertificatePath";
-  private static final String AZURE_PFX_CERTIFICATE_PASSWORD = "azurePfxCertificatePassword";
-
-  public AzureModule() {
-    super("AzureModule");
-    setMixInAnnotation(TokenCredential.class, TokenCredentialMixin.class);
-  }
-
-  /** A mixin to add Jackson annotations to {@link TokenCredential}. */
-  @JsonDeserialize(using = TokenCredentialDeserializer.class)
-  @JsonSerialize(using = TokenCredentialSerializer.class)
-  @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY)
-  private static class TokenCredentialMixin {}
-
-  private static class TokenCredentialDeserializer extends JsonDeserializer<TokenCredential> {
-
-    @Override
-    public TokenCredential deserialize(JsonParser jsonParser, DeserializationContext context)
-        throws IOException {
-      return context.readValue(jsonParser, TokenCredential.class);
-    }
-
-    @Override
-    public TokenCredential deserializeWithType(
-        JsonParser jsonParser, DeserializationContext context, TypeDeserializer typeDeserializer)
-        throws IOException {
-      Map<String, String> asMap =
-          jsonParser.readValueAs(new TypeReference<Map<String, String>>() {});
-
-      String typeNameKey = typeDeserializer.getPropertyName();
-      String typeName = asMap.get(typeNameKey);
-      if (typeName == null) {
-        throw new IOException(
-            String.format("Azure credentials provider type name key '%s' not found", typeNameKey));
-      }
-
-      if (typeName.equals(DefaultAzureCredential.class.getSimpleName())) {
-        return new DefaultAzureCredentialBuilder().build();
-      } else if (typeName.equals(ClientSecretCredential.class.getSimpleName())) {
-        return new ClientSecretCredentialBuilder()
-            .clientId(asMap.get(AZURE_CLIENT_ID))
-            .clientSecret(asMap.get(AZURE_CLIENT_SECRET))
-            .tenantId(asMap.get(AZURE_TENANT_ID))
-            .build();
-      } else if (typeName.equals(ManagedIdentityCredential.class.getSimpleName())) {
-        return new ManagedIdentityCredentialBuilder().clientId(asMap.get(AZURE_CLIENT_ID)).build();
-      } else if (typeName.equals(EnvironmentCredential.class.getSimpleName())) {
-        return new EnvironmentCredentialBuilder().build();
-      } else if (typeName.equals(ClientCertificateCredential.class.getSimpleName())) {
-        if (asMap.containsKey(AZURE_CLIENT_CERTIFICATE_PATH)) {
-          return new ClientCertificateCredentialBuilder()
-              .clientId(asMap.get(AZURE_CLIENT_ID))
-              .pemCertificate(asMap.get(AZURE_CLIENT_CERTIFICATE_PATH))
-              .tenantId(asMap.get(AZURE_TENANT_ID))
-              .build();
-        } else {
-          return new ClientCertificateCredentialBuilder()
-              .clientId(asMap.get(AZURE_CLIENT_ID))
-              .pfxCertificate(
-                  asMap.get(AZURE_PFX_CERTIFICATE_PATH), asMap.get(AZURE_PFX_CERTIFICATE_PASSWORD))
-              .tenantId(asMap.get(AZURE_TENANT_ID))
-              .build();
-        }
-      } else if (typeName.equals(UsernamePasswordCredential.class.getSimpleName())) {
-        return new UsernamePasswordCredentialBuilder()
-            .clientId(asMap.get(AZURE_CLIENT_ID))
-            .username(asMap.get(AZURE_USERNAME))
-            .password(asMap.get(AZURE_PASSWORD))
-            .build();
-      } else {
-        throw new IOException(
-            String.format("Azure credential provider type '%s' is not supported", typeName));
-      }
-    }
-  }
-
-  // TODO: Write this class
-  private static class TokenCredentialSerializer extends JsonSerializer<TokenCredential> {
-    // These providers are singletons, so don't require any serialization, other than type.
-    // add any singleton credentials...
-    private static final ImmutableSet<Object> SINGLETON_CREDENTIAL_PROVIDERS = ImmutableSet.of();
-
-    @Override
-    public void serialize(
-        TokenCredential tokenCredential,
-        JsonGenerator jsonGenerator,
-        SerializerProvider serializers)
-        throws IOException {
-      serializers.defaultSerializeValue(tokenCredential, jsonGenerator);
-    }
-
-    @Override
-    public void serializeWithType(
-        TokenCredential tokenCredential,
-        JsonGenerator jsonGenerator,
-        SerializerProvider serializers,
-        TypeSerializer typeSerializer) {
-      throw new UnsupportedOperationException();
-    }
-  }
-}
diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java
index 0e704ab..108cb0c 100644
--- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java
+++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/BlobstoreOptions.java
@@ -23,6 +23,7 @@ import com.azure.core.http.HttpPipeline;
 import com.azure.core.http.policy.HttpPipelinePolicy;
 import com.azure.identity.DefaultAzureCredentialBuilder;
 import com.azure.storage.blob.models.CustomerProvidedKey;
+import com.azure.storage.common.StorageSharedKeyCredential;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
 import org.apache.beam.sdk.io.azure.blobstore.DefaultBlobstoreClientBuilderFactory;
@@ -58,6 +59,16 @@ public interface BlobstoreOptions extends PipelineOptions {
 
   void setAzureConnectionString(String connectionString);
 
+  @Description("Sets a StorageSharedKeyCredential used to authorize requests sent to the service.")
+  StorageSharedKeyCredential getSharedKeyCredential();
+
+  void setSharedKeyCredential(StorageSharedKeyCredential sharedKeyCredential);
+
+  @Description("Sets a TokenCredential used to authorize requests sent to the service.")
+  TokenCredential getTokenCredential();
+
+  void setTokenCredential(TokenCredential tokenCredential);
+
   @Description("Sets the SAS token used to authorize requests sent to the service.")
   String getSasToken();
 
@@ -99,25 +110,14 @@ public interface BlobstoreOptions extends PipelineOptions {
 
   void setHttpPipeline(HttpPipeline httpPipeline);
 
-  /* Refer to {@link DefaultAWSCredentialsProviderChain} Javadoc for usage help. */
-
   /**
    * The credential instance that should be used to authenticate against Azure services. The option
    * value must contain a "@type" field and an Azure credentials provider class as the field value.
-   *
-   * <p>For example, to specify the Azure client id, tenant id, and client secret, specify the
-   * following: <code>
-   *     {"@type" : "ClientSecretCredential", "azureClientId": "client_id_value",
-   *     "azureTenantId": "tenant_id_value", "azureClientSecret": "client_secret_value"}
-   * </code>
    */
   @Description(
       "The credential instance that should be used to authenticate "
           + "against Azure services. The option value must contain \"@type\" field "
-          + "and an Azure credentials provider class name as the field value. "
-          + " For example, to specify the Azure client id, tenant id, and client secret, specify the following: "
-          + "{\"@type\" : \"ClientSecretCredential\", \"azureClientId\": \"client_id_value\", "
-          + "\"azureTenantId\": \"tenant_id_value\", \"azureClientSecret\": \"client_secret_value\"}")
+          + "and an Azure credentials provider class name as the field value.")
   @Default.InstanceFactory(AzureUserCredentialsFactory.class)
   TokenCredential getAzureCredentialsProvider();