Hi Derocco,

Good to hear that it is working. Let me create a Jira ticket and update the
document.

-Surendra


On Wed, May 17, 2023 at 9:29 PM DEROCCO, CHRISTOPHER <cd9...@att.com> wrote:

> Surendra,
>
>
>
> Your recommended config change fixed my issue. Azure Managed Service
> Identity works for me now and I can write checkpoints to ADLSGen2 storage.
> My client id is the managed identity that is attached to the azure
> Kubernetes nodepools. For anyone else facing this issue, my configurations
> to get this working in the Kubernetes yaml are:
>
>
>
> flinkConfigurations:
>
>     fs.azure.createRemoteFileSystemDuringInitialization: "true"
>
>     fs.azure.account.oauth.provider.type.<storage_account_name>.
> dfs.core.windows.net:
> *org.apache.flink.fs.shaded.hadoop3.org.apache.hadoop.fs.azurebfs.oauth2.MsiTokenProvider*
>
>     fs.azure.account.oauth2.msi.tenant. <storage_account_name>.
> dfs.core.windows.net: <Azure Tenant ID>
>
>     fs.azure.account.oauth2.client.id. <storage_account_name>.
> dfs.core.windows.net: <Managed Service Identity Client ID>
>
>     fs.azure.account.oauth2.client.endpoint. <storage_account_name>.
> dfs.core.windows.net: https://login.microsoftonline.com/<Azure Tenant
> ID>/oauth2/token
>
>
>
> Also this environment variable has to be added to the Kubernetes yaml
> configuration
>
>
>
>       containers:
>
>         # Do not change the main container name
>
>         - name: flink-main-container
>
>           env:
>
>           - name: ENABLE_BUILT_IN_PLUGINS
>
>             value: flink-azure-fs-hadoop-1.16.1.jar
>
>
>
>
>
> This azure managed service identity configuration should be added to the
> flink docs. I couldn’t find anywhere that the
> fs.azure.account.oauth.provider.type had to be set as
> *org.apache.flink.fs.shaded.hadoop3.org.apache.hadoop.fs.azurebfs.oauth2.MsiTokenProvider*
>
>
>
>
>
> *From:* Surendra Singh Lilhore <surendralilh...@gmail.com>
> *Sent:* Tuesday, May 16, 2023 11:46 PM
> *To:* Ivan Webber <ivan.web...@microsoft.com>
> *Cc:* DEROCCO, CHRISTOPHER <cd9...@att.com>; Shammon FY <zjur...@gmail.com>;
> user@flink.apache.org
> *Subject:* Re: MSI Auth to Azure Storage Account with Flink Apache
> Operator not working
>
>
>
> Hi DEROCCO,
>
>
>
> Flink uses shaded jars for the Hadoop Azure Storage plugin, so in order to
> correct the ClassNotFoundException, you need to adjust the configuration.
> Please configure the MSITokenProvider as shown below.
>
>
>
> fs.azure.account.oauth.provider.type:
> *org.apache.flink.fs.shaded.hadoop3.org.apache.hadoop.fs.azurebfs.oauth2.MsiTokenProvider*
>
>
>
> Thanks
>
> Surendra
>
>
>
>
>
> On Wed, May 17, 2023 at 5:32 AM Ivan Webber via user <
> user@flink.apache.org> wrote:
>
> When you create your cluster you probably need to ensure the following
> settings are set. I briefly looked into MSI but ended up using Azure Key
> Vault with CSI-storage driver for initial prototype (
> https://github.com/MicrosoftDocs/azure-docs/blob/main/articles/aks/csi-secrets-store-driver.md#upgrade-an-existing-aks-cluster-with-azure-key-vault-provider-for-secrets-store-csi-driver-support
> <https://urldefense.com/v3/__https:/github.com/MicrosoftDocs/azure-docs/blob/main/articles/aks/csi-secrets-store-driver.md*upgrade-an-existing-aks-cluster-with-azure-key-vault-provider-for-secrets-store-csi-driver-support__;Iw!!BhdT!iKqopXbK8CmuTNzfeMy7YIENZID6mDog1vC7RonlcCPJ33cegpcwXpHjtqYV1HDWO3bIWCPxwuHxW0-o0PEq2cCN$>
> ).
>
>
>
> For me it helped to think about it as Hadoop configuration.
>
>
>
> If you do get MSI working I would be interested in hearing what made it
> work for you, so be sure to update the docs or put it on this thread.
>
>
>
> *#### To create from scratch*
>
> Create an AKS cluster with the required settings.
>
> ```bash
>
> # create an AKS cluster with pod-managed identity and Azure CNI
>
> az aks create --resource-group $RESOURCE_GROUP --name $CLUSTER
> --enable-managed-identity --network-plugin azure --enable-pod-identity
>
> ```
>
>
>
> I hope that is somehow helpful.
>
>
>
> Best of luck,
>
>
>
> Ivan
>
>
>
> *From: *DEROCCO, CHRISTOPHER <cd9...@att.com>
> *Sent: *Monday, May 8, 2023 3:40 PM
> *To: *Shammon FY <zjur...@gmail.com>
> *Cc: *user@flink.apache.org
> *Subject: *[EXTERNAL] RE: MSI Auth to Azure Storage Account with Flink
> Apache Operator not working
>
>
>
> You don't often get email from cd9...@att.com. Learn why this is important
> <https://urldefense.com/v3/__https:/aka.ms/LearnAboutSenderIdentification__;!!BhdT!iKqopXbK8CmuTNzfeMy7YIENZID6mDog1vC7RonlcCPJ33cegpcwXpHjtqYV1HDWO3bIWCPxwuHxW0-o0Cz4a7J5$>
>
> Shammon,
>
>
>
> I’m still having trouble setting the package in my cluster environment. I 
> have these lines added to my dockerfile
>
> mkdir ./plugins/azure-fs-hadoop
>
> cp ./opt/flink-azure-fs-hadoop-1.16.0.jar ./plugins/azure-fs-hadoop/
>
>
>
> according to the flink docs here (
> https://nightlies.apache.org/flink/flink-docs-release-1.16/docs/deployment/filesystems/azure/
> <https://urldefense.com/v3/__https:/nightlies.apache.org/flink/flink-docs-release-1.16/docs/deployment/filesystems/azure/__;!!BhdT!iKqopXbK8CmuTNzfeMy7YIENZID6mDog1vC7RonlcCPJ33cegpcwXpHjtqYV1HDWO3bIWCPxwuHxW0-o0C-xBNry$>
> )
>
> This should enable the flink-azure-fs-hadoop jar in the environment which
> has the classes to enable the adls2 MSI authentication.
>
> I also have the following dependency in my pom to add it to the FAT Jar.
>
>
>
> <dependency>
>
>             <groupId>org.apache.flink</groupId>
>
>             <artifactId>flink-azure-fs-hadoop</artifactId>
>
>             <version>${flink.version}</version>
>
> </dependency>
>
>
>
> However, I still get the class not found error and the flink job is not
> able to authenticate to the azure storage account to store its checkpoints.
> I’m not sure what other configuration pieces I’m missing. Has anyone had
> successful with writing checkpoints to Azure ADLS2gen Storage with managed
> service identity (MSI) authentication.?
>
>
>
>
>
>
>
> *From:* Shammon FY <zjur...@gmail.com>
> *Sent:* Friday, May 5, 2023 8:38 PM
> *To:* DEROCCO, CHRISTOPHER <cd9...@att.com>
> *Cc:* user@flink.apache.org
> *Subject:* Re: MSI Auth to Azure Storage Account with Flink Apache
> Operator not working
>
>
>
> Hi DEROCCO,
>
>
>
> I think you can check the startup command of the job on k8s to see if the
> jar file is in the classpath.
>
>
>
> If your job is DataStream, you need to add hadoop azure dependency in your
> project, and if it is an SQL job, you need to include this jar file in your
> Flink release package. Or you can also add this package in your cluster
> environment.
>
>
>
> Best,
>
> Shammon FY
>
>
>
>
>
> On Fri, May 5, 2023 at 10:21 PM DEROCCO, CHRISTOPHER <cd9...@att.com>
> wrote:
>
> How can I add the package to the flink job or check if it is there?
>
>
>
> *From:* Shammon FY <zjur...@gmail.com>
> *Sent:* Thursday, May 4, 2023 9:59 PM
> *To:* DEROCCO, CHRISTOPHER <cd9...@att.com>
> *Cc:* user@flink.apache.org
> *Subject:* Re: MSI Auth to Azure Storage Account with Flink Apache
> Operator not working
>
>
>
> Hi DEROCCO,
>
>
>
> I think you need to check whether there is a hadoop-azure jar file in the
> classpath of your flink job. From an error message '*Caused by:
> java.lang.ClassNotFoundException: Class
> org.apache.hadoop.fs.azurebfs.oauth2.MsiTokenProvider not found.*', your
> flink job may be missing this package.
>
>
>
> Best,
>
> Shammon FY
>
>
>
>
>
> On Fri, May 5, 2023 at 4:40 AM DEROCCO, CHRISTOPHER <cd9...@att.com>
> wrote:
>
>
>
> I receive the error:  *Caused by: java.lang.ClassNotFoundException: Class
> org.apache.hadoop.fs.azurebfs.oauth2.MsiTokenProvider not found.*
>
> I’m using flink 1.16 running in Azure Kubernetes using the Flink Apache
> Kubernetes Operator.
>
> I have the following specified in the spec.flinkConfiguration: as per the
> Apache Kubernetes operator documentation.
>
>
>
>     fs.azure.createRemoteFileSystemDuringInitialization: "true"
>
>     fs.azure.account.auth.type.storageaccountname.dfs.core.windows.net
> <https://urldefense.com/v3/__http:/fs.azure.account.auth.type.storageaccountname.dfs.core.windows.net__;!!BhdT!nslIUVS9K-rzMRvjMFpWqBpcsAIiVPAfG6uroDOiSSQfmARHAQCYweWSe-TmKGHGzKD4HpyjvOZFkA$>:
> OAuth
>
>     fs.azure.account.oauth.provider.type.<storageaccountname>.
> dfs.core.windows.net
> <https://urldefense.com/v3/__http:/dfs.core.windows.net__;!!BhdT!nslIUVS9K-rzMRvjMFpWqBpcsAIiVPAfG6uroDOiSSQfmARHAQCYweWSe-TmKGHGzKD4Hpycm9yrUw$>:
> org.apache.hadoop.fs.azurebfs.oauth2.MsiTokenProvider
>
>     fs.azure.account.oauth2.msi.tenant. <storageaccountname>.
> dfs.core.windows.net
> <https://urldefense.com/v3/__http:/dfs.core.windows.net__;!!BhdT!nslIUVS9K-rzMRvjMFpWqBpcsAIiVPAfG6uroDOiSSQfmARHAQCYweWSe-TmKGHGzKD4Hpycm9yrUw$>:
> <MY TENANT ID>
>
>     fs.azure.account.oauth2.client.id
> <https://urldefense.com/v3/__http:/fs.azure.account.oauth2.client.id__;!!BhdT!nslIUVS9K-rzMRvjMFpWqBpcsAIiVPAfG6uroDOiSSQfmARHAQCYweWSe-TmKGHGzKD4HpwRB0LkWg$>.
> <storageaccountname>.dfs.core.windows.net
> <https://urldefense.com/v3/__http:/dfs.core.windows.net__;!!BhdT!nslIUVS9K-rzMRvjMFpWqBpcsAIiVPAfG6uroDOiSSQfmARHAQCYweWSe-TmKGHGzKD4Hpycm9yrUw$>:
> <MY CLIENT ID of VM>
>
>     fs.azure.account.oauth2.client.endpoint. <storageaccountname>.
> dfs.core.windows.net
> <https://urldefense.com/v3/__http:/dfs.core.windows.net__;!!BhdT!nslIUVS9K-rzMRvjMFpWqBpcsAIiVPAfG6uroDOiSSQfmARHAQCYweWSe-TmKGHGzKD4Hpycm9yrUw$>:
> https://login.microsoftonline.com/
> <https://urldefense.com/v3/__https:/login.microsoftonline.com/__;!!BhdT!nslIUVS9K-rzMRvjMFpWqBpcsAIiVPAfG6uroDOiSSQfmARHAQCYweWSe-TmKGHGzKD4HpzeWh7XLg$><MY
> TENANT ID>/oauth2/token
>
>
>
> I also have this specified in the container environment variables.
>
> - name: ENABLE_BUILT_IN_PLUGINS
>
>    value: flink-azure-fs-hadoop-1.16.1.jar
>
>
>
> I think I’m missing a configuration step because the MsiTokenProvider
> class is not found based on the logs. Any help would be appreciated.
>
>
>
>
>
> *Chris deRocco*
>
> Senior – Cybersecurity
>
> Chief Security Office | STORM Threat Analytics
>
>
>
> *AT&T*
>
> Middletown, NJ
>
> Phone: 732-639-9342
>
> Email: cd9...@att.com
>
>
>
>
>
>

Reply via email to