Hello,

I actually needed this myself so I have validated it. Again, this is if you
want Flink itself to access Azure, and I'm fairly certain you have to use
Java because the plugin's class loader won't have access to the Scala
library's jars.

* You have to build against
https://mvnrepository.com/artifact/org.apache.flink/flink-azure-fs-hadoop/1.16.1
(mark it as provided).
* You should implemente the azurebfs provider for ABFS.
* You can create 1 plugin folder and copy Flink's azure jar plus the one
with your interface implementation.

I can confirm that worked.

Regards,
Alexis.

On Tue, 7 Mar 2023, 06:47 Swathi C, <swathi.c.apa...@gmail.com> wrote:

> Hi Ivan,
>
> You can try to setup using MSI so that the flink pods access the storage
> account and you might need to add the podIdentity to the flink pod so that
> it can access it. ( MSI should have the access for the storage account as
> well )
> The pod identity will have the required permissions to access to the
> storage account. These changes might be required along with adding the
> plugins.
> Can you try adding the following to the flink-config ?
>
> fs.azure.account.auth.type: OAuth
> fs.azure.account.oauth2.msi.tenant: <TENANT_ID>
> fs.azure.account.oauth.provider.type:
> org.apache.flink.fs.shaded.hadoop3.org.apache.hadoop.fs.azurebfs.oauth2.MsiTokenProvider
> fs.azure.account.oauth2.client.id: <CLIENT_ID>
> fs.azure.identity.transformer.service.principal.substitution.list: '*'
> fs.azure.identity.transformer.service.principal.id: <SERVICE_PRINCIPAL>
>
> Regards,
> Swathi C
>
> On Tue, Mar 7, 2023 at 8:53 AM Ivan Webber via user <user@flink.apache.org>
> wrote:
>
>> Thanks for the pointers Alexis!
>>
>>
>>
>> Implementing `org.apache.hadoop.fs.azure.KeyProvider` has helped me make
>> progress, but I’m running into a new error:
>>
>> ```
>>
>> org.apache.flink.fs.shaded.hadoop3.org.apache.hadoop.fs.azure.KeyProviderException:
>> org.tryflink.wrappers.TrafficForecastEnvKeyProviderWrapper specified in
>> config is not a valid KeyProvider class.
>>
>> ```
>>
>>
>>
>> I get this error whether I implement the class in Scala or Java, or use `
>> org.apache.hadoop.fs.azure.KeyProvider` or `
>> org.apache.hadoop.fs.azurebfs.services.KeyProvider `. My best guess is that
>> it’s something to do with not building against the shaded interface which
>> you indicated I should do or possibly different class loaders. To build
>> against the shaded interfaces would I import a package that has them?
>>
>>
>>
>> This is the dependency I added with
>> `org.apache.hadoop.fs.azure.KeyProvider`.
>>
>> ```
>>
>>         <dependency>
>>
>>             <groupId>org.apache.hadoop</groupId>
>>
>>             <artifactId>hadoop-azure</artifactId>
>>
>>             <version>3.3.2</version>
>>
>>         </dependency>
>>
>> ```
>>
>>
>>
>> What I’ve learned so far is that this configuration has more to do with
>> configuring Hadoop than Flink as the configuration is forwarded
>> <https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/filesystems/azure/#:~:text=Flink%20forwards%20all%20Flink%20configurations%20with%20a%20key%20prefix%20of%20fs.azure%20to%20the%20Hadoop%20configuration%20of%20the%20filesystem>.
>> Thus, I tried setting the properties to use Azure Managed Identity
>> <https://hadoop.apache.org/docs/stable/hadoop-azure/abfs.html#Azure_Managed_Identity>,
>> but got an error [1]. If anyone has gotten that to work I’d be interested
>> in hearing about it.
>>
>>
>>
>> Thanks for the help so far; please, anyone who can give pointers send
>> them.
>>
>>
>>
>> Thanks,
>>
>>
>>
>> Ivan
>>
>>
>>
>>
>>
>> [1] -
>> org.apache.flink.fs.shaded.hadoop3.org.apache.hadoop.fs.azure.AzureException:
>> No credentials found for account myblob.blob.core.windows.net in the
>> configuration, and its container flink-forecast is not accessible using
>> anonymous credentials. Please check if the container exists first. If it is
>> not publicly available, you have to provide account credentials.
>>
>>
>>
>> *From: *Ivan Webber <ivan.web...@microsoft.com>
>> *Sent: *Friday, March 3, 2023 10:38 AM
>> *To: *Alexis Sarda-Espinosa <sarda.espin...@gmail.com>
>> *Cc: *user <user@flink.apache.org>
>> *Subject: *Re: [EXTERNAL] Re: Secure Azure Credential Configuration
>>
>>
>>
>> Thanks Alexis,
>>
>>
>>
>> I will be trying that out today. If it works I will share back and try
>> adding it to the docs.
>>
>>
>>
>>
>>
>> *From:* Alexis Sarda-Espinosa <sarda.espin...@gmail.com>
>> *Sent:* Thursday, March 2, 2023 3:33:03 PM
>> *To:* Ivan Webber <ivan.web...@microsoft.com>
>> *Cc:* user <user@flink.apache.org>
>> *Subject:* Re: [EXTERNAL] Re: Secure Azure Credential Configuration
>>
>>
>>
>> You don't often get email from sarda.espin...@gmail.com. Learn why this
>> is important <https://aka.ms/LearnAboutSenderIdentification>
>>
>> Hi Ivan,
>>
>>
>>
>> please always include the whole distribution list since answers may help
>> others as well.
>>
>>
>>
>> I would also think about implementing your own provider(s), but some
>> things I know:
>>
>>
>>
>> - There are 2 different KeyProvider interfaces (which isn't explicitly
>> documented from what I can tell):
>>
>>   * org.apache.hadoop.fs.azure.KeyProvider - WASB
>>
>>   * org.apache.hadoop.fs.azurebfs.services.KeyProvider - ABFS (I think)
>>
>> - Flink shades the hadoop classes
>> under org.apache.flink.fs.shaded.hadoop3... so you would need to implement
>> your providers against the shaded interfaces.
>>
>> - The documentation for Flink plugins [1] shows an s3 folder with
>> multiple jars, so I imagine you could add a jar with your key providers to
>> a folder with the azure-fs jar, but I've never tested this.
>>
>>
>>
>> However, I believe this whole shading and plugin details are only
>> relevant if you want Flink to access the azure FS for its checkpoints
>> and/or savepoints, if you need to access the FS directly in your code, I
>> imagine you're better off including the relevant hadoop jars in your fat
>> jar without going through Flink's plugin system.
>>
>>
>>
>> This is my impression, but maybe someone else can correct me if I'm wrong.
>>
>>
>>
>> [1]
>> https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/filesystems/plugins/
>> <https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Fnightlies.apache.org%2Fflink%2Fflink-docs-master%2Fdocs%2Fdeployment%2Ffilesystems%2Fplugins%2F&data=05%7C01%7CIvan.Webber%40microsoft.com%7C302a8781f35f4b64cfaf08db1b767f30%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C638133968331853429%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000%7C%7C%7C&sdata=8W361g8DGOeDIipbVIHZC4TyzxFs8XiQIMFoW7zLUbQ%3D&reserved=0>
>>
>>
>>
>> Regards,
>>
>> Alexis.
>>
>>
>>
>> Am Do., 2. März 2023 um 23:46 Uhr schrieb Ivan Webber <
>> ivan.web...@microsoft.com>:
>>
>> Hello Alexis,
>>
>>
>>
>> I was actually thinking I’d use both WASB and ABFS, but I looked at the
>> source for EnvironmentVariableKeyProvider and it only reads a single
>> specific environment variable where my pipeline actually needs to bring
>> together data stored in different blob and ADLS accounts. I couldn’t find
>> anything about providing my own KeyProvider but I considered trying it as
>> an experiment at one point.
>>
>>
>>
>> *From: *Alexis Sarda-Espinosa <sarda.espin...@gmail.com>
>> *Sent: *Thursday, March 2, 2023 2:38 PM
>> *To: *Ivan Webber <ivan.web...@microsoft.com>
>> *Cc: *user <user@flink.apache.org>
>> *Subject: *[EXTERNAL] Re: Secure Azure Credential Configuration
>>
>>
>>
>> You don't often get email from sarda.espin...@gmail.com. Learn why this
>> is important <https://aka.ms/LearnAboutSenderIdentification>
>>
>> Hi Ivan,
>>
>>
>>
>> Mercy is always free. Are you using WASB or ABFS? I presume it's the
>> latter, since that's the one that can't use EnvironmentVariableKeyProvider,
>> but just to be sure.
>>
>>
>>
>> Regards,
>>
>> Alexis.
>>
>>
>>
>> On Thu, 2 Mar 2023, 23:07 Ivan Webber via user, <user@flink.apache.org>
>> wrote:
>>
>> TLDR: I will buy your coffee if you can help me understand to securely
>> configure Azure credentials (doc page
>> <https://nam06.safelinks.protection.outlook.com/?url=https%3A%2F%2Fnightlies.apache.org%2Fflink%2Fflink-docs-master%2Fdocs%2Fdeployment%2Ffilesystems%2Fazure%2F&data=05%7C01%7CIvan.Webber%40microsoft.com%7C302a8781f35f4b64cfaf08db1b767f30%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C638133968331853429%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000%7C%7C%7C&sdata=C3898nZaxzivVsSaN2jMqME93PR%2BicS%2By9MFJKV%2B%2Fyk%3D&reserved=0>
>> for reference).
>>
>>
>>
>> I am a junior developer tasked with being the first person to learn the
>> Apache Flink framework. I know that storing secrets in flink-conf.yaml in a
>> container is a bad idea. I’ve tried exposing Azure storage keys as env vars
>> and using `config.setString`, but those properties seem to get overridden.
>> I plan on using Flink operator, so if you can show me in that context
>> that’d be ideal.
>>
>>
>>
>> Thanks, and sorry for bothering everyone. I’ve just exhausted myself and
>> am hopeful someone will have mercy for me. I really will Venmo you $5 for
>> coffee if you want.
>>
>>
>>
>> Thanks,
>>
>>
>>
>> Ivan
>>
>>
>>
>>
>>
>> Larger code examples:
>>
>>
>>
>> Setting dynamic properties before executing the job doesn’t work because
>> the values seem to get overridden or never forwarded.
>>
>> ```
>>
>>     val config = new Configuration()
>>
>>     config.setString("fs.azure.account.key.mystore1.blob.core.windows.net
>> <https://nam06.safelinks.protection.outlook.com/?url=http%3A%2F%2Ffs.azure.account.key.mystore1.blob.core.windows.net%2F&data=05%7C01%7CIvan.Webber%40microsoft.com%7C302a8781f35f4b64cfaf08db1b767f30%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C638133968331853429%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000%7C%7C%7C&sdata=DWFU%2F86blheDCyBsWbtvZGbtk%2BcGrNmPoDNyvjIdkmM%3D&reserved=0>
>> ", System.getenv("KEY_1"))
>>
>>     config.setString("fs.azure.account.key.mystore2.blob.core.windows.net
>> <https://nam06.safelinks.protection.outlook.com/?url=http%3A%2F%2Ffs.azure.account.key.mystore2.blob.core.windows.net%2F&data=05%7C01%7CIvan.Webber%40microsoft.com%7C302a8781f35f4b64cfaf08db1b767f30%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C638133968331853429%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000%7C%7C%7C&sdata=YiTsdw5gN4U5Y0jU6BBWadKQinf%2BtI3Wvu9EuKWn%2Bqc%3D&reserved=0>
>> ", System.getenv("KEY_2"))
>>
>>     config.setString("fs.azure.account.key.mystore3.blob.core.windows.net
>> <https://nam06.safelinks.protection.outlook.com/?url=http%3A%2F%2Ffs.azure.account.key.mystore3.blob.core.windows.net%2F&data=05%7C01%7CIvan.Webber%40microsoft.com%7C302a8781f35f4b64cfaf08db1b767f30%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C638133968332009678%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000%7C%7C%7C&sdata=Z049XD4%2BJuSuk1BmSOD9umfket1hC90Oy6Q8jIQ9icY%3D&reserved=0>
>> ", System.getenv("KEY_3"))
>>
>>     val env = environment.StreamExecutionEnvironment
>> .getExecutionEnvironment(config)
>>
>> ```
>>
>>
>>
>> In Flink operator configuration fields can be provided as follows, but
>> then I can’t commit the file with a secret inside. Ideally there would be a
>> way to reference a secret but the values must be literal strings.
>>
>> ```
>>
>> spec:
>>
>>   flinkConfiguration:
>>
>>     fs.azure.account.key.mystore1.blob.core.windows.net
>> <https://nam06.safelinks.protection.outlook.com/?url=http%3A%2F%2Ffs.azure.account.key.mystore1.blob.core.windows.net%2F&data=05%7C01%7CIvan.Webber%40microsoft.com%7C302a8781f35f4b64cfaf08db1b767f30%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C638133968332009678%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000%7C%7C%7C&sdata=yALWxcdL1bNYNCnAUzSuhI9TEzYDGgxIdn9sW5OphfI%3D&reserved=0>:
>> SECRET_STRING
>>
>>     fs.azure.account.key.mystore2.blob.core.windows.net
>> <https://nam06.safelinks.protection.outlook.com/?url=http%3A%2F%2Ffs.azure.account.key.mystore2.blob.core.windows.net%2F&data=05%7C01%7CIvan.Webber%40microsoft.com%7C302a8781f35f4b64cfaf08db1b767f30%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C638133968332009678%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000%7C%7C%7C&sdata=ldqNKK1%2FtgDcPo6izcR6Jn53YqUMhGEAEs79%2BN8nf3A%3D&reserved=0>:
>> SECRET_STRING
>>
>>     fs.azure.account.key.mystore3.blob.core.windows.net
>> <https://nam06.safelinks.protection.outlook.com/?url=http%3A%2F%2Ffs.azure.account.key.mystore3.blob.core.windows.net%2F&data=05%7C01%7CIvan.Webber%40microsoft.com%7C302a8781f35f4b64cfaf08db1b767f30%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C638133968332009678%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C1000%7C%7C%7C&sdata=Z049XD4%2BJuSuk1BmSOD9umfket1hC90Oy6Q8jIQ9icY%3D&reserved=0>:
>> SECRET_STRING
>>
>> ```
>>
>>
>>
>> The last possible solution I can think that I’ll be trying is putting the
>> entire flink-conf.yaml into a secret, or having a different container that
>> adds secrets to the flink-operator-job.yaml and then does the `kubectl
>> create -f flink-operator-job.yaml` (if that’s even possible).
>>
>>
>>
>>
>>
>

Reply via email to