This is an automated email from the ASF dual-hosted git repository. sewen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git
commit 56cd59f47e31e58e112e260f0f4b43c45c1aa74a Author: Konstantin Knauf <knauf.konstan...@gmail.com> AuthorDate: Fri Nov 29 16:20:29 2019 +0100 [FLINK-14907] [filesystems] Add EnvironmentVariableKeyProvider for Azure Blob Storage Also updates the documentation on credential configuration for ABS This closes #10376 --- docs/ops/filesystems/azure.md | 19 ++++++--- .../fs/azurefs/EnvironmentVariableKeyProvider.java | 48 ++++++++++++++++++++++ 2 files changed, 61 insertions(+), 6 deletions(-) diff --git a/docs/ops/filesystems/azure.md b/docs/ops/filesystems/azure.md index d721be5..d63c9d6 100644 --- a/docs/ops/filesystems/azure.md +++ b/docs/ops/filesystems/azure.md @@ -62,17 +62,24 @@ cp ./opt/flink-azure-fs-hadoop-{{ site.version }}.jar ./plugins/azure-fs-hadoop/ `flink-azure-fs-hadoop` registers default FileSystem wrappers for URIs with the *wasb://* and *wasbs://* (SSL encrypted access) scheme. -#### Configurations setup -After setting up the Azure Blob Storage FileSystem wrapper, you need to configure credentials to make sure that Flink is allowed to access Azure Blob Storage. +### Credentials Configuration -To allow for easy adoption, you can use the same configuration keys in `flink-conf.yaml` as in Hadoop's `core-site.xml` +Hadoop's Azure Filesystem supports configuration of credentials via the Hadoop configuration as +outlined in the [Hadoop Azure Blob Storage documentation](https://hadoop.apache.org/docs/current/hadoop-azure/index.html#Configuring_Credentials). +For convenience Flink forwards all Flink configurations with a key prefix of `fs.azure` to the +Hadoop configuration of the filesystem. Consequentially, the azure blob storage key can be configured +in `flink-conf.yaml` via: -You can see the configuration keys in the [Hadoop Azure Blob Storage documentation](https://hadoop.apache.org/docs/current/hadoop-azure/index.html#Configuring_Credentials). +{% highlight yaml %} +fs.azure.account.key.<account_name>.blob.core.windows.net: <azure_storage_key> +{% endhighlight %} -There are some required configurations that must be added to `flink-conf.yaml`: +Alternatively, the the filesystem can be configured to read the Azure Blob Storage key from an +environment variable `AZURE_STORAGE_KEY` by setting the following configuration keys in +`flink-conf.yaml`. {% highlight yaml %} -fs.azure.account.key.youraccount.blob.core.windows.net: Azure Blob Storage access key +fs.azure.account.keyprovider.<account_name>.blob.core.windows.net: org.apache.flink.fs.azurefs.EnvironmentVariableKeyProvider {% endhighlight %} {% top %} diff --git a/flink-filesystems/flink-azure-fs-hadoop/src/main/java/org/apache/flink/fs/azurefs/EnvironmentVariableKeyProvider.java b/flink-filesystems/flink-azure-fs-hadoop/src/main/java/org/apache/flink/fs/azurefs/EnvironmentVariableKeyProvider.java new file mode 100644 index 0000000..a7cd3fa --- /dev/null +++ b/flink-filesystems/flink-azure-fs-hadoop/src/main/java/org/apache/flink/fs/azurefs/EnvironmentVariableKeyProvider.java @@ -0,0 +1,48 @@ +/* + * 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.flink.fs.azurefs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azure.KeyProvider; +import org.apache.hadoop.fs.azure.KeyProviderException; + +/** + * An implementation of {@link org.apache.hadoop.fs.azure.KeyProvider}, which reads the Azure + * storage key from an environment variable named "AZURE_STORAGE_KEY". + * + */ +public class EnvironmentVariableKeyProvider implements KeyProvider { + + public static final String AZURE_STORAGE_KEY_ENV_VARIABLE = "AZURE_STORAGE_KEY"; + + @Override + public String getStorageAccountKey( + final String s, + final Configuration configuration) throws KeyProviderException { + + String azureStorageKey = System.getenv(AZURE_STORAGE_KEY_ENV_VARIABLE); + + if (azureStorageKey != null) { + return azureStorageKey; + } else { + throw new KeyProviderException("Unable to retrieve Azure storage key from environment. \"" + + AZURE_STORAGE_KEY_ENV_VARIABLE + "\" not set."); + } + } +}