You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/02/16 21:05:41 UTC

[GitHub] [nifi] jfrazee commented on a change in pull request #4367: NIFI-7573: Add an Azure Active Directory (AAD) User Group Provider

jfrazee commented on a change in pull request #4367:
URL: https://github.com/apache/nifi/pull/4367#discussion_r571296835



##########
File path: nifi-docs/src/main/asciidoc/administration-guide.adoc
##########
@@ -494,6 +494,46 @@ The ShellUserGroupProvider has the following properties:
 
 Like LdapUserGroupProvider, the ShellUserGroupProvider is commented out in the _authorizers.xml_ file.  Refer to that comment for usage examples.
 
+==== AzureGraphUserGroupProvider
+
+The AzureGraphUserGroupProvider fetches users and group details from Azure Activy Driectory (AAD) using Azure graph rest-api & SDK.

Review comment:
       ```suggestion
   The AzureGraphUserGroupProvider fetches users and group details from Azure Activy Driectory (AAD) using the Azure Active Directory Graph API.
   ```

##########
File path: nifi-docs/src/main/asciidoc/administration-guide.adoc
##########
@@ -494,6 +494,46 @@ The ShellUserGroupProvider has the following properties:
 
 Like LdapUserGroupProvider, the ShellUserGroupProvider is commented out in the _authorizers.xml_ file.  Refer to that comment for usage examples.
 
+==== AzureGraphUserGroupProvider
+
+The AzureGraphUserGroupProvider fetches users and group details from Azure Activy Driectory (AAD) using Azure graph rest-api & SDK.
+
+This provider first fetches the list of the groups based on group filter specification (e.g. prefix, suffix, substring of group names, or list of group names).
+The group filter conditions are evaluated against _displayName_ property of Group directory object. From the list of groups, this provider loads member users of these groups.
+
+Pre-requsites of this provider on Azure Activy Driectory (AAD) side are (link:https://docs.microsoft.com/en-us/graph/auth-v2-service[from Azure graph doc^]):
+
+* Register your NiFi Application to AAD
+* Configure application type permissions for Microsoft Graph on your app (Group.Read.All,User.Read.All)
+* Grant admin consent for the configured permissions
+* Create a client secret for your application
+* Enable 'upn' optional claim for ID token
+
+Reference the following sample and example for detail instructions to configure pre-requsites:
+
+* link:https://docs.microsoft.com/en-us/azure/active-directory/develop/scenario-daemon-app-registration[sample^]
+* link:https://github.com/Azure-Samples/ms-identity-java-daemon[another-example^]
+
+To enable `upn` optional claim for ID token, follow the instructions in link:https://docs.microsoft.com/en-us/azure/active-directory/develop/active-directory-optional-claims#configuring-optional-claims[configuring-optional-claims^],
+disable 'Replace has marks' option, and set `nifi.security.user.oidc.claim.identifying.user=upn` in _nifi.properties_ file.
+
+The AzureGraphUserGroupProvider has the following properties:
+
+[options="header,footer"]
+|==================================================================================================================================================
+| Property Name | Description
+|`REFRESH_DELAY` | Duration of delay between each user and group refresh. Default is `5 mins`.
+|`AUTHORITY_ENDPOINT` | The endpoint of your AAD login. This can be found in your Azure AD application registration -> overview -> Endpoint. For example, global authority endpoint is https://login.microsoftonline.com, and US Gov cloud authority endpoint is https://login.microsoftonline.us.
+|`TENANT_ID` | Tenant ID or Directory ID. This can be found in your AAD application registration -> overview -> Directory (Tenant) ID

Review comment:
       ```suggestion
   |`TENANT_ID` | Tenant ID or Directory ID. This can be found in the Azure portal under Azure Active Directory -> App registrations -> [application name] -> Directory (tenant) ID.
   ```

##########
File path: nifi-docs/src/main/asciidoc/administration-guide.adoc
##########
@@ -494,6 +494,46 @@ The ShellUserGroupProvider has the following properties:
 
 Like LdapUserGroupProvider, the ShellUserGroupProvider is commented out in the _authorizers.xml_ file.  Refer to that comment for usage examples.
 
+==== AzureGraphUserGroupProvider
+
+The AzureGraphUserGroupProvider fetches users and group details from Azure Activy Driectory (AAD) using Azure graph rest-api & SDK.
+
+This provider first fetches the list of the groups based on group filter specification (e.g. prefix, suffix, substring of group names, or list of group names).
+The group filter conditions are evaluated against _displayName_ property of Group directory object. From the list of groups, this provider loads member users of these groups.
+
+Pre-requsites of this provider on Azure Activy Driectory (AAD) side are (link:https://docs.microsoft.com/en-us/graph/auth-v2-service[from Azure graph doc^]):
+
+* Register your NiFi Application to AAD
+* Configure application type permissions for Microsoft Graph on your app (Group.Read.All,User.Read.All)
+* Grant admin consent for the configured permissions
+* Create a client secret for your application
+* Enable 'upn' optional claim for ID token
+
+Reference the following sample and example for detail instructions to configure pre-requsites:
+
+* link:https://docs.microsoft.com/en-us/azure/active-directory/develop/scenario-daemon-app-registration[sample^]
+* link:https://github.com/Azure-Samples/ms-identity-java-daemon[another-example^]
+
+To enable `upn` optional claim for ID token, follow the instructions in link:https://docs.microsoft.com/en-us/azure/active-directory/develop/active-directory-optional-claims#configuring-optional-claims[configuring-optional-claims^],
+disable 'Replace has marks' option, and set `nifi.security.user.oidc.claim.identifying.user=upn` in _nifi.properties_ file.
+
+The AzureGraphUserGroupProvider has the following properties:
+
+[options="header,footer"]
+|==================================================================================================================================================
+| Property Name | Description
+|`REFRESH_DELAY` | Duration of delay between each user and group refresh. Default is `5 mins`.
+|`AUTHORITY_ENDPOINT` | The endpoint of your AAD login. This can be found in your Azure AD application registration -> overview -> Endpoint. For example, global authority endpoint is https://login.microsoftonline.com, and US Gov cloud authority endpoint is https://login.microsoftonline.us.
+|`TENANT_ID` | Tenant ID or Directory ID. This can be found in your AAD application registration -> overview -> Directory (Tenant) ID
+|`APP_REG_CLIENT_ID` | Client ID or application ID of your Azure AD application registration

Review comment:
       ```suggestion
   |`APP_REG_CLIENT_ID` | Client ID or Application ID of the Azure AD app registration. This can be found in the Azure portal under Azure Active Directory -> App registrations -> [application name] -> Overview -> Application (client) ID.
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.net.MalformedURLException;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import com.microsoft.aad.msal4j.ClientCredentialFactory;
+import com.microsoft.aad.msal4j.ClientCredentialParameters;
+import com.microsoft.aad.msal4j.ConfidentialClientApplication;
+import com.microsoft.aad.msal4j.IAuthenticationResult;
+import com.microsoft.graph.authentication.IAuthenticationProvider;
+import com.microsoft.graph.http.IHttpRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientCredentialAuthProvider implements IAuthenticationProvider {
+
+    final private String authorityEndpoint; // { "Global", "https://login.microsoftonline.com/", "UsGovernment", "https://login.microsoftonline.us/" }
+    final private String tenantId;
+    final private String clientId;
+    final private String clientSecret;

Review comment:
       ```suggestion
       private final String authorityEndpoint;
       private final String tenantId;
       private final String clientId;
       private final String clientSecret;
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.net.MalformedURLException;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import com.microsoft.aad.msal4j.ClientCredentialFactory;
+import com.microsoft.aad.msal4j.ClientCredentialParameters;
+import com.microsoft.aad.msal4j.ConfidentialClientApplication;
+import com.microsoft.aad.msal4j.IAuthenticationResult;
+import com.microsoft.graph.authentication.IAuthenticationProvider;
+import com.microsoft.graph.http.IHttpRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientCredentialAuthProvider implements IAuthenticationProvider {
+
+    final private String authorityEndpoint; // { "Global", "https://login.microsoftonline.com/", "UsGovernment", "https://login.microsoftonline.us/" }
+    final private String tenantId;
+    final private String clientId;
+    final private String clientSecret;
+    private Date tokenExpiresOnDate;
+    private String lastAcessToken;
+    private final static String GRAPH_DEFAULT_SCOPE = "https://graph.microsoft.com/.default";
+    private final static Logger logger = LoggerFactory.getLogger(ClientCredentialAuthProvider.class);
+
+    private ClientCredentialAuthProvider(final Builder builder){
+        this.authorityEndpoint = builder.getAuthorityEndpoint();
+        this.tenantId = builder.getTenantId();
+        this.clientId = builder.getClientId();
+        this.clientSecret = builder.getClientSecret();
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(authorityEndpoint, tenantId, clientId, clientSecret);
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+            " authorityDNS='" + authorityEndpoint + "'" +
+            ", tenantId='" + tenantId + "'" +
+            ", clientId='" + clientId + "'" +
+            ", clientSecret='" + clientSecret + "'" +
+            "}";
+    }
+
+    public Date getLastAccessTokenExpirationDate(){
+        return tokenExpiresOnDate;
+    }
+
+    private IAuthenticationResult getAccessTokenByClientCredentialGrant()
+        throws MalformedURLException, ExecutionException, InterruptedException {
+
+        ConfidentialClientApplication app = ConfidentialClientApplication.builder(
+                this.clientId,
+                ClientCredentialFactory.createFromSecret(this.clientSecret))
+                .authority(String.format("%s/%s", authorityEndpoint, tenantId))
+                .build();
+
+        // With client credentials flows the scope is ALWAYS of the shape "resource/.default", as the
+        // application permissions need to be set statically (in the portal), and then granted by a tenant administrator
+        ClientCredentialParameters clientCredentialParam = ClientCredentialParameters.builder(
+                Collections.singleton(GRAPH_DEFAULT_SCOPE))
+                .build();
+
+        CompletableFuture<IAuthenticationResult> future = app.acquireToken(clientCredentialParam);
+        return future.get();
+    }
+
+    private String getAccessToken() {
+        Date now = new Date();
+        if ((lastAcessToken != null) && (tokenExpiresOnDate != null) && (tokenExpiresOnDate.getTime() -now.getTime() > 60000)) {
+            return lastAcessToken;
+        } else {
+            try {
+                IAuthenticationResult result = getAccessTokenByClientCredentialGrant();
+                tokenExpiresOnDate = result.expiresOnDate(); // store this for token expiration checking
+                lastAcessToken = result.accessToken();
+            } catch(Exception ex) {
+                logger.error("Failed to get access token", ex);
+            }
+            return lastAcessToken;
+        }
+
+
+    }
+
+    @Override
+    public void authenticateRequest(IHttpRequest request) {
+        String accessToken = getAccessToken();
+        if (accessToken != null) {
+            request.addHeader("Authorization", "Bearer " + accessToken);
+        }
+
+    }
+
+    public static class Builder {
+
+        private String authorityEndpoint ="";
+        private String tenantId = "";
+        private String clientId = "";
+        private String clientSecret = "";
+
+        public Builder() {
+
+        }
+
+        public Builder authorityEndpoint(final String authorityEndpoint){
+            this.authorityEndpoint = authorityEndpoint;
+            return this;
+        }
+
+        public String getAuthorityEndpoint() {
+            return this.authorityEndpoint;
+        }
+
+        public Builder tenantId(final String tenantId){
+            this.tenantId = tenantId;
+            return this;
+        }
+
+        public String getTenantId() {
+            return this.tenantId;
+        }
+
+        public Builder clientId(final String clientId){
+            this.clientId = clientId;
+            return this;
+        }
+
+        public String getClientId() {
+            return this.clientId;
+        }
+
+        public Builder clientSecret(final String clientSecret){
+            this.clientSecret = clientSecret;
+            return this;
+        }
+
+        public String getClientSecret() {
+            return this.clientSecret;
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(authorityEndpoint, tenantId, clientId, clientSecret);
+        }
+
+        @Override
+        public String toString() {
+            return "{" +
+                " authorityDNS='" + getAuthorityEndpoint() + "'" +
+                ", tenantId='" + getTenantId() + "'" +
+                ", clientId='" + getClientId() + "'" +
+                ", clientSecret='" + getClientSecret() + "'" +
+                "}";
+        }
+        public ClientCredentialAuthProvider build() {
+            return new ClientCredentialAuthProvider(this);
+        }
+
+    }
+}

Review comment:
       Add trailing newline.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ImmutableAzureGraphUserGroup.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import static java.util.stream.Collectors.toMap;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+
+public class ImmutableAzureGraphUserGroup {
+    private final Set<User> users;
+    private final Map<String, User> usersByObjectId;
+    private final Map<String, User> usersByPrincipalName;
+    private final Map<String, UserAndGroups> usersAndGroupsByUserObjectId;
+
+    private final Set<Group> groups;
+    private final Map<String, Group> groupsByObjectId;
+    private final Map<String, Group> groupsByDisplayName;
+
+    private ImmutableAzureGraphUserGroup(
+        final Set<User> users,
+        final Map<String, User> usersByObjectId,
+        final Map<String, User> usersByPrincipalName,
+        final Map<String, UserAndGroups> usersAndGroupsByUserObjectId,
+        final Set<Group> groups,
+        final Map<String, Group> groupsByObjectId,
+        final Map<String, Group> groupsByDisplayName) {
+
+        this.users = Collections.unmodifiableSet(users);
+        this.usersByObjectId = Collections.unmodifiableMap(usersByObjectId);
+        this.usersByPrincipalName = Collections.unmodifiableMap(usersByPrincipalName);
+        this.usersAndGroupsByUserObjectId = Collections.unmodifiableMap(usersAndGroupsByUserObjectId);
+
+        this.groups = Collections.unmodifiableSet(groups);
+        this.groupsByObjectId = Collections.unmodifiableMap(groupsByObjectId);
+        this.groupsByDisplayName = Collections.unmodifiableMap(groupsByDisplayName);
+    }
+
+    public Set<User> getUsers() {
+        return users;
+    }
+
+    public User getUser(final String objectId) {
+        return usersByObjectId.get(objectId);
+    }
+
+    public User getUserByPrincipalName(final String principalName) {
+        return usersByPrincipalName.get(principalName);
+    }
+    public UserAndGroups getUserAndGroupsByUserObjectId(final String objectId) {
+        return usersAndGroupsByUserObjectId.get(objectId);
+    }

Review comment:
       ```suggestion
       }
   
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }

Review comment:
       ```suggestion
       @Override
        public void initialize(UserGroupProviderInitializationContext initializationContext) throws AuthorizerCreationException {
           this.scheduler = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() {
               @Override
               public Thread newThread(Runnable r) {
                   final Thread thread = Executors.defaultThreadFactory().newThread(r);
                   thread.setName(String.format("%s (%s) - UserGroup Refresh", getClass().getSimpleName(), initializationContext.getIdentifier()));
                   return thread;
               }
           });
        }
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api

Review comment:
       ```suggestion
       // client side group filter 'endswith' operator, due to support limiation of azure graph rest-api
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }

Review comment:
       ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.net.MalformedURLException;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import com.microsoft.aad.msal4j.ClientCredentialFactory;
+import com.microsoft.aad.msal4j.ClientCredentialParameters;
+import com.microsoft.aad.msal4j.ConfidentialClientApplication;
+import com.microsoft.aad.msal4j.IAuthenticationResult;
+import com.microsoft.graph.authentication.IAuthenticationProvider;
+import com.microsoft.graph.http.IHttpRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientCredentialAuthProvider implements IAuthenticationProvider {
+
+    final private String authorityEndpoint; // { "Global", "https://login.microsoftonline.com/", "UsGovernment", "https://login.microsoftonline.us/" }
+    final private String tenantId;
+    final private String clientId;
+    final private String clientSecret;
+    private Date tokenExpiresOnDate;

Review comment:
       Can we switch to `java.time.LocalDateTime`?
   ```suggestion
       private LocalDateTime tokenExpiresOnDate;
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.net.MalformedURLException;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import com.microsoft.aad.msal4j.ClientCredentialFactory;
+import com.microsoft.aad.msal4j.ClientCredentialParameters;
+import com.microsoft.aad.msal4j.ConfidentialClientApplication;
+import com.microsoft.aad.msal4j.IAuthenticationResult;
+import com.microsoft.graph.authentication.IAuthenticationProvider;
+import com.microsoft.graph.http.IHttpRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientCredentialAuthProvider implements IAuthenticationProvider {
+
+    final private String authorityEndpoint; // { "Global", "https://login.microsoftonline.com/", "UsGovernment", "https://login.microsoftonline.us/" }
+    final private String tenantId;
+    final private String clientId;
+    final private String clientSecret;
+    private Date tokenExpiresOnDate;
+    private String lastAcessToken;
+    private final static String GRAPH_DEFAULT_SCOPE = "https://graph.microsoft.com/.default";
+    private final static Logger logger = LoggerFactory.getLogger(ClientCredentialAuthProvider.class);
+
+    private ClientCredentialAuthProvider(final Builder builder){
+        this.authorityEndpoint = builder.getAuthorityEndpoint();
+        this.tenantId = builder.getTenantId();
+        this.clientId = builder.getClientId();
+        this.clientSecret = builder.getClientSecret();
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(authorityEndpoint, tenantId, clientId, clientSecret);
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+            " authorityDNS='" + authorityEndpoint + "'" +
+            ", tenantId='" + tenantId + "'" +
+            ", clientId='" + clientId + "'" +
+            ", clientSecret='" + clientSecret + "'" +
+            "}";
+    }
+
+    public Date getLastAccessTokenExpirationDate(){
+        return tokenExpiresOnDate;
+    }
+
+    private IAuthenticationResult getAccessTokenByClientCredentialGrant()
+        throws MalformedURLException, ExecutionException, InterruptedException {
+
+        ConfidentialClientApplication app = ConfidentialClientApplication.builder(
+                this.clientId,
+                ClientCredentialFactory.createFromSecret(this.clientSecret))
+                .authority(String.format("%s/%s", authorityEndpoint, tenantId))
+                .build();
+
+        // With client credentials flows the scope is ALWAYS of the shape "resource/.default", as the
+        // application permissions need to be set statically (in the portal), and then granted by a tenant administrator
+        ClientCredentialParameters clientCredentialParam = ClientCredentialParameters.builder(
+                Collections.singleton(GRAPH_DEFAULT_SCOPE))
+                .build();
+
+        CompletableFuture<IAuthenticationResult> future = app.acquireToken(clientCredentialParam);
+        return future.get();
+    }
+
+    private String getAccessToken() {
+        Date now = new Date();
+        if ((lastAcessToken != null) && (tokenExpiresOnDate != null) && (tokenExpiresOnDate.getTime() -now.getTime() > 60000)) {
+            return lastAcessToken;
+        } else {
+            try {
+                IAuthenticationResult result = getAccessTokenByClientCredentialGrant();
+                tokenExpiresOnDate = result.expiresOnDate(); // store this for token expiration checking
+                lastAcessToken = result.accessToken();
+            } catch(Exception ex) {
+                logger.error("Failed to get access token", ex);
+            }
+            return lastAcessToken;
+        }
+
+
+    }

Review comment:
       ```suggestion
       }
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }

Review comment:
       ```suggestion
           if (pageSize > MAX_PAGE_SIZE) {
                throw new AuthorizerCreationException(String.format("Max page size for Microsoft Graph is %d.", MAX_PAGE_SIZE));
            }
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));

Review comment:
       ```suggestion
                       String.format("%s is a required field for %s", TENANT_ID_PROPERTY, getClass().getSimpleName()));
   ```

##########
File path: nifi-docs/src/main/asciidoc/administration-guide.adoc
##########
@@ -494,6 +494,46 @@ The ShellUserGroupProvider has the following properties:
 
 Like LdapUserGroupProvider, the ShellUserGroupProvider is commented out in the _authorizers.xml_ file.  Refer to that comment for usage examples.
 
+==== AzureGraphUserGroupProvider
+
+The AzureGraphUserGroupProvider fetches users and group details from Azure Activy Driectory (AAD) using Azure graph rest-api & SDK.
+
+This provider first fetches the list of the groups based on group filter specification (e.g. prefix, suffix, substring of group names, or list of group names).
+The group filter conditions are evaluated against _displayName_ property of Group directory object. From the list of groups, this provider loads member users of these groups.
+
+Pre-requsites of this provider on Azure Activy Driectory (AAD) side are (link:https://docs.microsoft.com/en-us/graph/auth-v2-service[from Azure graph doc^]):
+
+* Register your NiFi Application to AAD
+* Configure application type permissions for Microsoft Graph on your app (Group.Read.All,User.Read.All)
+* Grant admin consent for the configured permissions
+* Create a client secret for your application
+* Enable 'upn' optional claim for ID token
+
+Reference the following sample and example for detail instructions to configure pre-requsites:
+
+* link:https://docs.microsoft.com/en-us/azure/active-directory/develop/scenario-daemon-app-registration[sample^]
+* link:https://github.com/Azure-Samples/ms-identity-java-daemon[another-example^]
+
+To enable `upn` optional claim for ID token, follow the instructions in link:https://docs.microsoft.com/en-us/azure/active-directory/develop/active-directory-optional-claims#configuring-optional-claims[configuring-optional-claims^],
+disable 'Replace has marks' option, and set `nifi.security.user.oidc.claim.identifying.user=upn` in _nifi.properties_ file.
+
+The AzureGraphUserGroupProvider has the following properties:
+
+[options="header,footer"]
+|==================================================================================================================================================
+| Property Name | Description
+|`REFRESH_DELAY` | Duration of delay between each user and group refresh. Default is `5 mins`.
+|`AUTHORITY_ENDPOINT` | The endpoint of your AAD login. This can be found in your Azure AD application registration -> overview -> Endpoint. For example, global authority endpoint is https://login.microsoftonline.com, and US Gov cloud authority endpoint is https://login.microsoftonline.us.
+|`TENANT_ID` | Tenant ID or Directory ID. This can be found in your AAD application registration -> overview -> Directory (Tenant) ID
+|`APP_REG_CLIENT_ID` | Client ID or application ID of your Azure AD application registration
+|`APP_REG_CLIENT_SECRET` | Secret you have created and  saved in Azure AD application registration ->  Certificates & secrets

Review comment:
       ```suggestion
   |`APP_REG_CLIENT_SECRET` | A client secret from the Azure AD app registration. Secrets can be created in the Azure Portal under Azure Active Directory -> App registrations -> [application name] -> Certificates & secrets -> Client secrets -> [+] New client secret
   ```

##########
File path: nifi-docs/src/main/asciidoc/administration-guide.adoc
##########
@@ -494,6 +494,46 @@ The ShellUserGroupProvider has the following properties:
 
 Like LdapUserGroupProvider, the ShellUserGroupProvider is commented out in the _authorizers.xml_ file.  Refer to that comment for usage examples.
 
+==== AzureGraphUserGroupProvider
+
+The AzureGraphUserGroupProvider fetches users and group details from Azure Activy Driectory (AAD) using Azure graph rest-api & SDK.
+
+This provider first fetches the list of the groups based on group filter specification (e.g. prefix, suffix, substring of group names, or list of group names).
+The group filter conditions are evaluated against _displayName_ property of Group directory object. From the list of groups, this provider loads member users of these groups.
+
+Pre-requsites of this provider on Azure Activy Driectory (AAD) side are (link:https://docs.microsoft.com/en-us/graph/auth-v2-service[from Azure graph doc^]):
+
+* Register your NiFi Application to AAD
+* Configure application type permissions for Microsoft Graph on your app (Group.Read.All,User.Read.All)
+* Grant admin consent for the configured permissions
+* Create a client secret for your application
+* Enable 'upn' optional claim for ID token
+
+Reference the following sample and example for detail instructions to configure pre-requsites:
+
+* link:https://docs.microsoft.com/en-us/azure/active-directory/develop/scenario-daemon-app-registration[sample^]
+* link:https://github.com/Azure-Samples/ms-identity-java-daemon[another-example^]
+
+To enable `upn` optional claim for ID token, follow the instructions in link:https://docs.microsoft.com/en-us/azure/active-directory/develop/active-directory-optional-claims#configuring-optional-claims[configuring-optional-claims^],
+disable 'Replace has marks' option, and set `nifi.security.user.oidc.claim.identifying.user=upn` in _nifi.properties_ file.
+
+The AzureGraphUserGroupProvider has the following properties:
+
+[options="header,footer"]
+|==================================================================================================================================================
+| Property Name | Description
+|`REFRESH_DELAY` | Duration of delay between each user and group refresh. Default is `5 mins`.
+|`AUTHORITY_ENDPOINT` | The endpoint of your AAD login. This can be found in your Azure AD application registration -> overview -> Endpoint. For example, global authority endpoint is https://login.microsoftonline.com, and US Gov cloud authority endpoint is https://login.microsoftonline.us.
+|`TENANT_ID` | Tenant ID or Directory ID. This can be found in your AAD application registration -> overview -> Directory (Tenant) ID
+|`APP_REG_CLIENT_ID` | Client ID or application ID of your Azure AD application registration
+|`APP_REG_CLIENT_SECRET` | Secret you have created and  saved in Azure AD application registration ->  Certificates & secrets
+|`GROUP_FILTER_PREFIX` | Prefix will match against the displayName property of groups and retrives only those matching groups

Review comment:
       ```suggestion
   |`GROUP_FILTER_PREFIX` | Prefix filter for Azure AD groups. Matches against the group displayName to retrieve only groups with names starting with the provided prefix.
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.net.MalformedURLException;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import com.microsoft.aad.msal4j.ClientCredentialFactory;
+import com.microsoft.aad.msal4j.ClientCredentialParameters;
+import com.microsoft.aad.msal4j.ConfidentialClientApplication;
+import com.microsoft.aad.msal4j.IAuthenticationResult;
+import com.microsoft.graph.authentication.IAuthenticationProvider;
+import com.microsoft.graph.http.IHttpRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientCredentialAuthProvider implements IAuthenticationProvider {
+
+    final private String authorityEndpoint; // { "Global", "https://login.microsoftonline.com/", "UsGovernment", "https://login.microsoftonline.us/" }
+    final private String tenantId;
+    final private String clientId;
+    final private String clientSecret;
+    private Date tokenExpiresOnDate;
+    private String lastAcessToken;
+    private final static String GRAPH_DEFAULT_SCOPE = "https://graph.microsoft.com/.default";
+    private final static Logger logger = LoggerFactory.getLogger(ClientCredentialAuthProvider.class);
+
+    private ClientCredentialAuthProvider(final Builder builder){
+        this.authorityEndpoint = builder.getAuthorityEndpoint();
+        this.tenantId = builder.getTenantId();
+        this.clientId = builder.getClientId();
+        this.clientSecret = builder.getClientSecret();
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(authorityEndpoint, tenantId, clientId, clientSecret);
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+            " authorityDNS='" + authorityEndpoint + "'" +
+            ", tenantId='" + tenantId + "'" +
+            ", clientId='" + clientId + "'" +
+            ", clientSecret='" + clientSecret + "'" +
+            "}";
+    }
+
+    public Date getLastAccessTokenExpirationDate(){
+        return tokenExpiresOnDate;
+    }
+
+    private IAuthenticationResult getAccessTokenByClientCredentialGrant()
+        throws MalformedURLException, ExecutionException, InterruptedException {
+
+        ConfidentialClientApplication app = ConfidentialClientApplication.builder(
+                this.clientId,
+                ClientCredentialFactory.createFromSecret(this.clientSecret))
+                .authority(String.format("%s/%s", authorityEndpoint, tenantId))
+                .build();
+
+        // With client credentials flows the scope is ALWAYS of the shape "resource/.default", as the
+        // application permissions need to be set statically (in the portal), and then granted by a tenant administrator
+        ClientCredentialParameters clientCredentialParam = ClientCredentialParameters.builder(
+                Collections.singleton(GRAPH_DEFAULT_SCOPE))
+                .build();
+
+        CompletableFuture<IAuthenticationResult> future = app.acquireToken(clientCredentialParam);
+        return future.get();
+    }
+
+    private String getAccessToken() {
+        Date now = new Date();
+        if ((lastAcessToken != null) && (tokenExpiresOnDate != null) && (tokenExpiresOnDate.getTime() -now.getTime() > 60000)) {
+            return lastAcessToken;
+        } else {
+            try {
+                IAuthenticationResult result = getAccessTokenByClientCredentialGrant();
+                tokenExpiresOnDate = result.expiresOnDate(); // store this for token expiration checking
+                lastAcessToken = result.accessToken();
+            } catch(Exception ex) {
+                logger.error("Failed to get access token", ex);
+            }
+            return lastAcessToken;
+        }
+
+
+    }
+
+    @Override
+    public void authenticateRequest(IHttpRequest request) {
+        String accessToken = getAccessToken();
+        if (accessToken != null) {
+            request.addHeader("Authorization", "Bearer " + accessToken);
+        }
+
+    }

Review comment:
       ```suggestion
       }
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ImmutableAzureGraphUserGroup.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import static java.util.stream.Collectors.toMap;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+
+public class ImmutableAzureGraphUserGroup {
+    private final Set<User> users;
+    private final Map<String, User> usersByObjectId;
+    private final Map<String, User> usersByPrincipalName;
+    private final Map<String, UserAndGroups> usersAndGroupsByUserObjectId;
+
+    private final Set<Group> groups;
+    private final Map<String, Group> groupsByObjectId;
+    private final Map<String, Group> groupsByDisplayName;
+
+    private ImmutableAzureGraphUserGroup(
+        final Set<User> users,
+        final Map<String, User> usersByObjectId,
+        final Map<String, User> usersByPrincipalName,
+        final Map<String, UserAndGroups> usersAndGroupsByUserObjectId,
+        final Set<Group> groups,
+        final Map<String, Group> groupsByObjectId,
+        final Map<String, Group> groupsByDisplayName) {
+
+        this.users = Collections.unmodifiableSet(users);
+        this.usersByObjectId = Collections.unmodifiableMap(usersByObjectId);
+        this.usersByPrincipalName = Collections.unmodifiableMap(usersByPrincipalName);
+        this.usersAndGroupsByUserObjectId = Collections.unmodifiableMap(usersAndGroupsByUserObjectId);
+
+        this.groups = Collections.unmodifiableSet(groups);
+        this.groupsByObjectId = Collections.unmodifiableMap(groupsByObjectId);
+        this.groupsByDisplayName = Collections.unmodifiableMap(groupsByDisplayName);
+    }
+
+    public Set<User> getUsers() {
+        return users;
+    }
+
+    public User getUser(final String objectId) {
+        return usersByObjectId.get(objectId);
+    }
+
+    public User getUserByPrincipalName(final String principalName) {
+        return usersByPrincipalName.get(principalName);
+    }

Review comment:
       ```suggestion
       }
   
   ```

##########
File path: nifi-docs/src/main/asciidoc/administration-guide.adoc
##########
@@ -494,6 +494,46 @@ The ShellUserGroupProvider has the following properties:
 
 Like LdapUserGroupProvider, the ShellUserGroupProvider is commented out in the _authorizers.xml_ file.  Refer to that comment for usage examples.
 
+==== AzureGraphUserGroupProvider
+
+The AzureGraphUserGroupProvider fetches users and group details from Azure Activy Driectory (AAD) using Azure graph rest-api & SDK.
+
+This provider first fetches the list of the groups based on group filter specification (e.g. prefix, suffix, substring of group names, or list of group names).
+The group filter conditions are evaluated against _displayName_ property of Group directory object. From the list of groups, this provider loads member users of these groups.
+
+Pre-requsites of this provider on Azure Activy Driectory (AAD) side are (link:https://docs.microsoft.com/en-us/graph/auth-v2-service[from Azure graph doc^]):
+
+* Register your NiFi Application to AAD
+* Configure application type permissions for Microsoft Graph on your app (Group.Read.All,User.Read.All)
+* Grant admin consent for the configured permissions
+* Create a client secret for your application
+* Enable 'upn' optional claim for ID token
+
+Reference the following sample and example for detail instructions to configure pre-requsites:
+
+* link:https://docs.microsoft.com/en-us/azure/active-directory/develop/scenario-daemon-app-registration[sample^]
+* link:https://github.com/Azure-Samples/ms-identity-java-daemon[another-example^]
+
+To enable `upn` optional claim for ID token, follow the instructions in link:https://docs.microsoft.com/en-us/azure/active-directory/develop/active-directory-optional-claims#configuring-optional-claims[configuring-optional-claims^],
+disable 'Replace has marks' option, and set `nifi.security.user.oidc.claim.identifying.user=upn` in _nifi.properties_ file.
+
+The AzureGraphUserGroupProvider has the following properties:
+
+[options="header,footer"]
+|==================================================================================================================================================
+| Property Name | Description
+|`REFRESH_DELAY` | Duration of delay between each user and group refresh. Default is `5 mins`.
+|`AUTHORITY_ENDPOINT` | The endpoint of your AAD login. This can be found in your Azure AD application registration -> overview -> Endpoint. For example, global authority endpoint is https://login.microsoftonline.com, and US Gov cloud authority endpoint is https://login.microsoftonline.us.
+|`TENANT_ID` | Tenant ID or Directory ID. This can be found in your AAD application registration -> overview -> Directory (Tenant) ID
+|`APP_REG_CLIENT_ID` | Client ID or application ID of your Azure AD application registration
+|`APP_REG_CLIENT_SECRET` | Secret you have created and  saved in Azure AD application registration ->  Certificates & secrets
+|`GROUP_FILTER_PREFIX` | Prefix will match against the displayName property of groups and retrives only those matching groups
+|`GROUP_FILTER_SUFFIX` | Suffix will match against the displayName property of groups and retrives only those matching groups
+|`GROUP_FILTER_SUBSTRING` | Substring will match against the displayName property of groups and retrives only those groups that contain the substring

Review comment:
       ```suggestion
   |`GROUP_FILTER_SUBSTRING` | Substring filter for Azure AD Groups. Matches against the group displayName to retrieve only groups with names containing the provided substring.
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.net.MalformedURLException;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import com.microsoft.aad.msal4j.ClientCredentialFactory;
+import com.microsoft.aad.msal4j.ClientCredentialParameters;
+import com.microsoft.aad.msal4j.ConfidentialClientApplication;
+import com.microsoft.aad.msal4j.IAuthenticationResult;
+import com.microsoft.graph.authentication.IAuthenticationProvider;
+import com.microsoft.graph.http.IHttpRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientCredentialAuthProvider implements IAuthenticationProvider {
+
+    final private String authorityEndpoint; // { "Global", "https://login.microsoftonline.com/", "UsGovernment", "https://login.microsoftonline.us/" }
+    final private String tenantId;
+    final private String clientId;
+    final private String clientSecret;
+    private Date tokenExpiresOnDate;
+    private String lastAcessToken;
+    private final static String GRAPH_DEFAULT_SCOPE = "https://graph.microsoft.com/.default";
+    private final static Logger logger = LoggerFactory.getLogger(ClientCredentialAuthProvider.class);

Review comment:
       ```suggestion
   
       private static final String GRAPH_DEFAULT_SCOPE = "https://graph.microsoft.com/.default";
       private static final Logger logger = LoggerFactory.getLogger(ClientCredentialAuthProvider.class);
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);

Review comment:
       We need to sanitize `prefix` so we don't end up with some kind of code injection in `queryGroupsWith()`. I looked around but I wasn't able to find a definition of what valid group display names are.

##########
File path: nifi-docs/src/main/asciidoc/administration-guide.adoc
##########
@@ -494,6 +494,46 @@ The ShellUserGroupProvider has the following properties:
 
 Like LdapUserGroupProvider, the ShellUserGroupProvider is commented out in the _authorizers.xml_ file.  Refer to that comment for usage examples.
 
+==== AzureGraphUserGroupProvider
+
+The AzureGraphUserGroupProvider fetches users and group details from Azure Activy Driectory (AAD) using Azure graph rest-api & SDK.
+
+This provider first fetches the list of the groups based on group filter specification (e.g. prefix, suffix, substring of group names, or list of group names).
+The group filter conditions are evaluated against _displayName_ property of Group directory object. From the list of groups, this provider loads member users of these groups.
+
+Pre-requsites of this provider on Azure Activy Driectory (AAD) side are (link:https://docs.microsoft.com/en-us/graph/auth-v2-service[from Azure graph doc^]):
+
+* Register your NiFi Application to AAD
+* Configure application type permissions for Microsoft Graph on your app (Group.Read.All,User.Read.All)
+* Grant admin consent for the configured permissions
+* Create a client secret for your application
+* Enable 'upn' optional claim for ID token
+
+Reference the following sample and example for detail instructions to configure pre-requsites:
+
+* link:https://docs.microsoft.com/en-us/azure/active-directory/develop/scenario-daemon-app-registration[sample^]
+* link:https://github.com/Azure-Samples/ms-identity-java-daemon[another-example^]
+
+To enable `upn` optional claim for ID token, follow the instructions in link:https://docs.microsoft.com/en-us/azure/active-directory/develop/active-directory-optional-claims#configuring-optional-claims[configuring-optional-claims^],
+disable 'Replace has marks' option, and set `nifi.security.user.oidc.claim.identifying.user=upn` in _nifi.properties_ file.
+
+The AzureGraphUserGroupProvider has the following properties:
+
+[options="header,footer"]
+|==================================================================================================================================================
+| Property Name | Description
+|`REFRESH_DELAY` | Duration of delay between each user and group refresh. Default is `5 mins`.
+|`AUTHORITY_ENDPOINT` | The endpoint of your AAD login. This can be found in your Azure AD application registration -> overview -> Endpoint. For example, global authority endpoint is https://login.microsoftonline.com, and US Gov cloud authority endpoint is https://login.microsoftonline.us.
+|`TENANT_ID` | Tenant ID or Directory ID. This can be found in your AAD application registration -> overview -> Directory (Tenant) ID
+|`APP_REG_CLIENT_ID` | Client ID or application ID of your Azure AD application registration
+|`APP_REG_CLIENT_SECRET` | Secret you have created and  saved in Azure AD application registration ->  Certificates & secrets
+|`GROUP_FILTER_PREFIX` | Prefix will match against the displayName property of groups and retrives only those matching groups
+|`GROUP_FILTER_SUFFIX` | Suffix will match against the displayName property of groups and retrives only those matching groups

Review comment:
       ```suggestion
   |`GROUP_FILTER_SUFFIX` | Suffix filter for Azure AD groups. Matches against the group displayName to retrieve only groups with names ending with the provided suffix.
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";

Review comment:
       Can we use `com.azure.identity.AzureAuthorityHosts.AZURE_PUBLIC_CLOUD` for `"https://login.microsoftonline.com"` and skip defining and exposing our own constant? Since this isn't in it's own nar the processors already depend on it, it doesn't add anything new to the binary.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.net.MalformedURLException;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import com.microsoft.aad.msal4j.ClientCredentialFactory;
+import com.microsoft.aad.msal4j.ClientCredentialParameters;
+import com.microsoft.aad.msal4j.ConfidentialClientApplication;
+import com.microsoft.aad.msal4j.IAuthenticationResult;
+import com.microsoft.graph.authentication.IAuthenticationProvider;
+import com.microsoft.graph.http.IHttpRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientCredentialAuthProvider implements IAuthenticationProvider {
+
+    final private String authorityEndpoint; // { "Global", "https://login.microsoftonline.com/", "UsGovernment", "https://login.microsoftonline.us/" }
+    final private String tenantId;
+    final private String clientId;
+    final private String clientSecret;
+    private Date tokenExpiresOnDate;
+    private String lastAcessToken;
+    private final static String GRAPH_DEFAULT_SCOPE = "https://graph.microsoft.com/.default";
+    private final static Logger logger = LoggerFactory.getLogger(ClientCredentialAuthProvider.class);
+
+    private ClientCredentialAuthProvider(final Builder builder){
+        this.authorityEndpoint = builder.getAuthorityEndpoint();
+        this.tenantId = builder.getTenantId();
+        this.clientId = builder.getClientId();
+        this.clientSecret = builder.getClientSecret();
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(authorityEndpoint, tenantId, clientId, clientSecret);
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+            " authorityDNS='" + authorityEndpoint + "'" +
+            ", tenantId='" + tenantId + "'" +
+            ", clientId='" + clientId + "'" +
+            ", clientSecret='" + clientSecret + "'" +
+            "}";
+    }
+
+    public Date getLastAccessTokenExpirationDate(){
+        return tokenExpiresOnDate;
+    }
+
+    private IAuthenticationResult getAccessTokenByClientCredentialGrant()
+        throws MalformedURLException, ExecutionException, InterruptedException {
+
+        ConfidentialClientApplication app = ConfidentialClientApplication.builder(
+                this.clientId,
+                ClientCredentialFactory.createFromSecret(this.clientSecret))
+                .authority(String.format("%s/%s", authorityEndpoint, tenantId))
+                .build();
+
+        // With client credentials flows the scope is ALWAYS of the shape "resource/.default", as the
+        // application permissions need to be set statically (in the portal), and then granted by a tenant administrator
+        ClientCredentialParameters clientCredentialParam = ClientCredentialParameters.builder(
+                Collections.singleton(GRAPH_DEFAULT_SCOPE))
+                .build();
+
+        CompletableFuture<IAuthenticationResult> future = app.acquireToken(clientCredentialParam);
+        return future.get();
+    }
+
+    private String getAccessToken() {
+        Date now = new Date();
+        if ((lastAcessToken != null) && (tokenExpiresOnDate != null) && (tokenExpiresOnDate.getTime() -now.getTime() > 60000)) {
+            return lastAcessToken;
+        } else {
+            try {
+                IAuthenticationResult result = getAccessTokenByClientCredentialGrant();
+                tokenExpiresOnDate = result.expiresOnDate(); // store this for token expiration checking
+                lastAcessToken = result.accessToken();
+            } catch(Exception ex) {
+                logger.error("Failed to get access token", ex);
+            }
+            return lastAcessToken;
+        }
+
+
+    }
+
+    @Override
+    public void authenticateRequest(IHttpRequest request) {
+        String accessToken = getAccessToken();
+        if (accessToken != null) {
+            request.addHeader("Authorization", "Bearer " + accessToken);
+        }
+
+    }
+
+    public static class Builder {
+
+        private String authorityEndpoint ="";
+        private String tenantId = "";
+        private String clientId = "";
+        private String clientSecret = "";
+
+        public Builder() {
+
+        }
+
+        public Builder authorityEndpoint(final String authorityEndpoint){
+            this.authorityEndpoint = authorityEndpoint;
+            return this;
+        }
+
+        public String getAuthorityEndpoint() {
+            return this.authorityEndpoint;
+        }
+
+        public Builder tenantId(final String tenantId){
+            this.tenantId = tenantId;
+            return this;
+        }
+
+        public String getTenantId() {
+            return this.tenantId;
+        }
+
+        public Builder clientId(final String clientId){
+            this.clientId = clientId;
+            return this;
+        }
+
+        public String getClientId() {
+            return this.clientId;
+        }
+
+        public Builder clientSecret(final String clientSecret){
+            this.clientSecret = clientSecret;
+            return this;
+        }
+
+        public String getClientSecret() {
+            return this.clientSecret;
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(authorityEndpoint, tenantId, clientId, clientSecret);
+        }
+
+        @Override
+        public String toString() {
+            return "{" +
+                " authorityDNS='" + getAuthorityEndpoint() + "'" +
+                ", tenantId='" + getTenantId() + "'" +
+                ", clientId='" + getClientId() + "'" +
+                ", clientSecret='" + getClientSecret() + "'" +
+                "}";
+        }

Review comment:
       ```suggestion
           }
   
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);

Review comment:
       Can we move this to `initialize()`?
   ```suggestion
       private final ScheduledExecutorService scheduler;
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api

Review comment:
       ```suggestion
       // client side group filter 'contains' operator, due to support limiation of azure graph rest-api
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);

Review comment:
       ```suggestion
           final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY, DEFAULT_AAD_AUTHORITY_ENDPOINT);
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));

Review comment:
       ```suggestion
               throw new AuthorizerCreationException(String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD

Review comment:
       ```suggestion
       // comma separated list of group names to search from AAD
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {

Review comment:
       ```suggestion
       private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();

Review comment:
       ```suggestion
       private final AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";

Review comment:
       The other providers use title-cased text for these and not the constant format. So, e.g., `"REFRESH_DELAY"` would be `"Refresh Delay"`. I think we should make this change for the properties. As far as I can tell this would just have to be changed in these constants, the default authorizers.xml, and the docs.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));

Review comment:
       ```suggestion
           pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, DEFAULT_PAGE_SIZE));
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");

Review comment:
       ```suggestion
           this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, DEFAULT_CLAIM_FOR_USERNAME);
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.net.MalformedURLException;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import com.microsoft.aad.msal4j.ClientCredentialFactory;
+import com.microsoft.aad.msal4j.ClientCredentialParameters;
+import com.microsoft.aad.msal4j.ConfidentialClientApplication;
+import com.microsoft.aad.msal4j.IAuthenticationResult;
+import com.microsoft.graph.authentication.IAuthenticationProvider;
+import com.microsoft.graph.http.IHttpRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientCredentialAuthProvider implements IAuthenticationProvider {
+
+    final private String authorityEndpoint; // { "Global", "https://login.microsoftonline.com/", "UsGovernment", "https://login.microsoftonline.us/" }
+    final private String tenantId;
+    final private String clientId;
+    final private String clientSecret;
+    private Date tokenExpiresOnDate;
+    private String lastAcessToken;
+    private final static String GRAPH_DEFAULT_SCOPE = "https://graph.microsoft.com/.default";
+    private final static Logger logger = LoggerFactory.getLogger(ClientCredentialAuthProvider.class);
+
+    private ClientCredentialAuthProvider(final Builder builder){
+        this.authorityEndpoint = builder.getAuthorityEndpoint();
+        this.tenantId = builder.getTenantId();
+        this.clientId = builder.getClientId();
+        this.clientSecret = builder.getClientSecret();
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(authorityEndpoint, tenantId, clientId, clientSecret);
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+            " authorityDNS='" + authorityEndpoint + "'" +
+            ", tenantId='" + tenantId + "'" +
+            ", clientId='" + clientId + "'" +
+            ", clientSecret='" + clientSecret + "'" +
+            "}";
+    }
+
+    public Date getLastAccessTokenExpirationDate(){
+        return tokenExpiresOnDate;
+    }
+

Review comment:
       ```suggestion
   
       // This code for this method is derived from an example from https://github.com/Azure-Samples/ms-identity-java-daemon. 
       // It can be found in msal-client-credential-secret/src/main/java/ClientCredentialGrant.java
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.net.MalformedURLException;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import com.microsoft.aad.msal4j.ClientCredentialFactory;
+import com.microsoft.aad.msal4j.ClientCredentialParameters;
+import com.microsoft.aad.msal4j.ConfidentialClientApplication;
+import com.microsoft.aad.msal4j.IAuthenticationResult;
+import com.microsoft.graph.authentication.IAuthenticationProvider;
+import com.microsoft.graph.http.IHttpRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientCredentialAuthProvider implements IAuthenticationProvider {
+
+    final private String authorityEndpoint; // { "Global", "https://login.microsoftonline.com/", "UsGovernment", "https://login.microsoftonline.us/" }

Review comment:
       ```suggestion
       final private String authorityEndpoint;
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.net.MalformedURLException;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import com.microsoft.aad.msal4j.ClientCredentialFactory;
+import com.microsoft.aad.msal4j.ClientCredentialParameters;
+import com.microsoft.aad.msal4j.ConfidentialClientApplication;
+import com.microsoft.aad.msal4j.IAuthenticationResult;
+import com.microsoft.graph.authentication.IAuthenticationProvider;
+import com.microsoft.graph.http.IHttpRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientCredentialAuthProvider implements IAuthenticationProvider {
+
+    final private String authorityEndpoint; // { "Global", "https://login.microsoftonline.com/", "UsGovernment", "https://login.microsoftonline.us/" }
+    final private String tenantId;
+    final private String clientId;
+    final private String clientSecret;
+    private Date tokenExpiresOnDate;
+    private String lastAcessToken;
+    private final static String GRAPH_DEFAULT_SCOPE = "https://graph.microsoft.com/.default";
+    private final static Logger logger = LoggerFactory.getLogger(ClientCredentialAuthProvider.class);
+
+    private ClientCredentialAuthProvider(final Builder builder){
+        this.authorityEndpoint = builder.getAuthorityEndpoint();
+        this.tenantId = builder.getTenantId();
+        this.clientId = builder.getClientId();
+        this.clientSecret = builder.getClientSecret();
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(authorityEndpoint, tenantId, clientId, clientSecret);
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+            " authorityDNS='" + authorityEndpoint + "'" +
+            ", tenantId='" + tenantId + "'" +
+            ", clientId='" + clientId + "'" +
+            ", clientSecret='" + clientSecret + "'" +
+            "}";
+    }
+
+    public Date getLastAccessTokenExpirationDate(){
+        return tokenExpiresOnDate;
+    }
+
+    private IAuthenticationResult getAccessTokenByClientCredentialGrant()
+        throws MalformedURLException, ExecutionException, InterruptedException {
+
+        ConfidentialClientApplication app = ConfidentialClientApplication.builder(
+                this.clientId,
+                ClientCredentialFactory.createFromSecret(this.clientSecret))
+                .authority(String.format("%s/%s", authorityEndpoint, tenantId))
+                .build();
+
+        // With client credentials flows the scope is ALWAYS of the shape "resource/.default", as the
+        // application permissions need to be set statically (in the portal), and then granted by a tenant administrator
+        ClientCredentialParameters clientCredentialParam = ClientCredentialParameters.builder(
+                Collections.singleton(GRAPH_DEFAULT_SCOPE))
+                .build();
+
+        CompletableFuture<IAuthenticationResult> future = app.acquireToken(clientCredentialParam);
+        return future.get();
+    }
+
+    private String getAccessToken() {
+        Date now = new Date();
+        if ((lastAcessToken != null) && (tokenExpiresOnDate != null) && (tokenExpiresOnDate.getTime() -now.getTime() > 60000)) {

Review comment:
       ```suggestion
           if ((lastAcessToken != null) && (tokenExpiresOnDate != null) && tokenExpiresOnDate.after(LocalDateTime.now().plusMinutes(1))) {
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.net.MalformedURLException;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import com.microsoft.aad.msal4j.ClientCredentialFactory;
+import com.microsoft.aad.msal4j.ClientCredentialParameters;
+import com.microsoft.aad.msal4j.ConfidentialClientApplication;
+import com.microsoft.aad.msal4j.IAuthenticationResult;
+import com.microsoft.graph.authentication.IAuthenticationProvider;
+import com.microsoft.graph.http.IHttpRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientCredentialAuthProvider implements IAuthenticationProvider {
+
+    final private String authorityEndpoint; // { "Global", "https://login.microsoftonline.com/", "UsGovernment", "https://login.microsoftonline.us/" }
+    final private String tenantId;
+    final private String clientId;
+    final private String clientSecret;
+    private Date tokenExpiresOnDate;
+    private String lastAcessToken;
+    private final static String GRAPH_DEFAULT_SCOPE = "https://graph.microsoft.com/.default";
+    private final static Logger logger = LoggerFactory.getLogger(ClientCredentialAuthProvider.class);
+
+    private ClientCredentialAuthProvider(final Builder builder){
+        this.authorityEndpoint = builder.getAuthorityEndpoint();
+        this.tenantId = builder.getTenantId();
+        this.clientId = builder.getClientId();
+        this.clientSecret = builder.getClientSecret();
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(authorityEndpoint, tenantId, clientId, clientSecret);
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+            " authorityDNS='" + authorityEndpoint + "'" +
+            ", tenantId='" + tenantId + "'" +
+            ", clientId='" + clientId + "'" +
+            ", clientSecret='" + clientSecret + "'" +
+            "}";
+    }
+
+    public Date getLastAccessTokenExpirationDate(){
+        return tokenExpiresOnDate;
+    }
+
+    private IAuthenticationResult getAccessTokenByClientCredentialGrant()
+        throws MalformedURLException, ExecutionException, InterruptedException {
+
+        ConfidentialClientApplication app = ConfidentialClientApplication.builder(
+                this.clientId,
+                ClientCredentialFactory.createFromSecret(this.clientSecret))
+                .authority(String.format("%s/%s", authorityEndpoint, tenantId))
+                .build();
+
+        // With client credentials flows the scope is ALWAYS of the shape "resource/.default", as the
+        // application permissions need to be set statically (in the portal), and then granted by a tenant administrator

Review comment:
       ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.net.MalformedURLException;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import com.microsoft.aad.msal4j.ClientCredentialFactory;
+import com.microsoft.aad.msal4j.ClientCredentialParameters;
+import com.microsoft.aad.msal4j.ConfidentialClientApplication;
+import com.microsoft.aad.msal4j.IAuthenticationResult;
+import com.microsoft.graph.authentication.IAuthenticationProvider;
+import com.microsoft.graph.http.IHttpRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientCredentialAuthProvider implements IAuthenticationProvider {
+
+    final private String authorityEndpoint; // { "Global", "https://login.microsoftonline.com/", "UsGovernment", "https://login.microsoftonline.us/" }
+    final private String tenantId;
+    final private String clientId;
+    final private String clientSecret;
+    private Date tokenExpiresOnDate;
+    private String lastAcessToken;
+    private final static String GRAPH_DEFAULT_SCOPE = "https://graph.microsoft.com/.default";
+    private final static Logger logger = LoggerFactory.getLogger(ClientCredentialAuthProvider.class);
+
+    private ClientCredentialAuthProvider(final Builder builder){
+        this.authorityEndpoint = builder.getAuthorityEndpoint();
+        this.tenantId = builder.getTenantId();
+        this.clientId = builder.getClientId();
+        this.clientSecret = builder.getClientSecret();
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(authorityEndpoint, tenantId, clientId, clientSecret);
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+            " authorityDNS='" + authorityEndpoint + "'" +
+            ", tenantId='" + tenantId + "'" +
+            ", clientId='" + clientId + "'" +
+            ", clientSecret='" + clientSecret + "'" +
+            "}";
+    }
+
+    public Date getLastAccessTokenExpirationDate(){
+        return tokenExpiresOnDate;
+    }
+
+    private IAuthenticationResult getAccessTokenByClientCredentialGrant()
+        throws MalformedURLException, ExecutionException, InterruptedException {
+
+        ConfidentialClientApplication app = ConfidentialClientApplication.builder(
+                this.clientId,
+                ClientCredentialFactory.createFromSecret(this.clientSecret))
+                .authority(String.format("%s/%s", authorityEndpoint, tenantId))
+                .build();
+
+        // With client credentials flows the scope is ALWAYS of the shape "resource/.default", as the
+        // application permissions need to be set statically (in the portal), and then granted by a tenant administrator
+        ClientCredentialParameters clientCredentialParam = ClientCredentialParameters.builder(
+                Collections.singleton(GRAPH_DEFAULT_SCOPE))
+                .build();
+
+        CompletableFuture<IAuthenticationResult> future = app.acquireToken(clientCredentialParam);
+        return future.get();
+    }
+
+    private String getAccessToken() {
+        Date now = new Date();

Review comment:
       ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");

Review comment:
       Can you add a constant for the "5 mins"?
   ```suggestion
           long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, DEFAULT_REFRESH_DELAY);
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']

Review comment:
       ```suggestion
       // default: upn (or userPrincipalName). possible choices ['upn', 'email']
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.net.MalformedURLException;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import com.microsoft.aad.msal4j.ClientCredentialFactory;
+import com.microsoft.aad.msal4j.ClientCredentialParameters;
+import com.microsoft.aad.msal4j.ConfidentialClientApplication;
+import com.microsoft.aad.msal4j.IAuthenticationResult;
+import com.microsoft.graph.authentication.IAuthenticationProvider;
+import com.microsoft.graph.http.IHttpRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientCredentialAuthProvider implements IAuthenticationProvider {
+
+    final private String authorityEndpoint; // { "Global", "https://login.microsoftonline.com/", "UsGovernment", "https://login.microsoftonline.us/" }
+    final private String tenantId;
+    final private String clientId;
+    final private String clientSecret;
+    private Date tokenExpiresOnDate;
+    private String lastAcessToken;
+    private final static String GRAPH_DEFAULT_SCOPE = "https://graph.microsoft.com/.default";
+    private final static Logger logger = LoggerFactory.getLogger(ClientCredentialAuthProvider.class);
+
+    private ClientCredentialAuthProvider(final Builder builder){
+        this.authorityEndpoint = builder.getAuthorityEndpoint();
+        this.tenantId = builder.getTenantId();
+        this.clientId = builder.getClientId();
+        this.clientSecret = builder.getClientSecret();
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(authorityEndpoint, tenantId, clientId, clientSecret);
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+            " authorityDNS='" + authorityEndpoint + "'" +
+            ", tenantId='" + tenantId + "'" +
+            ", clientId='" + clientId + "'" +
+            ", clientSecret='" + clientSecret + "'" +
+            "}";
+    }
+
+    public Date getLastAccessTokenExpirationDate(){
+        return tokenExpiresOnDate;
+    }
+
+    private IAuthenticationResult getAccessTokenByClientCredentialGrant()
+        throws MalformedURLException, ExecutionException, InterruptedException {
+
+        ConfidentialClientApplication app = ConfidentialClientApplication.builder(
+                this.clientId,
+                ClientCredentialFactory.createFromSecret(this.clientSecret))
+                .authority(String.format("%s/%s", authorityEndpoint, tenantId))
+                .build();
+
+        // With client credentials flows the scope is ALWAYS of the shape "resource/.default", as the
+        // application permissions need to be set statically (in the portal), and then granted by a tenant administrator
+        ClientCredentialParameters clientCredentialParam = ClientCredentialParameters.builder(
+                Collections.singleton(GRAPH_DEFAULT_SCOPE))
+                .build();
+
+        CompletableFuture<IAuthenticationResult> future = app.acquireToken(clientCredentialParam);
+        return future.get();
+    }
+
+    private String getAccessToken() {
+        Date now = new Date();
+        if ((lastAcessToken != null) && (tokenExpiresOnDate != null) && (tokenExpiresOnDate.getTime() -now.getTime() > 60000)) {
+            return lastAcessToken;
+        } else {
+            try {
+                IAuthenticationResult result = getAccessTokenByClientCredentialGrant();
+                tokenExpiresOnDate = result.expiresOnDate(); // store this for token expiration checking
+                lastAcessToken = result.accessToken();
+            } catch(Exception ex) {
+                logger.error("Failed to get access token", ex);
+            }

Review comment:
       ```suggestion
               } catch (final Exception e) {
                   logger.error("Failed to get access token due to {}", e.getMessage(), e);
               }
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));

Review comment:
       ```suggestion
                       String.format("%s is a required field for %s", APP_REG_CLIENT_ID_PROPERTY, getClass().getSimpleName()));
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ImmutableAzureGraphUserGroup.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import static java.util.stream.Collectors.toMap;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+
+public class ImmutableAzureGraphUserGroup {
+    private final Set<User> users;
+    private final Map<String, User> usersByObjectId;
+    private final Map<String, User> usersByPrincipalName;
+    private final Map<String, UserAndGroups> usersAndGroupsByUserObjectId;
+
+    private final Set<Group> groups;
+    private final Map<String, Group> groupsByObjectId;
+    private final Map<String, Group> groupsByDisplayName;
+
+    private ImmutableAzureGraphUserGroup(
+        final Set<User> users,
+        final Map<String, User> usersByObjectId,
+        final Map<String, User> usersByPrincipalName,
+        final Map<String, UserAndGroups> usersAndGroupsByUserObjectId,
+        final Set<Group> groups,
+        final Map<String, Group> groupsByObjectId,
+        final Map<String, Group> groupsByDisplayName) {
+
+        this.users = Collections.unmodifiableSet(users);
+        this.usersByObjectId = Collections.unmodifiableMap(usersByObjectId);
+        this.usersByPrincipalName = Collections.unmodifiableMap(usersByPrincipalName);
+        this.usersAndGroupsByUserObjectId = Collections.unmodifiableMap(usersAndGroupsByUserObjectId);
+
+        this.groups = Collections.unmodifiableSet(groups);
+        this.groupsByObjectId = Collections.unmodifiableMap(groupsByObjectId);
+        this.groupsByDisplayName = Collections.unmodifiableMap(groupsByDisplayName);
+    }
+
+    public Set<User> getUsers() {
+        return users;
+    }
+
+    public User getUser(final String objectId) {
+        return usersByObjectId.get(objectId);
+    }
+
+    public User getUserByPrincipalName(final String principalName) {
+        return usersByPrincipalName.get(principalName);
+    }
+    public UserAndGroups getUserAndGroupsByUserObjectId(final String objectId) {
+        return usersAndGroupsByUserObjectId.get(objectId);
+    }
+    public UserAndGroups getUserAndGroups(final String principalName) {
+        User user = getUserByPrincipalName(principalName);

Review comment:
       ```suggestion
           final User user = getUserByPrincipalName(principalName);
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }

Review comment:
       ```suggestion
           } catch (final ClientException e) {
               throw new AuthorizerCreationException(String.format("Failed to create a GraphServiceClient due to %s", e.getMessage()), e);
           }
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {

Review comment:
       ```suggestion
           if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring) && StringUtils.isBlank(groupFilterList)) {
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.

Review comment:
       ```suggestion
            // first, load list of group name if there is any prefix, suffix, substring filter defined, paging through groups.
            // then, add additonal group list if there is group list inclusion defined.
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");

Review comment:
       For consistency with the other errors, can we change this to:
   ```suggestion
               throw new AuthorizerCreationException(String.format("At least one group filter (%s, %s, %s) should be specified for %s", 
                   GROUP_FILTER_PREFIX_PROPERTY, GROUP_FILTER_SUFFIX_PROPERTY, GROUP_FILTER_LIST_PROPERTY, getClass().getSimpleName()));
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));

Review comment:
       ```suggestion
                       String.format("%s is a required field for %s", APP_REG_CLIENT_SECRET_PROPERTY, getClass().getSimpleName()));
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }

Review comment:
       ```suggestion
           } catch (final IOException | ClientException e) {
               throw new AuthorizerCreationException(String.format("Failed to load UserGroup due to %s", e.getMessage()), e);
           }
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = getGroupsWith(groupFilterList, prefix, suffix, substring, pageSize);
+        refreshUserGroupData(groupDisplayNames);
+        stopWatch.stop();
+        if (logger.isDebugEnabled()) {
+            logger.debug("Refreshed {} user groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+    }
+
+    private Set<String> getGroupsWith(String groupFilterList, String prefix, String suffix, String substring, int pageSize) {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = new HashSet<>();
+
+        if (!StringUtils.isBlank(prefix) || !StringUtils.isBlank(suffix) || !StringUtils.isBlank(substring)) {
+            groupDisplayNames.addAll(queryGroupsWith(prefix, suffix, substring, pageSize));
+        }
+
+        if (!StringUtils.isBlank(groupFilterList)) {
+            groupDisplayNames.addAll(
+                Arrays.stream(groupFilterList.split(","))
+                    .map(String::trim)

Review comment:
       ```suggestion
                       .map(String::trim)
                       .filter(s -> !s.isEmpty())
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);

Review comment:
       ```suggestion
                   logger.error("Error refreshing user groups due to {}", t.getMessage(), t);
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {

Review comment:
       ```suggestion
       private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
           if (logger.isDebugEnabled()) {
               logger.debug("Refreshing user groups");
           }
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = getGroupsWith(groupFilterList, prefix, suffix, substring, pageSize);
+        refreshUserGroupData(groupDisplayNames);
+        stopWatch.stop();
+        if (logger.isDebugEnabled()) {
+            logger.debug("Refreshed {} user groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+    }
+
+    private Set<String> getGroupsWith(String groupFilterList, String prefix, String suffix, String substring, int pageSize) {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = new HashSet<>();
+
+        if (!StringUtils.isBlank(prefix) || !StringUtils.isBlank(suffix) || !StringUtils.isBlank(substring)) {
+            groupDisplayNames.addAll(queryGroupsWith(prefix, suffix, substring, pageSize));
+        }
+
+        if (!StringUtils.isBlank(groupFilterList)) {
+            groupDisplayNames.addAll(
+                Arrays.stream(groupFilterList.split(","))
+                    .map(String::trim)
+                    .collect(Collectors.toList())
+            );
+        }
+
+        stopWatch.stop();
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Fetched {} groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+
+        return Collections.unmodifiableSet(groupDisplayNames);
+    }
+
+    /**
+     * Get a set of group display names after filtering prefix, suffix, and substring
+     * @param prefix prefix filter string matching against displayName of group directory objects
+     * @param suffix suffix fitler string matching against displayName of group directory objects
+     * @param substring string matching against displayName of group directory objects
+     * @param pageSize page size to make graph rest calls in pagination
+     * @return set of group display names
+     */
+    private Set<String> queryGroupsWith(String prefix, String suffix, String substring, int pageSize) {
+        final Set<String> groups = new HashSet<>();
+        boolean filterEvaluation = false;

Review comment:
       ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");

Review comment:
       Can we move this into the `refreshUserGroup()`?
   ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = getGroupsWith(groupFilterList, prefix, suffix, substring, pageSize);
+        refreshUserGroupData(groupDisplayNames);
+        stopWatch.stop();
+        if (logger.isDebugEnabled()) {
+            logger.debug("Refreshed {} user groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+    }
+
+    private Set<String> getGroupsWith(String groupFilterList, String prefix, String suffix, String substring, int pageSize) {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = new HashSet<>();
+
+        if (!StringUtils.isBlank(prefix) || !StringUtils.isBlank(suffix) || !StringUtils.isBlank(substring)) {
+            groupDisplayNames.addAll(queryGroupsWith(prefix, suffix, substring, pageSize));
+        }
+
+        if (!StringUtils.isBlank(groupFilterList)) {
+            groupDisplayNames.addAll(
+                Arrays.stream(groupFilterList.split(","))
+                    .map(String::trim)
+                    .collect(Collectors.toList())
+            );
+        }
+
+        stopWatch.stop();
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Fetched {} groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }

Review comment:
       ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = getGroupsWith(groupFilterList, prefix, suffix, substring, pageSize);
+        refreshUserGroupData(groupDisplayNames);
+        stopWatch.stop();
+        if (logger.isDebugEnabled()) {
+            logger.debug("Refreshed {} user groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+    }
+
+    private Set<String> getGroupsWith(String groupFilterList, String prefix, String suffix, String substring, int pageSize) {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = new HashSet<>();
+
+        if (!StringUtils.isBlank(prefix) || !StringUtils.isBlank(suffix) || !StringUtils.isBlank(substring)) {
+            groupDisplayNames.addAll(queryGroupsWith(prefix, suffix, substring, pageSize));
+        }
+
+        if (!StringUtils.isBlank(groupFilterList)) {
+            groupDisplayNames.addAll(
+                Arrays.stream(groupFilterList.split(","))
+                    .map(String::trim)
+                    .collect(Collectors.toList())
+            );
+        }
+
+        stopWatch.stop();
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Fetched {} groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+
+        return Collections.unmodifiableSet(groupDisplayNames);
+    }
+
+    /**
+     * Get a set of group display names after filtering prefix, suffix, and substring
+     * @param prefix prefix filter string matching against displayName of group directory objects
+     * @param suffix suffix fitler string matching against displayName of group directory objects
+     * @param substring string matching against displayName of group directory objects
+     * @param pageSize page size to make graph rest calls in pagination
+     * @return set of group display names
+     */
+    private Set<String> queryGroupsWith(String prefix, String suffix, String substring, int pageSize) {
+        final Set<String> groups = new HashSet<>();
+        boolean filterEvaluation = false;
+        IGroupCollectionRequest gRequest;
+        IGroupCollectionPage filterResults;
+        if (prefix != null && !prefix.isEmpty()) {
+            // build a $filter query option and create a graph request if prefix is given
+            final List<Option> requestOptions = new LinkedList<Option>();
+            requestOptions.add(new QueryOption("$filter", String.format("startswith(displayName, '%s')", prefix)));
+            gRequest = graphClient.groups().buildRequest(requestOptions).select("displayName");
+        } else {
+            // default group graph request
+            gRequest = graphClient.groups().buildRequest().select("displayName");
+        }
+        if (pageSize > 0) {
+            gRequest = gRequest.top(pageSize);
+        }
+        filterResults = gRequest.get();
+
+        List<com.microsoft.graph.models.extensions.Group> currentPage = filterResults.getCurrentPage();
+        while (currentPage != null) {
+            for (com.microsoft.graph.models.extensions.Group grp : currentPage) {
+                filterEvaluation = true;

Review comment:
       ```suggestion
                   boolean filterEvaluation = true;
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = getGroupsWith(groupFilterList, prefix, suffix, substring, pageSize);
+        refreshUserGroupData(groupDisplayNames);

Review comment:
       ```suggestion
           refreshUserGroupData(groupDisplayNames, pageSize);
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = getGroupsWith(groupFilterList, prefix, suffix, substring, pageSize);
+        refreshUserGroupData(groupDisplayNames);
+        stopWatch.stop();
+        if (logger.isDebugEnabled()) {
+            logger.debug("Refreshed {} user groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+    }
+
+    private Set<String> getGroupsWith(String groupFilterList, String prefix, String suffix, String substring, int pageSize) {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = new HashSet<>();
+
+        if (!StringUtils.isBlank(prefix) || !StringUtils.isBlank(suffix) || !StringUtils.isBlank(substring)) {
+            groupDisplayNames.addAll(queryGroupsWith(prefix, suffix, substring, pageSize));
+        }
+
+        if (!StringUtils.isBlank(groupFilterList)) {
+            groupDisplayNames.addAll(
+                Arrays.stream(groupFilterList.split(","))
+                    .map(String::trim)
+                    .collect(Collectors.toList())
+            );
+        }
+
+        stopWatch.stop();
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Fetched {} groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+
+        return Collections.unmodifiableSet(groupDisplayNames);
+    }
+
+    /**
+     * Get a set of group display names after filtering prefix, suffix, and substring
+     * @param prefix prefix filter string matching against displayName of group directory objects
+     * @param suffix suffix fitler string matching against displayName of group directory objects
+     * @param substring string matching against displayName of group directory objects
+     * @param pageSize page size to make graph rest calls in pagination
+     * @return set of group display names
+     */
+    private Set<String> queryGroupsWith(String prefix, String suffix, String substring, int pageSize) {
+        final Set<String> groups = new HashSet<>();
+        boolean filterEvaluation = false;
+        IGroupCollectionRequest gRequest;
+        IGroupCollectionPage filterResults;
+        if (prefix != null && !prefix.isEmpty()) {
+            // build a $filter query option and create a graph request if prefix is given
+            final List<Option> requestOptions = new LinkedList<Option>();
+            requestOptions.add(new QueryOption("$filter", String.format("startswith(displayName, '%s')", prefix)));

Review comment:
       ```suggestion
               final List<Option> requestOptions = Arrays.asList(new QueryOption("$filter", String.format("startswith(displayName, '%s')", prefix)));
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = getGroupsWith(groupFilterList, prefix, suffix, substring, pageSize);
+        refreshUserGroupData(groupDisplayNames);
+        stopWatch.stop();
+        if (logger.isDebugEnabled()) {
+            logger.debug("Refreshed {} user groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+    }
+
+    private Set<String> getGroupsWith(String groupFilterList, String prefix, String suffix, String substring, int pageSize) {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = new HashSet<>();
+
+        if (!StringUtils.isBlank(prefix) || !StringUtils.isBlank(suffix) || !StringUtils.isBlank(substring)) {
+            groupDisplayNames.addAll(queryGroupsWith(prefix, suffix, substring, pageSize));
+        }
+
+        if (!StringUtils.isBlank(groupFilterList)) {
+            groupDisplayNames.addAll(
+                Arrays.stream(groupFilterList.split(","))
+                    .map(String::trim)
+                    .collect(Collectors.toList())
+            );
+        }
+
+        stopWatch.stop();
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Fetched {} groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+
+        return Collections.unmodifiableSet(groupDisplayNames);
+    }
+
+    /**
+     * Get a set of group display names after filtering prefix, suffix, and substring
+     * @param prefix prefix filter string matching against displayName of group directory objects
+     * @param suffix suffix fitler string matching against displayName of group directory objects
+     * @param substring string matching against displayName of group directory objects
+     * @param pageSize page size to make graph rest calls in pagination
+     * @return set of group display names
+     */
+    private Set<String> queryGroupsWith(String prefix, String suffix, String substring, int pageSize) {
+        final Set<String> groups = new HashSet<>();
+        boolean filterEvaluation = false;
+        IGroupCollectionRequest gRequest;
+        IGroupCollectionPage filterResults;
+        if (prefix != null && !prefix.isEmpty()) {
+            // build a $filter query option and create a graph request if prefix is given
+            final List<Option> requestOptions = new LinkedList<Option>();
+            requestOptions.add(new QueryOption("$filter", String.format("startswith(displayName, '%s')", prefix)));
+            gRequest = graphClient.groups().buildRequest(requestOptions).select("displayName");
+        } else {
+            // default group graph request
+            gRequest = graphClient.groups().buildRequest().select("displayName");
+        }
+        if (pageSize > 0) {
+            gRequest = gRequest.top(pageSize);
+        }
+        filterResults = gRequest.get();
+
+        List<com.microsoft.graph.models.extensions.Group> currentPage = filterResults.getCurrentPage();
+        while (currentPage != null) {
+            for (com.microsoft.graph.models.extensions.Group grp : currentPage) {
+                filterEvaluation = true;
+                if (!StringUtils.isEmpty(suffix) && !grp.displayName.endsWith(suffix)) {
+                    filterEvaluation = false;
+                }
+                if (!StringUtils.isEmpty(substring) && !grp.displayName.contains(substring)) {
+                    filterEvaluation = false;
+                }
+                if (filterEvaluation) {
+                    groups.add(grp.displayName);
+                }
+            }
+            IGroupCollectionRequestBuilder gBuilder = filterResults.getNextPage();
+            if (gBuilder != null) {
+                filterResults = gBuilder.buildRequest().get();
+                currentPage = filterResults.getCurrentPage();
+            } else {
+                currentPage = null;
+            }
+        }
+
+        return Collections.unmodifiableSet(groups);
+    }
+
+    /**
+     * Get member users of the given group name
+     * @param groupName group name to search for member users
+     * @return UserGroupQueryResult
+     */
+    private UserGroupQueryResult getUsersFrom(String groupName) throws IOException, ClientException {
+        final Set<User> users = new HashSet<>();
+
+        final List<Option> requestOptions = Arrays.asList(new QueryOption("$filter", String.format("displayName eq '%s'", groupName)));
+        final IGroupCollectionPage results = graphClient.groups().buildRequest(requestOptions).get();
+        final List<com.microsoft.graph.models.extensions.Group> currentPage = results.getCurrentPage();
+
+        if (currentPage != null && currentPage.size() > 0) {
+            final com.microsoft.graph.models.extensions.Group graphGroup = results.getCurrentPage().get(0);
+            final Group.Builder groupBuilder =
+                new Group.Builder()
+                    .identifier(graphGroup.id)
+                    .name(graphGroup.displayName);
+
+            IDirectoryObjectCollectionWithReferencesRequest uRequest =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName");
+
+            if (pageSize > 0) {
+                uRequest = uRequest.top(pageSize);
+            }
+            IDirectoryObjectCollectionWithReferencesPage userpage =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName").get();
+
+            while (userpage.getCurrentPage() != null) {
+                for (DirectoryObject userDO : userpage.getCurrentPage()) {
+                    JsonObject jsonUser = userDO.getRawObject();
+                    final String idUser = jsonUser.get("id").getAsString();
+                    // upn is default fallback claim for userName
+                    // upn claim maps to 'mail' property in Azure graph rest-api.
+                    final String userName;
+                    if (claimForUserName.equals("email")) {
+                        // authentication token contains email field, while graph api returns mail property
+                        if (!jsonUser.get("mail").isJsonNull()) {
+                            userName = jsonUser.get("mail").getAsString();
+                        } else {
+                            userName = jsonUser.get("userPrincipalName").getAsString();
+                        }
+                    } else {
+                        userName = jsonUser.get("userPrincipalName").getAsString();
+                    }
+                    final User user = new User.Builder().identifier(idUser).identity(userName).build();
+                    users.add(user);
+                    groupBuilder.addUser(idUser);
+                }
+                IDirectoryObjectCollectionWithReferencesRequestBuilder nextPageRequest = userpage.getNextPage();
+
+                if (nextPageRequest != null) {
+                    userpage = nextPageRequest.buildRequest().get();
+                } else {
+                    break;
+                }
+            }
+            final Group group = groupBuilder.build();
+            return new UserGroupQueryResult(group, users);
+        } else if (logger.isDebugEnabled()) {
+            logger.debug("Group collection page for {} was null or empty", groupName);
+        }
+        return null;
+    }
+
+    /**
+     * refresh the user & group data for UserGroupProvider plugin service
+     * @param groupDisplayNames a list of group display names
+     */
+    private void refreshUserGroupData(Set<String> groupDisplayNames) throws IOException, ClientException {
+        Objects.requireNonNull(groupDisplayNames);
+
+        final long startTime = System.currentTimeMillis();
+        final Set<User> _users = new HashSet<>();
+        final Set<Group> _groups = new HashSet<>();

Review comment:
       ```suggestion
           final Set<User> users = new HashSet<>();
           final Set<Group> groups = new HashSet<>();
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.net.MalformedURLException;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import com.microsoft.aad.msal4j.ClientCredentialFactory;
+import com.microsoft.aad.msal4j.ClientCredentialParameters;
+import com.microsoft.aad.msal4j.ConfidentialClientApplication;
+import com.microsoft.aad.msal4j.IAuthenticationResult;
+import com.microsoft.graph.authentication.IAuthenticationProvider;
+import com.microsoft.graph.http.IHttpRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientCredentialAuthProvider implements IAuthenticationProvider {
+
+    final private String authorityEndpoint; // { "Global", "https://login.microsoftonline.com/", "UsGovernment", "https://login.microsoftonline.us/" }
+    final private String tenantId;
+    final private String clientId;
+    final private String clientSecret;
+    private Date tokenExpiresOnDate;
+    private String lastAcessToken;
+    private final static String GRAPH_DEFAULT_SCOPE = "https://graph.microsoft.com/.default";
+    private final static Logger logger = LoggerFactory.getLogger(ClientCredentialAuthProvider.class);
+
+    private ClientCredentialAuthProvider(final Builder builder){
+        this.authorityEndpoint = builder.getAuthorityEndpoint();
+        this.tenantId = builder.getTenantId();
+        this.clientId = builder.getClientId();
+        this.clientSecret = builder.getClientSecret();
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(authorityEndpoint, tenantId, clientId, clientSecret);
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+            " authorityDNS='" + authorityEndpoint + "'" +
+            ", tenantId='" + tenantId + "'" +
+            ", clientId='" + clientId + "'" +
+            ", clientSecret='" + clientSecret + "'" +
+            "}";
+    }
+
+    public Date getLastAccessTokenExpirationDate(){
+        return tokenExpiresOnDate;
+    }
+
+    private IAuthenticationResult getAccessTokenByClientCredentialGrant()
+        throws MalformedURLException, ExecutionException, InterruptedException {
+
+        ConfidentialClientApplication app = ConfidentialClientApplication.builder(
+                this.clientId,
+                ClientCredentialFactory.createFromSecret(this.clientSecret))
+                .authority(String.format("%s/%s", authorityEndpoint, tenantId))
+                .build();
+
+        // With client credentials flows the scope is ALWAYS of the shape "resource/.default", as the
+        // application permissions need to be set statically (in the portal), and then granted by a tenant administrator
+        ClientCredentialParameters clientCredentialParam = ClientCredentialParameters.builder(
+                Collections.singleton(GRAPH_DEFAULT_SCOPE))
+                .build();
+
+        CompletableFuture<IAuthenticationResult> future = app.acquireToken(clientCredentialParam);
+        return future.get();
+    }
+
+    private String getAccessToken() {
+        Date now = new Date();
+        if ((lastAcessToken != null) && (tokenExpiresOnDate != null) && (tokenExpiresOnDate.getTime() -now.getTime() > 60000)) {
+            return lastAcessToken;
+        } else {
+            try {
+                IAuthenticationResult result = getAccessTokenByClientCredentialGrant();
+                tokenExpiresOnDate = result.expiresOnDate(); // store this for token expiration checking
+                lastAcessToken = result.accessToken();
+            } catch(Exception ex) {
+                logger.error("Failed to get access token", ex);
+            }
+            return lastAcessToken;
+        }
+
+
+    }
+
+    @Override
+    public void authenticateRequest(IHttpRequest request) {
+        String accessToken = getAccessToken();
+        if (accessToken != null) {
+            request.addHeader("Authorization", "Bearer " + accessToken);
+        }
+
+    }
+
+    public static class Builder {
+
+        private String authorityEndpoint ="";
+        private String tenantId = "";
+        private String clientId = "";
+        private String clientSecret = "";
+
+        public Builder() {
+
+        }
+

Review comment:
       ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;

Review comment:
       As far as I can tell this doesn't need to be shared data since the methods called from `onConfigured()` already have it as a parameter.
   ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = getGroupsWith(groupFilterList, prefix, suffix, substring, pageSize);
+        refreshUserGroupData(groupDisplayNames);
+        stopWatch.stop();
+        if (logger.isDebugEnabled()) {
+            logger.debug("Refreshed {} user groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+    }
+
+    private Set<String> getGroupsWith(String groupFilterList, String prefix, String suffix, String substring, int pageSize) {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = new HashSet<>();
+
+        if (!StringUtils.isBlank(prefix) || !StringUtils.isBlank(suffix) || !StringUtils.isBlank(substring)) {
+            groupDisplayNames.addAll(queryGroupsWith(prefix, suffix, substring, pageSize));
+        }
+
+        if (!StringUtils.isBlank(groupFilterList)) {
+            groupDisplayNames.addAll(
+                Arrays.stream(groupFilterList.split(","))
+                    .map(String::trim)
+                    .collect(Collectors.toList())
+            );
+        }
+
+        stopWatch.stop();
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Fetched {} groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+
+        return Collections.unmodifiableSet(groupDisplayNames);
+    }
+
+    /**
+     * Get a set of group display names after filtering prefix, suffix, and substring
+     * @param prefix prefix filter string matching against displayName of group directory objects
+     * @param suffix suffix fitler string matching against displayName of group directory objects
+     * @param substring string matching against displayName of group directory objects
+     * @param pageSize page size to make graph rest calls in pagination
+     * @return set of group display names
+     */
+    private Set<String> queryGroupsWith(String prefix, String suffix, String substring, int pageSize) {
+        final Set<String> groups = new HashSet<>();
+        boolean filterEvaluation = false;
+        IGroupCollectionRequest gRequest;
+        IGroupCollectionPage filterResults;
+        if (prefix != null && !prefix.isEmpty()) {
+            // build a $filter query option and create a graph request if prefix is given
+            final List<Option> requestOptions = new LinkedList<Option>();
+            requestOptions.add(new QueryOption("$filter", String.format("startswith(displayName, '%s')", prefix)));
+            gRequest = graphClient.groups().buildRequest(requestOptions).select("displayName");
+        } else {
+            // default group graph request
+            gRequest = graphClient.groups().buildRequest().select("displayName");
+        }
+        if (pageSize > 0) {
+            gRequest = gRequest.top(pageSize);
+        }
+        filterResults = gRequest.get();
+
+        List<com.microsoft.graph.models.extensions.Group> currentPage = filterResults.getCurrentPage();
+        while (currentPage != null) {
+            for (com.microsoft.graph.models.extensions.Group grp : currentPage) {
+                filterEvaluation = true;
+                if (!StringUtils.isEmpty(suffix) && !grp.displayName.endsWith(suffix)) {
+                    filterEvaluation = false;
+                }
+                if (!StringUtils.isEmpty(substring) && !grp.displayName.contains(substring)) {
+                    filterEvaluation = false;
+                }
+                if (filterEvaluation) {
+                    groups.add(grp.displayName);
+                }
+            }
+            IGroupCollectionRequestBuilder gBuilder = filterResults.getNextPage();
+            if (gBuilder != null) {
+                filterResults = gBuilder.buildRequest().get();
+                currentPage = filterResults.getCurrentPage();
+            } else {
+                currentPage = null;
+            }
+        }
+
+        return Collections.unmodifiableSet(groups);
+    }
+
+    /**
+     * Get member users of the given group name
+     * @param groupName group name to search for member users
+     * @return UserGroupQueryResult
+     */
+    private UserGroupQueryResult getUsersFrom(String groupName) throws IOException, ClientException {
+        final Set<User> users = new HashSet<>();
+
+        final List<Option> requestOptions = Arrays.asList(new QueryOption("$filter", String.format("displayName eq '%s'", groupName)));
+        final IGroupCollectionPage results = graphClient.groups().buildRequest(requestOptions).get();
+        final List<com.microsoft.graph.models.extensions.Group> currentPage = results.getCurrentPage();
+
+        if (currentPage != null && currentPage.size() > 0) {
+            final com.microsoft.graph.models.extensions.Group graphGroup = results.getCurrentPage().get(0);
+            final Group.Builder groupBuilder =
+                new Group.Builder()
+                    .identifier(graphGroup.id)
+                    .name(graphGroup.displayName);
+
+            IDirectoryObjectCollectionWithReferencesRequest uRequest =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName");
+
+            if (pageSize > 0) {
+                uRequest = uRequest.top(pageSize);
+            }
+            IDirectoryObjectCollectionWithReferencesPage userpage =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName").get();
+
+            while (userpage.getCurrentPage() != null) {
+                for (DirectoryObject userDO : userpage.getCurrentPage()) {
+                    JsonObject jsonUser = userDO.getRawObject();
+                    final String idUser = jsonUser.get("id").getAsString();
+                    // upn is default fallback claim for userName
+                    // upn claim maps to 'mail' property in Azure graph rest-api.
+                    final String userName;
+                    if (claimForUserName.equals("email")) {
+                        // authentication token contains email field, while graph api returns mail property
+                        if (!jsonUser.get("mail").isJsonNull()) {
+                            userName = jsonUser.get("mail").getAsString();
+                        } else {
+                            userName = jsonUser.get("userPrincipalName").getAsString();
+                        }
+                    } else {
+                        userName = jsonUser.get("userPrincipalName").getAsString();
+                    }
+                    final User user = new User.Builder().identifier(idUser).identity(userName).build();
+                    users.add(user);
+                    groupBuilder.addUser(idUser);
+                }
+                IDirectoryObjectCollectionWithReferencesRequestBuilder nextPageRequest = userpage.getNextPage();
+
+                if (nextPageRequest != null) {
+                    userpage = nextPageRequest.buildRequest().get();
+                } else {
+                    break;
+                }
+            }
+            final Group group = groupBuilder.build();
+            return new UserGroupQueryResult(group, users);
+        } else if (logger.isDebugEnabled()) {
+            logger.debug("Group collection page for {} was null or empty", groupName);
+        }
+        return null;
+    }
+
+    /**
+     * refresh the user & group data for UserGroupProvider plugin service
+     * @param groupDisplayNames a list of group display names
+     */
+    private void refreshUserGroupData(Set<String> groupDisplayNames) throws IOException, ClientException {

Review comment:
       ```suggestion
       private void refreshUserGroupData(Set<String> groupDisplayNames, int pageSize) throws IOException, ClientException {
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = getGroupsWith(groupFilterList, prefix, suffix, substring, pageSize);
+        refreshUserGroupData(groupDisplayNames);
+        stopWatch.stop();
+        if (logger.isDebugEnabled()) {
+            logger.debug("Refreshed {} user groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+    }
+
+    private Set<String> getGroupsWith(String groupFilterList, String prefix, String suffix, String substring, int pageSize) {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = new HashSet<>();
+
+        if (!StringUtils.isBlank(prefix) || !StringUtils.isBlank(suffix) || !StringUtils.isBlank(substring)) {
+            groupDisplayNames.addAll(queryGroupsWith(prefix, suffix, substring, pageSize));
+        }
+
+        if (!StringUtils.isBlank(groupFilterList)) {
+            groupDisplayNames.addAll(
+                Arrays.stream(groupFilterList.split(","))
+                    .map(String::trim)
+                    .collect(Collectors.toList())
+            );
+        }
+
+        stopWatch.stop();
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Fetched {} groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+
+        return Collections.unmodifiableSet(groupDisplayNames);
+    }
+
+    /**
+     * Get a set of group display names after filtering prefix, suffix, and substring
+     * @param prefix prefix filter string matching against displayName of group directory objects
+     * @param suffix suffix fitler string matching against displayName of group directory objects
+     * @param substring string matching against displayName of group directory objects
+     * @param pageSize page size to make graph rest calls in pagination
+     * @return set of group display names
+     */
+    private Set<String> queryGroupsWith(String prefix, String suffix, String substring, int pageSize) {
+        final Set<String> groups = new HashSet<>();
+        boolean filterEvaluation = false;
+        IGroupCollectionRequest gRequest;
+        IGroupCollectionPage filterResults;
+        if (prefix != null && !prefix.isEmpty()) {
+            // build a $filter query option and create a graph request if prefix is given
+            final List<Option> requestOptions = new LinkedList<Option>();
+            requestOptions.add(new QueryOption("$filter", String.format("startswith(displayName, '%s')", prefix)));
+            gRequest = graphClient.groups().buildRequest(requestOptions).select("displayName");
+        } else {
+            // default group graph request
+            gRequest = graphClient.groups().buildRequest().select("displayName");
+        }
+        if (pageSize > 0) {
+            gRequest = gRequest.top(pageSize);
+        }
+        filterResults = gRequest.get();
+
+        List<com.microsoft.graph.models.extensions.Group> currentPage = filterResults.getCurrentPage();
+        while (currentPage != null) {
+            for (com.microsoft.graph.models.extensions.Group grp : currentPage) {
+                filterEvaluation = true;
+                if (!StringUtils.isEmpty(suffix) && !grp.displayName.endsWith(suffix)) {
+                    filterEvaluation = false;
+                }
+                if (!StringUtils.isEmpty(substring) && !grp.displayName.contains(substring)) {
+                    filterEvaluation = false;
+                }
+                if (filterEvaluation) {
+                    groups.add(grp.displayName);
+                }
+            }
+            IGroupCollectionRequestBuilder gBuilder = filterResults.getNextPage();
+            if (gBuilder != null) {
+                filterResults = gBuilder.buildRequest().get();
+                currentPage = filterResults.getCurrentPage();
+            } else {
+                currentPage = null;
+            }
+        }
+
+        return Collections.unmodifiableSet(groups);
+    }
+
+    /**
+     * Get member users of the given group name
+     * @param groupName group name to search for member users
+     * @return UserGroupQueryResult
+     */
+    private UserGroupQueryResult getUsersFrom(String groupName) throws IOException, ClientException {
+        final Set<User> users = new HashSet<>();
+
+        final List<Option> requestOptions = Arrays.asList(new QueryOption("$filter", String.format("displayName eq '%s'", groupName)));
+        final IGroupCollectionPage results = graphClient.groups().buildRequest(requestOptions).get();
+        final List<com.microsoft.graph.models.extensions.Group> currentPage = results.getCurrentPage();
+
+        if (currentPage != null && currentPage.size() > 0) {
+            final com.microsoft.graph.models.extensions.Group graphGroup = results.getCurrentPage().get(0);
+            final Group.Builder groupBuilder =
+                new Group.Builder()
+                    .identifier(graphGroup.id)
+                    .name(graphGroup.displayName);
+
+            IDirectoryObjectCollectionWithReferencesRequest uRequest =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName");
+
+            if (pageSize > 0) {
+                uRequest = uRequest.top(pageSize);
+            }
+            IDirectoryObjectCollectionWithReferencesPage userpage =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName").get();
+
+            while (userpage.getCurrentPage() != null) {
+                for (DirectoryObject userDO : userpage.getCurrentPage()) {
+                    JsonObject jsonUser = userDO.getRawObject();
+                    final String idUser = jsonUser.get("id").getAsString();
+                    // upn is default fallback claim for userName
+                    // upn claim maps to 'mail' property in Azure graph rest-api.
+                    final String userName;
+                    if (claimForUserName.equals("email")) {
+                        // authentication token contains email field, while graph api returns mail property
+                        if (!jsonUser.get("mail").isJsonNull()) {
+                            userName = jsonUser.get("mail").getAsString();
+                        } else {
+                            userName = jsonUser.get("userPrincipalName").getAsString();
+                        }
+                    } else {
+                        userName = jsonUser.get("userPrincipalName").getAsString();
+                    }
+                    final User user = new User.Builder().identifier(idUser).identity(userName).build();
+                    users.add(user);
+                    groupBuilder.addUser(idUser);
+                }
+                IDirectoryObjectCollectionWithReferencesRequestBuilder nextPageRequest = userpage.getNextPage();
+
+                if (nextPageRequest != null) {
+                    userpage = nextPageRequest.buildRequest().get();
+                } else {
+                    break;
+                }
+            }
+            final Group group = groupBuilder.build();
+            return new UserGroupQueryResult(group, users);
+        } else if (logger.isDebugEnabled()) {
+            logger.debug("Group collection page for {} was null or empty", groupName);
+        }
+        return null;
+    }
+
+    /**
+     * refresh the user & group data for UserGroupProvider plugin service
+     * @param groupDisplayNames a list of group display names
+     */
+    private void refreshUserGroupData(Set<String> groupDisplayNames) throws IOException, ClientException {
+        Objects.requireNonNull(groupDisplayNames);
+
+        final long startTime = System.currentTimeMillis();
+        final Set<User> _users = new HashSet<>();
+        final Set<Group> _groups = new HashSet<>();
+
+        for (String grpFilter : groupDisplayNames) {
+            if (logger.isDebugEnabled()) logger.debug("Getting users for group filter: {}", grpFilter);
+            UserGroupQueryResult queryResult = getUsersFrom(grpFilter);
+            if (queryResult != null) {
+                if (logger.isDebugEnabled()) {
+                    logger.debug("Found {} users for group: {}", queryResult.getUsers().size(), queryResult.getGroup().getName());
+                }
+                _groups.add(queryResult.getGroup());
+                _users.addAll(queryResult.getUsers());
+            } else {
+                if (logger.isDebugEnabled()) logger.debug("Query result was null");
+            }

Review comment:
       ```suggestion
               if (queryResult != null) {
                   groups.add(queryResult.getGroup());
                   users.addAll(queryResult.getUsers());
               }
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = getGroupsWith(groupFilterList, prefix, suffix, substring, pageSize);
+        refreshUserGroupData(groupDisplayNames);
+        stopWatch.stop();
+        if (logger.isDebugEnabled()) {
+            logger.debug("Refreshed {} user groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+    }
+
+    private Set<String> getGroupsWith(String groupFilterList, String prefix, String suffix, String substring, int pageSize) {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = new HashSet<>();
+
+        if (!StringUtils.isBlank(prefix) || !StringUtils.isBlank(suffix) || !StringUtils.isBlank(substring)) {
+            groupDisplayNames.addAll(queryGroupsWith(prefix, suffix, substring, pageSize));
+        }
+
+        if (!StringUtils.isBlank(groupFilterList)) {
+            groupDisplayNames.addAll(
+                Arrays.stream(groupFilterList.split(","))
+                    .map(String::trim)
+                    .collect(Collectors.toList())
+            );
+        }
+
+        stopWatch.stop();
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Fetched {} groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+
+        return Collections.unmodifiableSet(groupDisplayNames);
+    }
+
+    /**
+     * Get a set of group display names after filtering prefix, suffix, and substring
+     * @param prefix prefix filter string matching against displayName of group directory objects
+     * @param suffix suffix fitler string matching against displayName of group directory objects
+     * @param substring string matching against displayName of group directory objects
+     * @param pageSize page size to make graph rest calls in pagination
+     * @return set of group display names
+     */
+    private Set<String> queryGroupsWith(String prefix, String suffix, String substring, int pageSize) {
+        final Set<String> groups = new HashSet<>();
+        boolean filterEvaluation = false;
+        IGroupCollectionRequest gRequest;
+        IGroupCollectionPage filterResults;
+        if (prefix != null && !prefix.isEmpty()) {
+            // build a $filter query option and create a graph request if prefix is given
+            final List<Option> requestOptions = new LinkedList<Option>();
+            requestOptions.add(new QueryOption("$filter", String.format("startswith(displayName, '%s')", prefix)));
+            gRequest = graphClient.groups().buildRequest(requestOptions).select("displayName");
+        } else {
+            // default group graph request
+            gRequest = graphClient.groups().buildRequest().select("displayName");
+        }
+        if (pageSize > 0) {
+            gRequest = gRequest.top(pageSize);
+        }
+        filterResults = gRequest.get();
+
+        List<com.microsoft.graph.models.extensions.Group> currentPage = filterResults.getCurrentPage();
+        while (currentPage != null) {
+            for (com.microsoft.graph.models.extensions.Group grp : currentPage) {
+                filterEvaluation = true;
+                if (!StringUtils.isEmpty(suffix) && !grp.displayName.endsWith(suffix)) {
+                    filterEvaluation = false;
+                }
+                if (!StringUtils.isEmpty(substring) && !grp.displayName.contains(substring)) {
+                    filterEvaluation = false;
+                }
+                if (filterEvaluation) {
+                    groups.add(grp.displayName);
+                }
+            }
+            IGroupCollectionRequestBuilder gBuilder = filterResults.getNextPage();
+            if (gBuilder != null) {
+                filterResults = gBuilder.buildRequest().get();
+                currentPage = filterResults.getCurrentPage();
+            } else {
+                currentPage = null;
+            }
+        }
+
+        return Collections.unmodifiableSet(groups);
+    }
+
+    /**
+     * Get member users of the given group name
+     * @param groupName group name to search for member users
+     * @return UserGroupQueryResult
+     */
+    private UserGroupQueryResult getUsersFrom(String groupName) throws IOException, ClientException {

Review comment:
       ```suggestion
       private UserGroupQueryResult getUsersFrom(String groupName, int pageSize) throws IOException, ClientException {
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.net.MalformedURLException;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import com.microsoft.aad.msal4j.ClientCredentialFactory;
+import com.microsoft.aad.msal4j.ClientCredentialParameters;
+import com.microsoft.aad.msal4j.ConfidentialClientApplication;
+import com.microsoft.aad.msal4j.IAuthenticationResult;
+import com.microsoft.graph.authentication.IAuthenticationProvider;
+import com.microsoft.graph.http.IHttpRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ClientCredentialAuthProvider implements IAuthenticationProvider {
+
+    final private String authorityEndpoint; // { "Global", "https://login.microsoftonline.com/", "UsGovernment", "https://login.microsoftonline.us/" }
+    final private String tenantId;
+    final private String clientId;
+    final private String clientSecret;
+    private Date tokenExpiresOnDate;
+    private String lastAcessToken;
+    private final static String GRAPH_DEFAULT_SCOPE = "https://graph.microsoft.com/.default";
+    private final static Logger logger = LoggerFactory.getLogger(ClientCredentialAuthProvider.class);
+
+    private ClientCredentialAuthProvider(final Builder builder){
+        this.authorityEndpoint = builder.getAuthorityEndpoint();
+        this.tenantId = builder.getTenantId();
+        this.clientId = builder.getClientId();
+        this.clientSecret = builder.getClientSecret();
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(authorityEndpoint, tenantId, clientId, clientSecret);
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+            " authorityDNS='" + authorityEndpoint + "'" +
+            ", tenantId='" + tenantId + "'" +
+            ", clientId='" + clientId + "'" +
+            ", clientSecret='" + clientSecret + "'" +
+            "}";
+    }
+
+    public Date getLastAccessTokenExpirationDate(){
+        return tokenExpiresOnDate;
+    }
+
+    private IAuthenticationResult getAccessTokenByClientCredentialGrant()
+        throws MalformedURLException, ExecutionException, InterruptedException {
+
+        ConfidentialClientApplication app = ConfidentialClientApplication.builder(
+                this.clientId,
+                ClientCredentialFactory.createFromSecret(this.clientSecret))
+                .authority(String.format("%s/%s", authorityEndpoint, tenantId))
+                .build();
+
+        // With client credentials flows the scope is ALWAYS of the shape "resource/.default", as the
+        // application permissions need to be set statically (in the portal), and then granted by a tenant administrator
+        ClientCredentialParameters clientCredentialParam = ClientCredentialParameters.builder(
+                Collections.singleton(GRAPH_DEFAULT_SCOPE))
+                .build();
+
+        CompletableFuture<IAuthenticationResult> future = app.acquireToken(clientCredentialParam);
+        return future.get();
+    }
+
+    private String getAccessToken() {
+        Date now = new Date();
+        if ((lastAcessToken != null) && (tokenExpiresOnDate != null) && (tokenExpiresOnDate.getTime() -now.getTime() > 60000)) {
+            return lastAcessToken;
+        } else {
+            try {
+                IAuthenticationResult result = getAccessTokenByClientCredentialGrant();
+                tokenExpiresOnDate = result.expiresOnDate(); // store this for token expiration checking
+                lastAcessToken = result.accessToken();
+            } catch(Exception ex) {
+                logger.error("Failed to get access token", ex);
+            }
+            return lastAcessToken;
+        }
+
+
+    }
+
+    @Override
+    public void authenticateRequest(IHttpRequest request) {
+        String accessToken = getAccessToken();
+        if (accessToken != null) {
+            request.addHeader("Authorization", "Bearer " + accessToken);
+        }
+
+    }
+
+    public static class Builder {
+
+        private String authorityEndpoint ="";

Review comment:
       ```suggestion
           private String authorityEndpoint = "";
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = getGroupsWith(groupFilterList, prefix, suffix, substring, pageSize);
+        refreshUserGroupData(groupDisplayNames);
+        stopWatch.stop();
+        if (logger.isDebugEnabled()) {
+            logger.debug("Refreshed {} user groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+    }
+
+    private Set<String> getGroupsWith(String groupFilterList, String prefix, String suffix, String substring, int pageSize) {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = new HashSet<>();
+
+        if (!StringUtils.isBlank(prefix) || !StringUtils.isBlank(suffix) || !StringUtils.isBlank(substring)) {
+            groupDisplayNames.addAll(queryGroupsWith(prefix, suffix, substring, pageSize));
+        }
+
+        if (!StringUtils.isBlank(groupFilterList)) {
+            groupDisplayNames.addAll(
+                Arrays.stream(groupFilterList.split(","))
+                    .map(String::trim)
+                    .collect(Collectors.toList())
+            );
+        }
+
+        stopWatch.stop();
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Fetched {} groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+
+        return Collections.unmodifiableSet(groupDisplayNames);
+    }
+
+    /**
+     * Get a set of group display names after filtering prefix, suffix, and substring
+     * @param prefix prefix filter string matching against displayName of group directory objects
+     * @param suffix suffix fitler string matching against displayName of group directory objects
+     * @param substring string matching against displayName of group directory objects
+     * @param pageSize page size to make graph rest calls in pagination
+     * @return set of group display names
+     */
+    private Set<String> queryGroupsWith(String prefix, String suffix, String substring, int pageSize) {
+        final Set<String> groups = new HashSet<>();
+        boolean filterEvaluation = false;
+        IGroupCollectionRequest gRequest;
+        IGroupCollectionPage filterResults;
+        if (prefix != null && !prefix.isEmpty()) {
+            // build a $filter query option and create a graph request if prefix is given
+            final List<Option> requestOptions = new LinkedList<Option>();
+            requestOptions.add(new QueryOption("$filter", String.format("startswith(displayName, '%s')", prefix)));
+            gRequest = graphClient.groups().buildRequest(requestOptions).select("displayName");
+        } else {
+            // default group graph request
+            gRequest = graphClient.groups().buildRequest().select("displayName");
+        }
+        if (pageSize > 0) {
+            gRequest = gRequest.top(pageSize);
+        }
+        filterResults = gRequest.get();
+
+        List<com.microsoft.graph.models.extensions.Group> currentPage = filterResults.getCurrentPage();
+        while (currentPage != null) {
+            for (com.microsoft.graph.models.extensions.Group grp : currentPage) {
+                filterEvaluation = true;
+                if (!StringUtils.isEmpty(suffix) && !grp.displayName.endsWith(suffix)) {
+                    filterEvaluation = false;
+                }
+                if (!StringUtils.isEmpty(substring) && !grp.displayName.contains(substring)) {
+                    filterEvaluation = false;
+                }
+                if (filterEvaluation) {
+                    groups.add(grp.displayName);
+                }
+            }
+            IGroupCollectionRequestBuilder gBuilder = filterResults.getNextPage();
+            if (gBuilder != null) {
+                filterResults = gBuilder.buildRequest().get();
+                currentPage = filterResults.getCurrentPage();
+            } else {
+                currentPage = null;
+            }
+        }
+
+        return Collections.unmodifiableSet(groups);
+    }
+
+    /**
+     * Get member users of the given group name
+     * @param groupName group name to search for member users
+     * @return UserGroupQueryResult
+     */
+    private UserGroupQueryResult getUsersFrom(String groupName) throws IOException, ClientException {
+        final Set<User> users = new HashSet<>();
+
+        final List<Option> requestOptions = Arrays.asList(new QueryOption("$filter", String.format("displayName eq '%s'", groupName)));
+        final IGroupCollectionPage results = graphClient.groups().buildRequest(requestOptions).get();
+        final List<com.microsoft.graph.models.extensions.Group> currentPage = results.getCurrentPage();
+
+        if (currentPage != null && currentPage.size() > 0) {
+            final com.microsoft.graph.models.extensions.Group graphGroup = results.getCurrentPage().get(0);
+            final Group.Builder groupBuilder =
+                new Group.Builder()
+                    .identifier(graphGroup.id)
+                    .name(graphGroup.displayName);
+
+            IDirectoryObjectCollectionWithReferencesRequest uRequest =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName");
+
+            if (pageSize > 0) {
+                uRequest = uRequest.top(pageSize);
+            }
+            IDirectoryObjectCollectionWithReferencesPage userpage =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName").get();
+
+            while (userpage.getCurrentPage() != null) {
+                for (DirectoryObject userDO : userpage.getCurrentPage()) {
+                    JsonObject jsonUser = userDO.getRawObject();
+                    final String idUser = jsonUser.get("id").getAsString();
+                    // upn is default fallback claim for userName
+                    // upn claim maps to 'mail' property in Azure graph rest-api.
+                    final String userName;
+                    if (claimForUserName.equals("email")) {
+                        // authentication token contains email field, while graph api returns mail property
+                        if (!jsonUser.get("mail").isJsonNull()) {
+                            userName = jsonUser.get("mail").getAsString();
+                        } else {
+                            userName = jsonUser.get("userPrincipalName").getAsString();
+                        }
+                    } else {
+                        userName = jsonUser.get("userPrincipalName").getAsString();
+                    }
+                    final User user = new User.Builder().identifier(idUser).identity(userName).build();
+                    users.add(user);
+                    groupBuilder.addUser(idUser);
+                }
+                IDirectoryObjectCollectionWithReferencesRequestBuilder nextPageRequest = userpage.getNextPage();
+
+                if (nextPageRequest != null) {
+                    userpage = nextPageRequest.buildRequest().get();
+                } else {
+                    break;
+                }
+            }
+            final Group group = groupBuilder.build();
+            return new UserGroupQueryResult(group, users);
+        } else if (logger.isDebugEnabled()) {
+            logger.debug("Group collection page for {} was null or empty", groupName);
+        }
+        return null;
+    }
+
+    /**
+     * refresh the user & group data for UserGroupProvider plugin service
+     * @param groupDisplayNames a list of group display names
+     */
+    private void refreshUserGroupData(Set<String> groupDisplayNames) throws IOException, ClientException {
+        Objects.requireNonNull(groupDisplayNames);
+
+        final long startTime = System.currentTimeMillis();
+        final Set<User> _users = new HashSet<>();
+        final Set<Group> _groups = new HashSet<>();
+
+        for (String grpFilter : groupDisplayNames) {
+            if (logger.isDebugEnabled()) logger.debug("Getting users for group filter: {}", grpFilter);

Review comment:
       ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/test/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProviderIT.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import static org.junit.Assert.fail;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.util.MockPropertyValue;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AzureGraphUserGroupProviderIT {
+    private static final Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProviderIT.class);
+
+    private static final Properties CONFIG;
+
+    private static final String CREDENTIALS_FILE = System.getProperty("user.home") + "/azure-aad-app-reg.PROPERTIES";
+
+    static {
+        CONFIG = new Properties();
+        try {
+            final FileInputStream fis = new FileInputStream(CREDENTIALS_FILE);
+            try {
+                CONFIG.load(fis);
+            } catch (IOException e) {
+                fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+            } finally {
+                FileUtils.closeQuietly(fis);
+            }
+        } catch (FileNotFoundException e) {
+            fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+        }
+    }
+
+    protected static String getAuthorityEndpoint() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY);
+    }
+
+    protected static String getTenantId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientSecret() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY);
+    }
+
+    protected static String getKnownTestUserName() {
+        return CONFIG.getProperty("KNOWN_TEST_USER");
+
+    }
+
+    protected static String getKnownTestGroupName() {
+        return CONFIG.getProperty("KNOWN_TEST_GROUP");
+    }
+
+    protected static String getGroupListInclusion() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY);
+    }
+
+    private AuthorizerConfigurationContext authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+    private AzureGraphUserGroupProvider testingProvider;
+    private UserGroupProviderInitializationContext initContext;
+
+    @Before
+    public void setup() throws IOException {
+        authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+        initContext = Mockito.mock(UserGroupProviderInitializationContext.class);
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAuthorityEndpoint()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getTenantId()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientId()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientSecret()));
+    }
+
+    private void setupTestingProvider() {
+        testingProvider = new AzureGraphUserGroupProvider();
+        try {
+            testingProvider.initialize(initContext);
+            testingProvider.onConfigured(authContext);
+        } catch (final Exception exc) {
+            logger.error("setup() exception: " + exc + "; tests cannot run on this system.");
+            return;
+        }
+    }
+
+

Review comment:
       ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/test/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProviderIT.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import static org.junit.Assert.fail;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.util.MockPropertyValue;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AzureGraphUserGroupProviderIT {
+    private static final Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProviderIT.class);
+
+    private static final Properties CONFIG;
+
+    private static final String CREDENTIALS_FILE = System.getProperty("user.home") + "/azure-aad-app-reg.PROPERTIES";
+
+    static {
+        CONFIG = new Properties();
+        try {
+            final FileInputStream fis = new FileInputStream(CREDENTIALS_FILE);
+            try {
+                CONFIG.load(fis);
+            } catch (IOException e) {
+                fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+            } finally {
+                FileUtils.closeQuietly(fis);
+            }
+        } catch (FileNotFoundException e) {
+            fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+        }
+    }
+
+    protected static String getAuthorityEndpoint() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY);
+    }
+
+    protected static String getTenantId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientSecret() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY);
+    }
+
+    protected static String getKnownTestUserName() {
+        return CONFIG.getProperty("KNOWN_TEST_USER");
+
+    }
+
+    protected static String getKnownTestGroupName() {
+        return CONFIG.getProperty("KNOWN_TEST_GROUP");
+    }
+
+    protected static String getGroupListInclusion() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY);
+    }
+
+    private AuthorizerConfigurationContext authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+    private AzureGraphUserGroupProvider testingProvider;
+    private UserGroupProviderInitializationContext initContext;
+
+    @Before
+    public void setup() throws IOException {
+        authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+        initContext = Mockito.mock(UserGroupProviderInitializationContext.class);
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAuthorityEndpoint()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getTenantId()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientId()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientSecret()));
+    }
+
+    private void setupTestingProvider() {
+        testingProvider = new AzureGraphUserGroupProvider();
+        try {
+            testingProvider.initialize(initContext);
+            testingProvider.onConfigured(authContext);
+        } catch (final Exception exc) {
+            logger.error("setup() exception: " + exc + "; tests cannot run on this system.");
+            return;
+        }
+    }
+
+
+    @After
+    public void tearDown() {
+        testingProvider.preDestruction();
+    }
+
+    @Test
+    public void testWithGroupListFilter(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+    }
+
+    @Test
+    public void testWithPaging(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.PAGE_SIZE_PROPERTY)))
+        .thenReturn(new MockPropertyValue("3")); // in the real scenario, this should be 20 or bigger.
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);

Review comment:
       ```suggestion
           Assert.assertNotNull(uag.getUser());
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/test/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProviderIT.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import static org.junit.Assert.fail;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.util.MockPropertyValue;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AzureGraphUserGroupProviderIT {
+    private static final Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProviderIT.class);
+
+    private static final Properties CONFIG;
+
+    private static final String CREDENTIALS_FILE = System.getProperty("user.home") + "/azure-aad-app-reg.PROPERTIES";
+
+    static {
+        CONFIG = new Properties();
+        try {
+            final FileInputStream fis = new FileInputStream(CREDENTIALS_FILE);
+            try {
+                CONFIG.load(fis);
+            } catch (IOException e) {
+                fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+            } finally {
+                FileUtils.closeQuietly(fis);
+            }
+        } catch (FileNotFoundException e) {
+            fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+        }
+    }
+
+    protected static String getAuthorityEndpoint() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY);
+    }
+
+    protected static String getTenantId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientSecret() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY);
+    }
+
+    protected static String getKnownTestUserName() {
+        return CONFIG.getProperty("KNOWN_TEST_USER");
+
+    }
+
+    protected static String getKnownTestGroupName() {
+        return CONFIG.getProperty("KNOWN_TEST_GROUP");
+    }
+
+    protected static String getGroupListInclusion() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY);
+    }
+
+    private AuthorizerConfigurationContext authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+    private AzureGraphUserGroupProvider testingProvider;
+    private UserGroupProviderInitializationContext initContext;
+
+    @Before
+    public void setup() throws IOException {
+        authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+        initContext = Mockito.mock(UserGroupProviderInitializationContext.class);
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAuthorityEndpoint()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getTenantId()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientId()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientSecret()));
+    }
+
+    private void setupTestingProvider() {
+        testingProvider = new AzureGraphUserGroupProvider();
+        try {
+            testingProvider.initialize(initContext);
+            testingProvider.onConfigured(authContext);
+        } catch (final Exception exc) {
+            logger.error("setup() exception: " + exc + "; tests cannot run on this system.");

Review comment:
       ```suggestion
               logger.error("Error during setup; tests cannot run on this system due to {}", exc.getMessage(), exc);
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = getGroupsWith(groupFilterList, prefix, suffix, substring, pageSize);
+        refreshUserGroupData(groupDisplayNames);
+        stopWatch.stop();
+        if (logger.isDebugEnabled()) {
+            logger.debug("Refreshed {} user groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+    }
+
+    private Set<String> getGroupsWith(String groupFilterList, String prefix, String suffix, String substring, int pageSize) {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = new HashSet<>();
+
+        if (!StringUtils.isBlank(prefix) || !StringUtils.isBlank(suffix) || !StringUtils.isBlank(substring)) {
+            groupDisplayNames.addAll(queryGroupsWith(prefix, suffix, substring, pageSize));
+        }
+
+        if (!StringUtils.isBlank(groupFilterList)) {
+            groupDisplayNames.addAll(
+                Arrays.stream(groupFilterList.split(","))
+                    .map(String::trim)
+                    .collect(Collectors.toList())
+            );
+        }
+
+        stopWatch.stop();
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Fetched {} groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+
+        return Collections.unmodifiableSet(groupDisplayNames);
+    }
+
+    /**
+     * Get a set of group display names after filtering prefix, suffix, and substring
+     * @param prefix prefix filter string matching against displayName of group directory objects
+     * @param suffix suffix fitler string matching against displayName of group directory objects
+     * @param substring string matching against displayName of group directory objects
+     * @param pageSize page size to make graph rest calls in pagination
+     * @return set of group display names
+     */
+    private Set<String> queryGroupsWith(String prefix, String suffix, String substring, int pageSize) {
+        final Set<String> groups = new HashSet<>();
+        boolean filterEvaluation = false;
+        IGroupCollectionRequest gRequest;
+        IGroupCollectionPage filterResults;
+        if (prefix != null && !prefix.isEmpty()) {
+            // build a $filter query option and create a graph request if prefix is given
+            final List<Option> requestOptions = new LinkedList<Option>();
+            requestOptions.add(new QueryOption("$filter", String.format("startswith(displayName, '%s')", prefix)));
+            gRequest = graphClient.groups().buildRequest(requestOptions).select("displayName");
+        } else {
+            // default group graph request
+            gRequest = graphClient.groups().buildRequest().select("displayName");
+        }
+        if (pageSize > 0) {
+            gRequest = gRequest.top(pageSize);
+        }
+        filterResults = gRequest.get();
+
+        List<com.microsoft.graph.models.extensions.Group> currentPage = filterResults.getCurrentPage();
+        while (currentPage != null) {
+            for (com.microsoft.graph.models.extensions.Group grp : currentPage) {
+                filterEvaluation = true;
+                if (!StringUtils.isEmpty(suffix) && !grp.displayName.endsWith(suffix)) {
+                    filterEvaluation = false;
+                }
+                if (!StringUtils.isEmpty(substring) && !grp.displayName.contains(substring)) {
+                    filterEvaluation = false;
+                }
+                if (filterEvaluation) {
+                    groups.add(grp.displayName);
+                }
+            }
+            IGroupCollectionRequestBuilder gBuilder = filterResults.getNextPage();
+            if (gBuilder != null) {
+                filterResults = gBuilder.buildRequest().get();
+                currentPage = filterResults.getCurrentPage();
+            } else {
+                currentPage = null;
+            }
+        }
+
+        return Collections.unmodifiableSet(groups);
+    }
+
+    /**
+     * Get member users of the given group name
+     * @param groupName group name to search for member users
+     * @return UserGroupQueryResult
+     */
+    private UserGroupQueryResult getUsersFrom(String groupName) throws IOException, ClientException {
+        final Set<User> users = new HashSet<>();
+
+        final List<Option> requestOptions = Arrays.asList(new QueryOption("$filter", String.format("displayName eq '%s'", groupName)));
+        final IGroupCollectionPage results = graphClient.groups().buildRequest(requestOptions).get();
+        final List<com.microsoft.graph.models.extensions.Group> currentPage = results.getCurrentPage();
+
+        if (currentPage != null && currentPage.size() > 0) {
+            final com.microsoft.graph.models.extensions.Group graphGroup = results.getCurrentPage().get(0);
+            final Group.Builder groupBuilder =
+                new Group.Builder()
+                    .identifier(graphGroup.id)
+                    .name(graphGroup.displayName);
+
+            IDirectoryObjectCollectionWithReferencesRequest uRequest =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName");
+
+            if (pageSize > 0) {
+                uRequest = uRequest.top(pageSize);
+            }
+            IDirectoryObjectCollectionWithReferencesPage userpage =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName").get();
+
+            while (userpage.getCurrentPage() != null) {
+                for (DirectoryObject userDO : userpage.getCurrentPage()) {
+                    JsonObject jsonUser = userDO.getRawObject();
+                    final String idUser = jsonUser.get("id").getAsString();
+                    // upn is default fallback claim for userName
+                    // upn claim maps to 'mail' property in Azure graph rest-api.
+                    final String userName;
+                    if (claimForUserName.equals("email")) {
+                        // authentication token contains email field, while graph api returns mail property
+                        if (!jsonUser.get("mail").isJsonNull()) {
+                            userName = jsonUser.get("mail").getAsString();
+                        } else {
+                            userName = jsonUser.get("userPrincipalName").getAsString();
+                        }
+                    } else {
+                        userName = jsonUser.get("userPrincipalName").getAsString();
+                    }
+                    final User user = new User.Builder().identifier(idUser).identity(userName).build();
+                    users.add(user);
+                    groupBuilder.addUser(idUser);
+                }
+                IDirectoryObjectCollectionWithReferencesRequestBuilder nextPageRequest = userpage.getNextPage();
+
+                if (nextPageRequest != null) {
+                    userpage = nextPageRequest.buildRequest().get();
+                } else {
+                    break;
+                }
+            }
+            final Group group = groupBuilder.build();
+            return new UserGroupQueryResult(group, users);
+        } else if (logger.isDebugEnabled()) {
+            logger.debug("Group collection page for {} was null or empty", groupName);
+        }
+        return null;
+    }
+
+    /**
+     * refresh the user & group data for UserGroupProvider plugin service
+     * @param groupDisplayNames a list of group display names
+     */
+    private void refreshUserGroupData(Set<String> groupDisplayNames) throws IOException, ClientException {
+        Objects.requireNonNull(groupDisplayNames);
+
+        final long startTime = System.currentTimeMillis();
+        final Set<User> _users = new HashSet<>();
+        final Set<Group> _groups = new HashSet<>();
+
+        for (String grpFilter : groupDisplayNames) {
+            if (logger.isDebugEnabled()) logger.debug("Getting users for group filter: {}", grpFilter);
+            UserGroupQueryResult queryResult = getUsersFrom(grpFilter);
+            if (queryResult != null) {
+                if (logger.isDebugEnabled()) {
+                    logger.debug("Found {} users for group: {}", queryResult.getUsers().size(), queryResult.getGroup().getName());
+                }
+                _groups.add(queryResult.getGroup());
+                _users.addAll(queryResult.getUsers());
+            } else {
+                if (logger.isDebugEnabled()) logger.debug("Query result was null");
+            }
+        }
+        final ImmutableAzureGraphUserGroup azureGraphUserGroup =
+            ImmutableAzureGraphUserGroup.newInstance(_users, _groups);
+        azureGraphUserGroupRef.set(azureGraphUserGroup);
+        final long endTime = System.currentTimeMillis();
+        if (logger.isDebugEnabled()) logger.debug("Refreshed users and groups, took {} miliseconds", (endTime - startTime));

Review comment:
       ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = getGroupsWith(groupFilterList, prefix, suffix, substring, pageSize);
+        refreshUserGroupData(groupDisplayNames);
+        stopWatch.stop();
+        if (logger.isDebugEnabled()) {
+            logger.debug("Refreshed {} user groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+    }
+
+    private Set<String> getGroupsWith(String groupFilterList, String prefix, String suffix, String substring, int pageSize) {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = new HashSet<>();
+
+        if (!StringUtils.isBlank(prefix) || !StringUtils.isBlank(suffix) || !StringUtils.isBlank(substring)) {
+            groupDisplayNames.addAll(queryGroupsWith(prefix, suffix, substring, pageSize));
+        }
+
+        if (!StringUtils.isBlank(groupFilterList)) {
+            groupDisplayNames.addAll(
+                Arrays.stream(groupFilterList.split(","))
+                    .map(String::trim)
+                    .collect(Collectors.toList())
+            );
+        }
+
+        stopWatch.stop();
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Fetched {} groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+
+        return Collections.unmodifiableSet(groupDisplayNames);
+    }
+
+    /**
+     * Get a set of group display names after filtering prefix, suffix, and substring
+     * @param prefix prefix filter string matching against displayName of group directory objects
+     * @param suffix suffix fitler string matching against displayName of group directory objects
+     * @param substring string matching against displayName of group directory objects
+     * @param pageSize page size to make graph rest calls in pagination
+     * @return set of group display names
+     */
+    private Set<String> queryGroupsWith(String prefix, String suffix, String substring, int pageSize) {
+        final Set<String> groups = new HashSet<>();
+        boolean filterEvaluation = false;
+        IGroupCollectionRequest gRequest;
+        IGroupCollectionPage filterResults;
+        if (prefix != null && !prefix.isEmpty()) {
+            // build a $filter query option and create a graph request if prefix is given
+            final List<Option> requestOptions = new LinkedList<Option>();
+            requestOptions.add(new QueryOption("$filter", String.format("startswith(displayName, '%s')", prefix)));
+            gRequest = graphClient.groups().buildRequest(requestOptions).select("displayName");
+        } else {
+            // default group graph request
+            gRequest = graphClient.groups().buildRequest().select("displayName");
+        }
+        if (pageSize > 0) {
+            gRequest = gRequest.top(pageSize);
+        }
+        filterResults = gRequest.get();
+
+        List<com.microsoft.graph.models.extensions.Group> currentPage = filterResults.getCurrentPage();
+        while (currentPage != null) {
+            for (com.microsoft.graph.models.extensions.Group grp : currentPage) {
+                filterEvaluation = true;
+                if (!StringUtils.isEmpty(suffix) && !grp.displayName.endsWith(suffix)) {
+                    filterEvaluation = false;
+                }
+                if (!StringUtils.isEmpty(substring) && !grp.displayName.contains(substring)) {
+                    filterEvaluation = false;
+                }
+                if (filterEvaluation) {
+                    groups.add(grp.displayName);
+                }
+            }
+            IGroupCollectionRequestBuilder gBuilder = filterResults.getNextPage();
+            if (gBuilder != null) {
+                filterResults = gBuilder.buildRequest().get();
+                currentPage = filterResults.getCurrentPage();
+            } else {
+                currentPage = null;
+            }
+        }
+
+        return Collections.unmodifiableSet(groups);
+    }
+
+    /**
+     * Get member users of the given group name
+     * @param groupName group name to search for member users
+     * @return UserGroupQueryResult
+     */
+    private UserGroupQueryResult getUsersFrom(String groupName) throws IOException, ClientException {
+        final Set<User> users = new HashSet<>();
+
+        final List<Option> requestOptions = Arrays.asList(new QueryOption("$filter", String.format("displayName eq '%s'", groupName)));
+        final IGroupCollectionPage results = graphClient.groups().buildRequest(requestOptions).get();
+        final List<com.microsoft.graph.models.extensions.Group> currentPage = results.getCurrentPage();
+
+        if (currentPage != null && currentPage.size() > 0) {
+            final com.microsoft.graph.models.extensions.Group graphGroup = results.getCurrentPage().get(0);
+            final Group.Builder groupBuilder =
+                new Group.Builder()
+                    .identifier(graphGroup.id)
+                    .name(graphGroup.displayName);
+
+            IDirectoryObjectCollectionWithReferencesRequest uRequest =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName");
+
+            if (pageSize > 0) {
+                uRequest = uRequest.top(pageSize);
+            }
+            IDirectoryObjectCollectionWithReferencesPage userpage =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName").get();
+
+            while (userpage.getCurrentPage() != null) {
+                for (DirectoryObject userDO : userpage.getCurrentPage()) {
+                    JsonObject jsonUser = userDO.getRawObject();
+                    final String idUser = jsonUser.get("id").getAsString();
+                    // upn is default fallback claim for userName
+                    // upn claim maps to 'mail' property in Azure graph rest-api.
+                    final String userName;
+                    if (claimForUserName.equals("email")) {
+                        // authentication token contains email field, while graph api returns mail property
+                        if (!jsonUser.get("mail").isJsonNull()) {
+                            userName = jsonUser.get("mail").getAsString();
+                        } else {
+                            userName = jsonUser.get("userPrincipalName").getAsString();
+                        }
+                    } else {
+                        userName = jsonUser.get("userPrincipalName").getAsString();
+                    }
+                    final User user = new User.Builder().identifier(idUser).identity(userName).build();
+                    users.add(user);
+                    groupBuilder.addUser(idUser);
+                }
+                IDirectoryObjectCollectionWithReferencesRequestBuilder nextPageRequest = userpage.getNextPage();
+
+                if (nextPageRequest != null) {
+                    userpage = nextPageRequest.buildRequest().get();
+                } else {
+                    break;
+                }
+            }
+            final Group group = groupBuilder.build();
+            return new UserGroupQueryResult(group, users);
+        } else if (logger.isDebugEnabled()) {
+            logger.debug("Group collection page for {} was null or empty", groupName);
+        }

Review comment:
       ```suggestion
           }
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/test/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProviderIT.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import static org.junit.Assert.fail;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.util.MockPropertyValue;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AzureGraphUserGroupProviderIT {
+    private static final Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProviderIT.class);
+
+    private static final Properties CONFIG;
+
+    private static final String CREDENTIALS_FILE = System.getProperty("user.home") + "/azure-aad-app-reg.PROPERTIES";
+
+    static {
+        CONFIG = new Properties();
+        try {
+            final FileInputStream fis = new FileInputStream(CREDENTIALS_FILE);
+            try {
+                CONFIG.load(fis);
+            } catch (IOException e) {
+                fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+            } finally {
+                FileUtils.closeQuietly(fis);
+            }
+        } catch (FileNotFoundException e) {
+            fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+        }
+    }
+
+    protected static String getAuthorityEndpoint() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY);
+    }
+
+    protected static String getTenantId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientSecret() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY);
+    }
+
+    protected static String getKnownTestUserName() {
+        return CONFIG.getProperty("KNOWN_TEST_USER");
+
+    }
+
+    protected static String getKnownTestGroupName() {
+        return CONFIG.getProperty("KNOWN_TEST_GROUP");
+    }
+
+    protected static String getGroupListInclusion() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY);
+    }
+
+    private AuthorizerConfigurationContext authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+    private AzureGraphUserGroupProvider testingProvider;
+    private UserGroupProviderInitializationContext initContext;
+
+    @Before
+    public void setup() throws IOException {
+        authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+        initContext = Mockito.mock(UserGroupProviderInitializationContext.class);
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAuthorityEndpoint()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getTenantId()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientId()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientSecret()));
+    }
+
+    private void setupTestingProvider() {
+        testingProvider = new AzureGraphUserGroupProvider();
+        try {
+            testingProvider.initialize(initContext);
+            testingProvider.onConfigured(authContext);
+        } catch (final Exception exc) {
+            logger.error("setup() exception: " + exc + "; tests cannot run on this system.");
+            return;
+        }
+    }
+
+
+    @After
+    public void tearDown() {
+        testingProvider.preDestruction();
+    }
+
+    @Test
+    public void testWithGroupListFilter(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+    }
+
+    @Test
+    public void testWithPaging(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.PAGE_SIZE_PROPERTY)))
+        .thenReturn(new MockPropertyValue("3")); // in the real scenario, this should be 20 or bigger.
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+        String knownGroupName = getKnownTestGroupName();
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());

Review comment:
       ```suggestion
           List<Group> search = testingProvider.getGroups().stream().filter(g -> g.getName().equals(knownGroupName)).collect(Collectors.toList());
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = getGroupsWith(groupFilterList, prefix, suffix, substring, pageSize);
+        refreshUserGroupData(groupDisplayNames);
+        stopWatch.stop();
+        if (logger.isDebugEnabled()) {
+            logger.debug("Refreshed {} user groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+    }
+
+    private Set<String> getGroupsWith(String groupFilterList, String prefix, String suffix, String substring, int pageSize) {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = new HashSet<>();
+
+        if (!StringUtils.isBlank(prefix) || !StringUtils.isBlank(suffix) || !StringUtils.isBlank(substring)) {
+            groupDisplayNames.addAll(queryGroupsWith(prefix, suffix, substring, pageSize));
+        }
+
+        if (!StringUtils.isBlank(groupFilterList)) {
+            groupDisplayNames.addAll(
+                Arrays.stream(groupFilterList.split(","))
+                    .map(String::trim)
+                    .collect(Collectors.toList())
+            );
+        }
+
+        stopWatch.stop();
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Fetched {} groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+
+        return Collections.unmodifiableSet(groupDisplayNames);
+    }
+
+    /**
+     * Get a set of group display names after filtering prefix, suffix, and substring
+     * @param prefix prefix filter string matching against displayName of group directory objects
+     * @param suffix suffix fitler string matching against displayName of group directory objects
+     * @param substring string matching against displayName of group directory objects
+     * @param pageSize page size to make graph rest calls in pagination
+     * @return set of group display names
+     */
+    private Set<String> queryGroupsWith(String prefix, String suffix, String substring, int pageSize) {
+        final Set<String> groups = new HashSet<>();
+        boolean filterEvaluation = false;
+        IGroupCollectionRequest gRequest;
+        IGroupCollectionPage filterResults;
+        if (prefix != null && !prefix.isEmpty()) {
+            // build a $filter query option and create a graph request if prefix is given
+            final List<Option> requestOptions = new LinkedList<Option>();
+            requestOptions.add(new QueryOption("$filter", String.format("startswith(displayName, '%s')", prefix)));
+            gRequest = graphClient.groups().buildRequest(requestOptions).select("displayName");
+        } else {
+            // default group graph request
+            gRequest = graphClient.groups().buildRequest().select("displayName");
+        }
+        if (pageSize > 0) {
+            gRequest = gRequest.top(pageSize);
+        }
+        filterResults = gRequest.get();
+
+        List<com.microsoft.graph.models.extensions.Group> currentPage = filterResults.getCurrentPage();
+        while (currentPage != null) {
+            for (com.microsoft.graph.models.extensions.Group grp : currentPage) {
+                filterEvaluation = true;
+                if (!StringUtils.isEmpty(suffix) && !grp.displayName.endsWith(suffix)) {
+                    filterEvaluation = false;
+                }
+                if (!StringUtils.isEmpty(substring) && !grp.displayName.contains(substring)) {
+                    filterEvaluation = false;
+                }
+                if (filterEvaluation) {
+                    groups.add(grp.displayName);
+                }
+            }
+            IGroupCollectionRequestBuilder gBuilder = filterResults.getNextPage();
+            if (gBuilder != null) {
+                filterResults = gBuilder.buildRequest().get();
+                currentPage = filterResults.getCurrentPage();
+            } else {
+                currentPage = null;
+            }
+        }
+
+        return Collections.unmodifiableSet(groups);
+    }
+
+    /**
+     * Get member users of the given group name
+     * @param groupName group name to search for member users
+     * @return UserGroupQueryResult
+     */
+    private UserGroupQueryResult getUsersFrom(String groupName) throws IOException, ClientException {
+        final Set<User> users = new HashSet<>();
+
+        final List<Option> requestOptions = Arrays.asList(new QueryOption("$filter", String.format("displayName eq '%s'", groupName)));
+        final IGroupCollectionPage results = graphClient.groups().buildRequest(requestOptions).get();
+        final List<com.microsoft.graph.models.extensions.Group> currentPage = results.getCurrentPage();
+
+        if (currentPage != null && currentPage.size() > 0) {
+            final com.microsoft.graph.models.extensions.Group graphGroup = results.getCurrentPage().get(0);
+            final Group.Builder groupBuilder =
+                new Group.Builder()
+                    .identifier(graphGroup.id)
+                    .name(graphGroup.displayName);
+
+            IDirectoryObjectCollectionWithReferencesRequest uRequest =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName");
+
+            if (pageSize > 0) {
+                uRequest = uRequest.top(pageSize);
+            }
+            IDirectoryObjectCollectionWithReferencesPage userpage =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName").get();
+
+            while (userpage.getCurrentPage() != null) {

Review comment:
       I think we need to guard against nulls in `jsonUser.get("id").getAsString()` and in `userName`.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = getGroupsWith(groupFilterList, prefix, suffix, substring, pageSize);
+        refreshUserGroupData(groupDisplayNames);
+        stopWatch.stop();
+        if (logger.isDebugEnabled()) {
+            logger.debug("Refreshed {} user groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+    }
+
+    private Set<String> getGroupsWith(String groupFilterList, String prefix, String suffix, String substring, int pageSize) {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = new HashSet<>();
+
+        if (!StringUtils.isBlank(prefix) || !StringUtils.isBlank(suffix) || !StringUtils.isBlank(substring)) {
+            groupDisplayNames.addAll(queryGroupsWith(prefix, suffix, substring, pageSize));
+        }
+
+        if (!StringUtils.isBlank(groupFilterList)) {
+            groupDisplayNames.addAll(
+                Arrays.stream(groupFilterList.split(","))
+                    .map(String::trim)
+                    .collect(Collectors.toList())
+            );
+        }
+
+        stopWatch.stop();
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Fetched {} groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+
+        return Collections.unmodifiableSet(groupDisplayNames);
+    }
+
+    /**
+     * Get a set of group display names after filtering prefix, suffix, and substring
+     * @param prefix prefix filter string matching against displayName of group directory objects
+     * @param suffix suffix fitler string matching against displayName of group directory objects
+     * @param substring string matching against displayName of group directory objects
+     * @param pageSize page size to make graph rest calls in pagination
+     * @return set of group display names
+     */
+    private Set<String> queryGroupsWith(String prefix, String suffix, String substring, int pageSize) {
+        final Set<String> groups = new HashSet<>();
+        boolean filterEvaluation = false;
+        IGroupCollectionRequest gRequest;
+        IGroupCollectionPage filterResults;
+        if (prefix != null && !prefix.isEmpty()) {
+            // build a $filter query option and create a graph request if prefix is given
+            final List<Option> requestOptions = new LinkedList<Option>();
+            requestOptions.add(new QueryOption("$filter", String.format("startswith(displayName, '%s')", prefix)));
+            gRequest = graphClient.groups().buildRequest(requestOptions).select("displayName");
+        } else {
+            // default group graph request
+            gRequest = graphClient.groups().buildRequest().select("displayName");
+        }
+        if (pageSize > 0) {
+            gRequest = gRequest.top(pageSize);
+        }
+        filterResults = gRequest.get();
+
+        List<com.microsoft.graph.models.extensions.Group> currentPage = filterResults.getCurrentPage();
+        while (currentPage != null) {
+            for (com.microsoft.graph.models.extensions.Group grp : currentPage) {
+                filterEvaluation = true;
+                if (!StringUtils.isEmpty(suffix) && !grp.displayName.endsWith(suffix)) {
+                    filterEvaluation = false;
+                }
+                if (!StringUtils.isEmpty(substring) && !grp.displayName.contains(substring)) {
+                    filterEvaluation = false;
+                }
+                if (filterEvaluation) {
+                    groups.add(grp.displayName);
+                }
+            }
+            IGroupCollectionRequestBuilder gBuilder = filterResults.getNextPage();
+            if (gBuilder != null) {
+                filterResults = gBuilder.buildRequest().get();
+                currentPage = filterResults.getCurrentPage();
+            } else {
+                currentPage = null;
+            }
+        }
+
+        return Collections.unmodifiableSet(groups);
+    }
+
+    /**
+     * Get member users of the given group name
+     * @param groupName group name to search for member users
+     * @return UserGroupQueryResult
+     */
+    private UserGroupQueryResult getUsersFrom(String groupName) throws IOException, ClientException {
+        final Set<User> users = new HashSet<>();
+
+        final List<Option> requestOptions = Arrays.asList(new QueryOption("$filter", String.format("displayName eq '%s'", groupName)));
+        final IGroupCollectionPage results = graphClient.groups().buildRequest(requestOptions).get();
+        final List<com.microsoft.graph.models.extensions.Group> currentPage = results.getCurrentPage();
+
+        if (currentPage != null && currentPage.size() > 0) {
+            final com.microsoft.graph.models.extensions.Group graphGroup = results.getCurrentPage().get(0);
+            final Group.Builder groupBuilder =
+                new Group.Builder()
+                    .identifier(graphGroup.id)
+                    .name(graphGroup.displayName);
+
+            IDirectoryObjectCollectionWithReferencesRequest uRequest =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName");
+
+            if (pageSize > 0) {
+                uRequest = uRequest.top(pageSize);
+            }
+            IDirectoryObjectCollectionWithReferencesPage userpage =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName").get();
+
+            while (userpage.getCurrentPage() != null) {
+                for (DirectoryObject userDO : userpage.getCurrentPage()) {
+                    JsonObject jsonUser = userDO.getRawObject();
+                    final String idUser = jsonUser.get("id").getAsString();
+                    // upn is default fallback claim for userName
+                    // upn claim maps to 'mail' property in Azure graph rest-api.
+                    final String userName;
+                    if (claimForUserName.equals("email")) {
+                        // authentication token contains email field, while graph api returns mail property
+                        if (!jsonUser.get("mail").isJsonNull()) {
+                            userName = jsonUser.get("mail").getAsString();
+                        } else {
+                            userName = jsonUser.get("userPrincipalName").getAsString();
+                        }
+                    } else {
+                        userName = jsonUser.get("userPrincipalName").getAsString();
+                    }
+                    final User user = new User.Builder().identifier(idUser).identity(userName).build();
+                    users.add(user);
+                    groupBuilder.addUser(idUser);
+                }
+                IDirectoryObjectCollectionWithReferencesRequestBuilder nextPageRequest = userpage.getNextPage();
+
+                if (nextPageRequest != null) {
+                    userpage = nextPageRequest.buildRequest().get();
+                } else {
+                    break;
+                }
+            }
+            final Group group = groupBuilder.build();
+            return new UserGroupQueryResult(group, users);
+        } else if (logger.isDebugEnabled()) {
+            logger.debug("Group collection page for {} was null or empty", groupName);
+        }
+        return null;
+    }
+
+    /**
+     * refresh the user & group data for UserGroupProvider plugin service
+     * @param groupDisplayNames a list of group display names
+     */
+    private void refreshUserGroupData(Set<String> groupDisplayNames) throws IOException, ClientException {
+        Objects.requireNonNull(groupDisplayNames);
+
+        final long startTime = System.currentTimeMillis();
+        final Set<User> _users = new HashSet<>();
+        final Set<Group> _groups = new HashSet<>();
+
+        for (String grpFilter : groupDisplayNames) {
+            if (logger.isDebugEnabled()) logger.debug("Getting users for group filter: {}", grpFilter);
+            UserGroupQueryResult queryResult = getUsersFrom(grpFilter);
+            if (queryResult != null) {
+                if (logger.isDebugEnabled()) {
+                    logger.debug("Found {} users for group: {}", queryResult.getUsers().size(), queryResult.getGroup().getName());
+                }
+                _groups.add(queryResult.getGroup());
+                _users.addAll(queryResult.getUsers());
+            } else {
+                if (logger.isDebugEnabled()) logger.debug("Query result was null");
+            }
+        }
+        final ImmutableAzureGraphUserGroup azureGraphUserGroup =
+            ImmutableAzureGraphUserGroup.newInstance(_users, _groups);

Review comment:
       ```suggestion
               ImmutableAzureGraphUserGroup.newInstance(users, groups);
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/test/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProviderIT.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import static org.junit.Assert.fail;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.util.MockPropertyValue;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AzureGraphUserGroupProviderIT {
+    private static final Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProviderIT.class);
+
+    private static final Properties CONFIG;
+
+    private static final String CREDENTIALS_FILE = System.getProperty("user.home") + "/azure-aad-app-reg.PROPERTIES";
+
+    static {
+        CONFIG = new Properties();
+        try {
+            final FileInputStream fis = new FileInputStream(CREDENTIALS_FILE);
+            try {
+                CONFIG.load(fis);
+            } catch (IOException e) {
+                fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+            } finally {
+                FileUtils.closeQuietly(fis);
+            }
+        } catch (FileNotFoundException e) {
+            fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+        }
+    }
+
+    protected static String getAuthorityEndpoint() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY);
+    }
+
+    protected static String getTenantId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientSecret() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY);
+    }
+
+    protected static String getKnownTestUserName() {
+        return CONFIG.getProperty("KNOWN_TEST_USER");
+
+    }
+
+    protected static String getKnownTestGroupName() {
+        return CONFIG.getProperty("KNOWN_TEST_GROUP");
+    }
+
+    protected static String getGroupListInclusion() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY);
+    }
+
+    private AuthorizerConfigurationContext authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+    private AzureGraphUserGroupProvider testingProvider;
+    private UserGroupProviderInitializationContext initContext;
+
+    @Before
+    public void setup() throws IOException {
+        authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+        initContext = Mockito.mock(UserGroupProviderInitializationContext.class);
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAuthorityEndpoint()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getTenantId()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientId()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientSecret()));
+    }
+
+    private void setupTestingProvider() {
+        testingProvider = new AzureGraphUserGroupProvider();
+        try {
+            testingProvider.initialize(initContext);
+            testingProvider.onConfigured(authContext);
+        } catch (final Exception exc) {
+            logger.error("setup() exception: " + exc + "; tests cannot run on this system.");
+            return;
+        }
+    }
+
+
+    @After
+    public void tearDown() {
+        testingProvider.preDestruction();
+    }
+
+    @Test
+    public void testWithGroupListFilter(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+    }
+
+    @Test
+    public void testWithPaging(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.PAGE_SIZE_PROPERTY)))
+        .thenReturn(new MockPropertyValue("3")); // in the real scenario, this should be 20 or bigger.

Review comment:
       ```suggestion
               .thenReturn(new MockPropertyValue("3")); // in the real scenario, this should be 20 or bigger.
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/test/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProviderIT.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import static org.junit.Assert.fail;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.util.MockPropertyValue;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AzureGraphUserGroupProviderIT {
+    private static final Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProviderIT.class);
+
+    private static final Properties CONFIG;
+
+    private static final String CREDENTIALS_FILE = System.getProperty("user.home") + "/azure-aad-app-reg.PROPERTIES";
+
+    static {
+        CONFIG = new Properties();
+        try {
+            final FileInputStream fis = new FileInputStream(CREDENTIALS_FILE);
+            try {
+                CONFIG.load(fis);
+            } catch (IOException e) {
+                fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+            } finally {
+                FileUtils.closeQuietly(fis);
+            }
+        } catch (FileNotFoundException e) {
+            fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+        }
+    }
+
+    protected static String getAuthorityEndpoint() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY);
+    }
+
+    protected static String getTenantId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientSecret() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY);
+    }
+
+    protected static String getKnownTestUserName() {
+        return CONFIG.getProperty("KNOWN_TEST_USER");
+
+    }
+
+    protected static String getKnownTestGroupName() {
+        return CONFIG.getProperty("KNOWN_TEST_GROUP");
+    }
+
+    protected static String getGroupListInclusion() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY);
+    }
+
+    private AuthorizerConfigurationContext authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+    private AzureGraphUserGroupProvider testingProvider;
+    private UserGroupProviderInitializationContext initContext;
+
+    @Before
+    public void setup() throws IOException {
+        authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+        initContext = Mockito.mock(UserGroupProviderInitializationContext.class);
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAuthorityEndpoint()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getTenantId()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientId()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientSecret()));
+    }
+
+    private void setupTestingProvider() {
+        testingProvider = new AzureGraphUserGroupProvider();
+        try {
+            testingProvider.initialize(initContext);
+            testingProvider.onConfigured(authContext);
+        } catch (final Exception exc) {
+            logger.error("setup() exception: " + exc + "; tests cannot run on this system.");
+            return;
+        }
+    }
+
+
+    @After
+    public void tearDown() {
+        testingProvider.preDestruction();
+    }
+
+    @Test
+    public void testWithGroupListFilter(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+    }
+
+    @Test
+    public void testWithPaging(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.PAGE_SIZE_PROPERTY)))
+        .thenReturn(new MockPropertyValue("3")); // in the real scenario, this should be 20 or bigger.
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+        String knownGroupName = getKnownTestGroupName();
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());
+        Assert.assertTrue(search.size() > 0);
+    }
+
+    @Test
+    public void testWithGroupFilterPrefix(){
+        // make sure to set up a test group name whose name length is longer than 5
+        String knownGroupName = getKnownTestGroupName();
+        String prefix = knownGroupName.substring(0, 2);
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_PREFIX_PROPERTY)))
+        .thenReturn(new MockPropertyValue(prefix));
+
+        setupTestingProvider();
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());

Review comment:
       ```suggestion
           List<Group> search = testingProvider.getGroups().stream().filter(g -> g.getName().equals(knownGroupName)).collect(Collectors.toList());
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
##########
@@ -0,0 +1,458 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import com.google.gson.JsonObject;
+import com.microsoft.graph.core.ClientException;
+import com.microsoft.graph.models.extensions.DirectoryObject;
+import com.microsoft.graph.models.extensions.IGraphServiceClient;
+import com.microsoft.graph.options.Option;
+import com.microsoft.graph.options.QueryOption;
+import com.microsoft.graph.requests.extensions.GraphServiceClient;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesPage;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequest;
+import com.microsoft.graph.requests.extensions.IDirectoryObjectCollectionWithReferencesRequestBuilder;
+import com.microsoft.graph.requests.extensions.IGroupCollectionPage;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequest;
+import com.microsoft.graph.requests.extensions.IGroupCollectionRequestBuilder;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.User;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProvider;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The AzureGraphUserGroupProvider provides support for retrieving users and
+ * groups from Azure Activy Driectory (AAD) using graph rest-api & SDK.
+ */
+public class AzureGraphUserGroupProvider implements UserGroupProvider {
+    private final static Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProvider.class);
+
+    private int pageSize;
+    private String claimForUserName;
+
+    private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(2);
+
+    public static final String REFRESH_DELAY_PROPERTY = "REFRESH_DELAY";
+    private static final long MINIMUM_SYNC_INTERVAL_MILLISECONDS = 10_000;
+    public static final String AUTHORITY_ENDPOINT_PROPERTY = "AUTHORITY_ENDPOINT";
+    public static final String TENANT_ID_PROPERTY = "TENANT_ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "APP_REG_CLIENT_ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "APP_REG_CLIENT_SECRET";
+    // comma separate list of group names to search from AAD
+    public static final String GROUP_FILTER_LIST_PROPERTY = "GROUP_FILTER_LIST_INCLUSION";
+    // group filter with startswith
+    public static final String GROUP_FILTER_PREFIX_PROPERTY = "GROUP_FILTER_PREFIX";
+    // client side group filter 'endswith' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUFFIX_PROPERTY = "GROUP_FILTER_SUFFIX";
+    // client side group filter 'contains' operator, due to support limiation of
+    // azure graph rest-api
+    public static final String GROUP_FILTER_SUBSTRING_PROPERTY = "GROUP_FILTER_SUBSTRING";
+    public static final String PAGE_SIZE_PROPERTY = "PAGE_SIZE";
+    // default: upn (or userPrincipalName). possilbe choices ['upn', 'email']
+    // this should be matched with oidc configuration in nifi.properties
+    public static final String CLAIM_FOR_USERNAME = "CLAIM_FOR_USERNAME";
+    public static final String DEFAULT_AAD_AUTHORITY_ENDPOINT = "https://login.microsoftonline.com";
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private AtomicReference<ImmutableAzureGraphUserGroup> azureGraphUserGroupRef = new AtomicReference<ImmutableAzureGraphUserGroup>();
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroup(identifier);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getGroups();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUser(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserAndGroups(principalName);
+    }
+
+    @Override
+    public User getUserByIdentity(String principalName) throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUserByPrincipalName(principalName);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return azureGraphUserGroupRef.get().getUsers();
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext)
+            throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.initialize");
+        }
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+        final String value;
+
+        if (property != null && property.isSet()) {
+            value = property.getValue();
+        } else {
+            value = defaultValue;
+        }
+        return value;
+    }
+
+    private long getDelayProperty(AuthorizerConfigurationContext authContext, String propertyName,
+            String defaultValue) {
+        final String propertyValue = getProperty(authContext, propertyName, defaultValue);
+        final long syncInterval;
+        try {
+            syncInterval = Math.round(FormatUtils.getPreciseTimeDuration(propertyValue, TimeUnit.MILLISECONDS));
+        } catch (final IllegalArgumentException ignored) {
+            throw new AuthorizerCreationException(
+                    String.format("The %s '%s' is not a valid time interval.", propertyName, propertyValue));
+        }
+
+        if (syncInterval < MINIMUM_SYNC_INTERVAL_MILLISECONDS) {
+            throw new AuthorizerCreationException(String.format("The %s '%s' is below the minimum value of '%d ms'",
+                    propertyName, propertyValue, MINIMUM_SYNC_INTERVAL_MILLISECONDS));
+        }
+        return syncInterval;
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("calling AzureGraphUserGroupProvder.onConfigured");
+        }
+        long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, "5 mins");
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY,
+                DEFAULT_AAD_AUTHORITY_ENDPOINT);
+        final String tenantId = getProperty(configurationContext, TENANT_ID_PROPERTY, null);
+        final String clientId = getProperty(configurationContext, APP_REG_CLIENT_ID_PROPERTY, null);
+        final String clientSecret = getProperty(configurationContext, APP_REG_CLIENT_SECRET_PROPERTY, null);
+        this.pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, "50"));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, "upn");
+
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", TENANT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(
+                    String.format("%s is a required field for AzureGraphUserGroupProvder", APP_REG_CLIENT_ID_PROPERTY));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for AzureGraphUserGroupProvder",
+                    APP_REG_CLIENT_SECRET_PROPERTY));
+        }
+        if (this.pageSize > 999) {
+            throw new AuthorizerCreationException("Max page size for graph rest api call is 999.");
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (ClientException ep) {
+            throw new AuthorizerCreationException("Failed to create a GraphServiceClient", ep);
+        }
+
+        // first, load list of group name if there is any prefix, suffix, substring
+        // filter defined, paging thru groups.
+        // then, add additonal group list if there is group list inclusion defined.
+        final String prefix = getProperty(configurationContext, GROUP_FILTER_PREFIX_PROPERTY, null);
+        final String suffix = getProperty(configurationContext, GROUP_FILTER_SUFFIX_PROPERTY, null);
+        final String substring = getProperty(configurationContext, GROUP_FILTER_SUBSTRING_PROPERTY, null);
+        final String groupFilterList = getProperty(configurationContext, GROUP_FILTER_LIST_PROPERTY, null);
+
+        // if no group filter is specified, generate exception since we don't want to
+        // load whole groups from AAD.
+        if (StringUtils.isBlank(prefix) && StringUtils.isBlank(suffix) && StringUtils.isBlank(substring)
+                && StringUtils.isBlank(groupFilterList)) {
+            throw new AuthorizerCreationException("At least one GROUP_FILTER should be specified");
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (IOException | ClientException ep) {
+            throw new AuthorizerCreationException("Failed to load user/group from Azure AD", ep);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                logger.info("scheduling refreshUserGroupData()");
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Exception while refreshUserGroupData", t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = getGroupsWith(groupFilterList, prefix, suffix, substring, pageSize);
+        refreshUserGroupData(groupDisplayNames);
+        stopWatch.stop();
+        if (logger.isDebugEnabled()) {
+            logger.debug("Refreshed {} user groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+    }
+
+    private Set<String> getGroupsWith(String groupFilterList, String prefix, String suffix, String substring, int pageSize) {
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = new HashSet<>();
+
+        if (!StringUtils.isBlank(prefix) || !StringUtils.isBlank(suffix) || !StringUtils.isBlank(substring)) {
+            groupDisplayNames.addAll(queryGroupsWith(prefix, suffix, substring, pageSize));
+        }
+
+        if (!StringUtils.isBlank(groupFilterList)) {
+            groupDisplayNames.addAll(
+                Arrays.stream(groupFilterList.split(","))
+                    .map(String::trim)
+                    .collect(Collectors.toList())
+            );
+        }
+
+        stopWatch.stop();
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Fetched {} groups in {}", groupDisplayNames.size(), stopWatch.getDuration());
+        }
+
+        return Collections.unmodifiableSet(groupDisplayNames);
+    }
+
+    /**
+     * Get a set of group display names after filtering prefix, suffix, and substring
+     * @param prefix prefix filter string matching against displayName of group directory objects
+     * @param suffix suffix fitler string matching against displayName of group directory objects
+     * @param substring string matching against displayName of group directory objects
+     * @param pageSize page size to make graph rest calls in pagination
+     * @return set of group display names
+     */
+    private Set<String> queryGroupsWith(String prefix, String suffix, String substring, int pageSize) {
+        final Set<String> groups = new HashSet<>();
+        boolean filterEvaluation = false;
+        IGroupCollectionRequest gRequest;
+        IGroupCollectionPage filterResults;
+        if (prefix != null && !prefix.isEmpty()) {
+            // build a $filter query option and create a graph request if prefix is given
+            final List<Option> requestOptions = new LinkedList<Option>();
+            requestOptions.add(new QueryOption("$filter", String.format("startswith(displayName, '%s')", prefix)));
+            gRequest = graphClient.groups().buildRequest(requestOptions).select("displayName");
+        } else {
+            // default group graph request
+            gRequest = graphClient.groups().buildRequest().select("displayName");
+        }
+        if (pageSize > 0) {
+            gRequest = gRequest.top(pageSize);
+        }
+        filterResults = gRequest.get();
+
+        List<com.microsoft.graph.models.extensions.Group> currentPage = filterResults.getCurrentPage();
+        while (currentPage != null) {
+            for (com.microsoft.graph.models.extensions.Group grp : currentPage) {
+                filterEvaluation = true;
+                if (!StringUtils.isEmpty(suffix) && !grp.displayName.endsWith(suffix)) {
+                    filterEvaluation = false;
+                }
+                if (!StringUtils.isEmpty(substring) && !grp.displayName.contains(substring)) {
+                    filterEvaluation = false;
+                }
+                if (filterEvaluation) {
+                    groups.add(grp.displayName);
+                }
+            }
+            IGroupCollectionRequestBuilder gBuilder = filterResults.getNextPage();
+            if (gBuilder != null) {
+                filterResults = gBuilder.buildRequest().get();
+                currentPage = filterResults.getCurrentPage();
+            } else {
+                currentPage = null;
+            }
+        }
+
+        return Collections.unmodifiableSet(groups);
+    }
+
+    /**
+     * Get member users of the given group name
+     * @param groupName group name to search for member users
+     * @return UserGroupQueryResult
+     */
+    private UserGroupQueryResult getUsersFrom(String groupName) throws IOException, ClientException {
+        final Set<User> users = new HashSet<>();
+
+        final List<Option> requestOptions = Arrays.asList(new QueryOption("$filter", String.format("displayName eq '%s'", groupName)));
+        final IGroupCollectionPage results = graphClient.groups().buildRequest(requestOptions).get();
+        final List<com.microsoft.graph.models.extensions.Group> currentPage = results.getCurrentPage();
+
+        if (currentPage != null && currentPage.size() > 0) {
+            final com.microsoft.graph.models.extensions.Group graphGroup = results.getCurrentPage().get(0);
+            final Group.Builder groupBuilder =
+                new Group.Builder()
+                    .identifier(graphGroup.id)
+                    .name(graphGroup.displayName);
+
+            IDirectoryObjectCollectionWithReferencesRequest uRequest =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName");
+
+            if (pageSize > 0) {
+                uRequest = uRequest.top(pageSize);
+            }
+            IDirectoryObjectCollectionWithReferencesPage userpage =
+                graphClient.groups(graphGroup.id)
+                    .members()
+                    .buildRequest()
+                    .select("id, displayName, mail, userPrincipalName").get();
+
+            while (userpage.getCurrentPage() != null) {
+                for (DirectoryObject userDO : userpage.getCurrentPage()) {
+                    JsonObject jsonUser = userDO.getRawObject();
+                    final String idUser = jsonUser.get("id").getAsString();
+                    // upn is default fallback claim for userName
+                    // upn claim maps to 'mail' property in Azure graph rest-api.
+                    final String userName;
+                    if (claimForUserName.equals("email")) {
+                        // authentication token contains email field, while graph api returns mail property
+                        if (!jsonUser.get("mail").isJsonNull()) {
+                            userName = jsonUser.get("mail").getAsString();
+                        } else {
+                            userName = jsonUser.get("userPrincipalName").getAsString();
+                        }
+                    } else {
+                        userName = jsonUser.get("userPrincipalName").getAsString();
+                    }
+                    final User user = new User.Builder().identifier(idUser).identity(userName).build();
+                    users.add(user);
+                    groupBuilder.addUser(idUser);
+                }
+                IDirectoryObjectCollectionWithReferencesRequestBuilder nextPageRequest = userpage.getNextPage();
+
+                if (nextPageRequest != null) {
+                    userpage = nextPageRequest.buildRequest().get();
+                } else {
+                    break;
+                }
+            }
+            final Group group = groupBuilder.build();
+            return new UserGroupQueryResult(group, users);
+        } else if (logger.isDebugEnabled()) {
+            logger.debug("Group collection page for {} was null or empty", groupName);
+        }
+        return null;
+    }
+
+    /**
+     * refresh the user & group data for UserGroupProvider plugin service
+     * @param groupDisplayNames a list of group display names
+     */
+    private void refreshUserGroupData(Set<String> groupDisplayNames) throws IOException, ClientException {
+        Objects.requireNonNull(groupDisplayNames);
+
+        final long startTime = System.currentTimeMillis();
+        final Set<User> _users = new HashSet<>();
+        final Set<Group> _groups = new HashSet<>();
+
+        for (String grpFilter : groupDisplayNames) {
+            if (logger.isDebugEnabled()) logger.debug("Getting users for group filter: {}", grpFilter);
+            UserGroupQueryResult queryResult = getUsersFrom(grpFilter);
+            if (queryResult != null) {
+                if (logger.isDebugEnabled()) {
+                    logger.debug("Found {} users for group: {}", queryResult.getUsers().size(), queryResult.getGroup().getName());
+                }
+                _groups.add(queryResult.getGroup());
+                _users.addAll(queryResult.getUsers());
+            } else {
+                if (logger.isDebugEnabled()) logger.debug("Query result was null");
+            }
+        }
+        final ImmutableAzureGraphUserGroup azureGraphUserGroup =
+            ImmutableAzureGraphUserGroup.newInstance(_users, _groups);
+        azureGraphUserGroupRef.set(azureGraphUserGroup);
+        final long endTime = System.currentTimeMillis();
+        if (logger.isDebugEnabled()) logger.debug("Refreshed users and groups, took {} miliseconds", (endTime - startTime));
+    }
+
+    @Override
+    public void preDestruction() throws AuthorizerDestructionException {
+        try {
+            scheduler.shutdownNow();
+        } catch (final Exception e) {
+            logger.warn("Error shutting down refresh scheduler: " + e.getMessage(), e);
+        }

Review comment:
       ```suggestion
           scheduler.shutdown();
           try {
               if (!scheduler.awaitTermination(10000, TimeUnit.MILLISECONDS)) {
                   scheduler.shutdownNow();
               }
           } catch (final InterruptedException e) {
               logger.warn("Error shutting down user group refresh scheduler due to {}", e.getMessage(), e);
           }
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/test/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProviderIT.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import static org.junit.Assert.fail;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.util.MockPropertyValue;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AzureGraphUserGroupProviderIT {
+    private static final Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProviderIT.class);
+
+    private static final Properties CONFIG;
+
+    private static final String CREDENTIALS_FILE = System.getProperty("user.home") + "/azure-aad-app-reg.PROPERTIES";
+
+    static {
+        CONFIG = new Properties();
+        try {
+            final FileInputStream fis = new FileInputStream(CREDENTIALS_FILE);
+            try {
+                CONFIG.load(fis);
+            } catch (IOException e) {
+                fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+            } finally {
+                FileUtils.closeQuietly(fis);
+            }
+        } catch (FileNotFoundException e) {
+            fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+        }
+    }
+
+    protected static String getAuthorityEndpoint() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY);
+    }
+
+    protected static String getTenantId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientSecret() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY);
+    }
+
+    protected static String getKnownTestUserName() {
+        return CONFIG.getProperty("KNOWN_TEST_USER");
+
+    }
+
+    protected static String getKnownTestGroupName() {
+        return CONFIG.getProperty("KNOWN_TEST_GROUP");
+    }
+
+    protected static String getGroupListInclusion() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY);
+    }
+
+    private AuthorizerConfigurationContext authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+    private AzureGraphUserGroupProvider testingProvider;
+    private UserGroupProviderInitializationContext initContext;
+
+    @Before
+    public void setup() throws IOException {
+        authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+        initContext = Mockito.mock(UserGroupProviderInitializationContext.class);
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAuthorityEndpoint()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getTenantId()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientId()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientSecret()));
+    }
+
+    private void setupTestingProvider() {
+        testingProvider = new AzureGraphUserGroupProvider();
+        try {
+            testingProvider.initialize(initContext);
+            testingProvider.onConfigured(authContext);
+        } catch (final Exception exc) {
+            logger.error("setup() exception: " + exc + "; tests cannot run on this system.");
+            return;
+        }
+    }
+
+
+    @After
+    public void tearDown() {
+        testingProvider.preDestruction();
+    }
+
+    @Test
+    public void testWithGroupListFilter(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);

Review comment:
       ```suggestion
           Assert.assertNotNull(uag.getUser());
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/test/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProviderIT.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import static org.junit.Assert.fail;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.util.MockPropertyValue;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AzureGraphUserGroupProviderIT {
+    private static final Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProviderIT.class);
+
+    private static final Properties CONFIG;
+
+    private static final String CREDENTIALS_FILE = System.getProperty("user.home") + "/azure-aad-app-reg.PROPERTIES";
+
+    static {
+        CONFIG = new Properties();
+        try {
+            final FileInputStream fis = new FileInputStream(CREDENTIALS_FILE);
+            try {
+                CONFIG.load(fis);
+            } catch (IOException e) {
+                fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+            } finally {
+                FileUtils.closeQuietly(fis);
+            }
+        } catch (FileNotFoundException e) {
+            fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+        }
+    }
+
+    protected static String getAuthorityEndpoint() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY);
+    }
+
+    protected static String getTenantId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientSecret() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY);
+    }
+
+    protected static String getKnownTestUserName() {
+        return CONFIG.getProperty("KNOWN_TEST_USER");
+
+    }
+
+    protected static String getKnownTestGroupName() {
+        return CONFIG.getProperty("KNOWN_TEST_GROUP");
+    }
+
+    protected static String getGroupListInclusion() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY);
+    }
+
+    private AuthorizerConfigurationContext authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+    private AzureGraphUserGroupProvider testingProvider;
+    private UserGroupProviderInitializationContext initContext;
+
+    @Before
+    public void setup() throws IOException {
+        authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+        initContext = Mockito.mock(UserGroupProviderInitializationContext.class);
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAuthorityEndpoint()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getTenantId()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientId()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientSecret()));
+    }
+
+    private void setupTestingProvider() {
+        testingProvider = new AzureGraphUserGroupProvider();
+        try {
+            testingProvider.initialize(initContext);
+            testingProvider.onConfigured(authContext);
+        } catch (final Exception exc) {
+            logger.error("setup() exception: " + exc + "; tests cannot run on this system.");
+            return;
+        }
+    }
+
+
+    @After
+    public void tearDown() {
+        testingProvider.preDestruction();
+    }
+
+    @Test
+    public void testWithGroupListFilter(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+    }
+
+    @Test
+    public void testWithPaging(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.PAGE_SIZE_PROPERTY)))
+        .thenReturn(new MockPropertyValue("3")); // in the real scenario, this should be 20 or bigger.
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+        String knownGroupName = getKnownTestGroupName();
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());
+        Assert.assertTrue(search.size() > 0);
+    }
+
+    @Test
+    public void testWithGroupFilterPrefix(){
+        // make sure to set up a test group name whose name length is longer than 5
+        String knownGroupName = getKnownTestGroupName();
+        String prefix = knownGroupName.substring(0, 2);
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_PREFIX_PROPERTY)))
+        .thenReturn(new MockPropertyValue(prefix));
+
+        setupTestingProvider();
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());
+        Assert.assertTrue(search.size() > 0);
+    }
+
+    @Test
+    public void testWithGroupFilterSuffix(){
+        // make sure to set up a test group name whose name length is longer than 5
+        String knownGroupName = getKnownTestGroupName();
+        String suffix = knownGroupName.substring(knownGroupName.length()-2);
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_SUFFIX_PROPERTY)))
+        .thenReturn(new MockPropertyValue(suffix));
+
+        setupTestingProvider();
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());

Review comment:
       ```suggestion
           List<Group> search = testingProvider.getGroups().stream().filter(g -> g.getName().equals(knownGroupName)).collect(Collectors.toList());
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/test/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProviderIT.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import static org.junit.Assert.fail;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.util.MockPropertyValue;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AzureGraphUserGroupProviderIT {
+    private static final Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProviderIT.class);
+
+    private static final Properties CONFIG;
+
+    private static final String CREDENTIALS_FILE = System.getProperty("user.home") + "/azure-aad-app-reg.PROPERTIES";
+
+    static {
+        CONFIG = new Properties();
+        try {
+            final FileInputStream fis = new FileInputStream(CREDENTIALS_FILE);
+            try {
+                CONFIG.load(fis);
+            } catch (IOException e) {
+                fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+            } finally {
+                FileUtils.closeQuietly(fis);
+            }
+        } catch (FileNotFoundException e) {
+            fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+        }
+    }
+
+    protected static String getAuthorityEndpoint() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY);
+    }
+
+    protected static String getTenantId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientSecret() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY);
+    }
+
+    protected static String getKnownTestUserName() {
+        return CONFIG.getProperty("KNOWN_TEST_USER");
+
+    }
+
+    protected static String getKnownTestGroupName() {
+        return CONFIG.getProperty("KNOWN_TEST_GROUP");
+    }
+
+    protected static String getGroupListInclusion() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY);
+    }
+
+    private AuthorizerConfigurationContext authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+    private AzureGraphUserGroupProvider testingProvider;
+    private UserGroupProviderInitializationContext initContext;
+
+    @Before
+    public void setup() throws IOException {
+        authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+        initContext = Mockito.mock(UserGroupProviderInitializationContext.class);
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAuthorityEndpoint()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getTenantId()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientId()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientSecret()));
+    }
+
+    private void setupTestingProvider() {
+        testingProvider = new AzureGraphUserGroupProvider();
+        try {
+            testingProvider.initialize(initContext);
+            testingProvider.onConfigured(authContext);
+        } catch (final Exception exc) {
+            logger.error("setup() exception: " + exc + "; tests cannot run on this system.");
+            return;
+        }
+    }
+
+
+    @After
+    public void tearDown() {
+        testingProvider.preDestruction();
+    }
+
+    @Test
+    public void testWithGroupListFilter(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+    }
+
+    @Test
+    public void testWithPaging(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.PAGE_SIZE_PROPERTY)))
+        .thenReturn(new MockPropertyValue("3")); // in the real scenario, this should be 20 or bigger.
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+        String knownGroupName = getKnownTestGroupName();
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());
+        Assert.assertTrue(search.size() > 0);
+    }
+
+    @Test
+    public void testWithGroupFilterPrefix(){
+        // make sure to set up a test group name whose name length is longer than 5
+        String knownGroupName = getKnownTestGroupName();
+        String prefix = knownGroupName.substring(0, 2);
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_PREFIX_PROPERTY)))
+        .thenReturn(new MockPropertyValue(prefix));

Review comment:
       ```suggestion
               .thenReturn(new MockPropertyValue(prefix));
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/test/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProviderIT.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import static org.junit.Assert.fail;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.util.MockPropertyValue;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AzureGraphUserGroupProviderIT {
+    private static final Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProviderIT.class);
+
+    private static final Properties CONFIG;
+
+    private static final String CREDENTIALS_FILE = System.getProperty("user.home") + "/azure-aad-app-reg.PROPERTIES";
+
+    static {
+        CONFIG = new Properties();
+        try {
+            final FileInputStream fis = new FileInputStream(CREDENTIALS_FILE);
+            try {
+                CONFIG.load(fis);
+            } catch (IOException e) {
+                fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+            } finally {
+                FileUtils.closeQuietly(fis);
+            }
+        } catch (FileNotFoundException e) {
+            fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+        }
+    }
+
+    protected static String getAuthorityEndpoint() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY);
+    }
+
+    protected static String getTenantId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientSecret() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY);
+    }
+
+    protected static String getKnownTestUserName() {
+        return CONFIG.getProperty("KNOWN_TEST_USER");
+
+    }
+
+    protected static String getKnownTestGroupName() {
+        return CONFIG.getProperty("KNOWN_TEST_GROUP");
+    }
+
+    protected static String getGroupListInclusion() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY);
+    }
+
+    private AuthorizerConfigurationContext authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+    private AzureGraphUserGroupProvider testingProvider;
+    private UserGroupProviderInitializationContext initContext;
+
+    @Before
+    public void setup() throws IOException {
+        authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+        initContext = Mockito.mock(UserGroupProviderInitializationContext.class);
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAuthorityEndpoint()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getTenantId()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientId()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientSecret()));
+    }
+
+    private void setupTestingProvider() {
+        testingProvider = new AzureGraphUserGroupProvider();
+        try {
+            testingProvider.initialize(initContext);
+            testingProvider.onConfigured(authContext);
+        } catch (final Exception exc) {
+            logger.error("setup() exception: " + exc + "; tests cannot run on this system.");
+            return;
+        }
+    }
+
+
+    @After
+    public void tearDown() {
+        testingProvider.preDestruction();
+    }
+
+    @Test
+    public void testWithGroupListFilter(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+    }
+
+    @Test
+    public void testWithPaging(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.PAGE_SIZE_PROPERTY)))
+        .thenReturn(new MockPropertyValue("3")); // in the real scenario, this should be 20 or bigger.
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+        String knownGroupName = getKnownTestGroupName();
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());
+        Assert.assertTrue(search.size() > 0);
+    }
+
+    @Test
+    public void testWithGroupFilterPrefix(){
+        // make sure to set up a test group name whose name length is longer than 5
+        String knownGroupName = getKnownTestGroupName();
+        String prefix = knownGroupName.substring(0, 2);
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_PREFIX_PROPERTY)))
+        .thenReturn(new MockPropertyValue(prefix));
+
+        setupTestingProvider();
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());
+        Assert.assertTrue(search.size() > 0);
+    }
+
+    @Test
+    public void testWithGroupFilterSuffix(){
+        // make sure to set up a test group name whose name length is longer than 5
+        String knownGroupName = getKnownTestGroupName();
+        String suffix = knownGroupName.substring(knownGroupName.length()-2);
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_SUFFIX_PROPERTY)))
+        .thenReturn(new MockPropertyValue(suffix));
+
+        setupTestingProvider();
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());
+        Assert.assertTrue(search.size() > 0);
+    }
+
+    @Test
+    public void testWithGroupFilterSubstring(){
+        // make sure to set up a test group name whose name length is longer than 5
+        String knownGroupName = getKnownTestGroupName();
+        String substring = knownGroupName.substring(1, knownGroupName.length()-1);
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_SUBSTRING_PROPERTY)))
+        .thenReturn(new MockPropertyValue(substring));

Review comment:
       ```suggestion
               .thenReturn(new MockPropertyValue(substring));
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/test/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProviderIT.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import static org.junit.Assert.fail;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.util.MockPropertyValue;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AzureGraphUserGroupProviderIT {
+    private static final Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProviderIT.class);
+
+    private static final Properties CONFIG;
+
+    private static final String CREDENTIALS_FILE = System.getProperty("user.home") + "/azure-aad-app-reg.PROPERTIES";
+
+    static {
+        CONFIG = new Properties();
+        try {
+            final FileInputStream fis = new FileInputStream(CREDENTIALS_FILE);
+            try {
+                CONFIG.load(fis);
+            } catch (IOException e) {
+                fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+            } finally {
+                FileUtils.closeQuietly(fis);
+            }
+        } catch (FileNotFoundException e) {
+            fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+        }
+    }
+
+    protected static String getAuthorityEndpoint() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY);
+    }
+
+    protected static String getTenantId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientSecret() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY);
+    }
+
+    protected static String getKnownTestUserName() {
+        return CONFIG.getProperty("KNOWN_TEST_USER");
+
+    }
+
+    protected static String getKnownTestGroupName() {
+        return CONFIG.getProperty("KNOWN_TEST_GROUP");
+    }
+
+    protected static String getGroupListInclusion() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY);
+    }
+
+    private AuthorizerConfigurationContext authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+    private AzureGraphUserGroupProvider testingProvider;
+    private UserGroupProviderInitializationContext initContext;
+
+    @Before
+    public void setup() throws IOException {
+        authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+        initContext = Mockito.mock(UserGroupProviderInitializationContext.class);
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAuthorityEndpoint()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getTenantId()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientId()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientSecret()));
+    }
+
+    private void setupTestingProvider() {
+        testingProvider = new AzureGraphUserGroupProvider();
+        try {
+            testingProvider.initialize(initContext);
+            testingProvider.onConfigured(authContext);
+        } catch (final Exception exc) {
+            logger.error("setup() exception: " + exc + "; tests cannot run on this system.");
+            return;
+        }
+    }
+
+
+    @After
+    public void tearDown() {
+        testingProvider.preDestruction();
+    }
+
+    @Test
+    public void testWithGroupListFilter(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+    }
+
+    @Test
+    public void testWithPaging(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.PAGE_SIZE_PROPERTY)))
+        .thenReturn(new MockPropertyValue("3")); // in the real scenario, this should be 20 or bigger.
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+        String knownGroupName = getKnownTestGroupName();
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());
+        Assert.assertTrue(search.size() > 0);
+    }
+
+    @Test
+    public void testWithGroupFilterPrefix(){
+        // make sure to set up a test group name whose name length is longer than 5
+        String knownGroupName = getKnownTestGroupName();
+        String prefix = knownGroupName.substring(0, 2);
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_PREFIX_PROPERTY)))
+        .thenReturn(new MockPropertyValue(prefix));
+
+        setupTestingProvider();
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());
+        Assert.assertTrue(search.size() > 0);
+    }
+
+    @Test
+    public void testWithGroupFilterSuffix(){
+        // make sure to set up a test group name whose name length is longer than 5
+        String knownGroupName = getKnownTestGroupName();
+        String suffix = knownGroupName.substring(knownGroupName.length()-2);
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_SUFFIX_PROPERTY)))
+        .thenReturn(new MockPropertyValue(suffix));
+
+        setupTestingProvider();
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());
+        Assert.assertTrue(search.size() > 0);
+    }
+
+    @Test
+    public void testWithGroupFilterSubstring(){
+        // make sure to set up a test group name whose name length is longer than 5
+        String knownGroupName = getKnownTestGroupName();
+        String substring = knownGroupName.substring(1, knownGroupName.length()-1);
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_SUBSTRING_PROPERTY)))
+        .thenReturn(new MockPropertyValue(substring));
+
+        setupTestingProvider();
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());
+        Assert.assertTrue(search.size() > 0);
+    }
+
+    @Test
+    public void testWithGroupFilterOperatorAndListInclusion(){
+        // make sure to set up a test group name whose name length is longer than 5
+        String knownGroupName = getKnownTestGroupName();
+        String substring = knownGroupName.substring(1, knownGroupName.length()-1);
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_SUBSTRING_PROPERTY)))
+        .thenReturn(new MockPropertyValue(substring));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+        .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+
+        setupTestingProvider();
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Set<Group> search = testingProvider.getGroups().stream().collect(Collectors.toSet());
+        // check there is no duplicate group
+        Assert.assertTrue(search.size() == testingProvider.getGroups().size());

Review comment:
       ```suggestion
           Assert.assertEquals(search.size(), testingProvider.getGroups().size());
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/test/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProviderIT.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import static org.junit.Assert.fail;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.util.MockPropertyValue;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AzureGraphUserGroupProviderIT {
+    private static final Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProviderIT.class);
+
+    private static final Properties CONFIG;
+
+    private static final String CREDENTIALS_FILE = System.getProperty("user.home") + "/azure-aad-app-reg.PROPERTIES";
+
+    static {
+        CONFIG = new Properties();
+        try {
+            final FileInputStream fis = new FileInputStream(CREDENTIALS_FILE);
+            try {
+                CONFIG.load(fis);
+            } catch (IOException e) {
+                fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+            } finally {
+                FileUtils.closeQuietly(fis);
+            }
+        } catch (FileNotFoundException e) {
+            fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+        }
+    }
+
+    protected static String getAuthorityEndpoint() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY);
+    }
+
+    protected static String getTenantId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientSecret() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY);
+    }
+
+    protected static String getKnownTestUserName() {
+        return CONFIG.getProperty("KNOWN_TEST_USER");
+
+    }
+
+    protected static String getKnownTestGroupName() {
+        return CONFIG.getProperty("KNOWN_TEST_GROUP");
+    }
+
+    protected static String getGroupListInclusion() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY);
+    }
+
+    private AuthorizerConfigurationContext authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+    private AzureGraphUserGroupProvider testingProvider;
+    private UserGroupProviderInitializationContext initContext;
+
+    @Before
+    public void setup() throws IOException {
+        authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+        initContext = Mockito.mock(UserGroupProviderInitializationContext.class);
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAuthorityEndpoint()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getTenantId()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientId()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientSecret()));
+    }
+
+    private void setupTestingProvider() {
+        testingProvider = new AzureGraphUserGroupProvider();
+        try {
+            testingProvider.initialize(initContext);
+            testingProvider.onConfigured(authContext);
+        } catch (final Exception exc) {
+            logger.error("setup() exception: " + exc + "; tests cannot run on this system.");
+            return;
+        }
+    }
+
+
+    @After
+    public void tearDown() {
+        testingProvider.preDestruction();
+    }
+
+    @Test
+    public void testWithGroupListFilter(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+    }
+
+    @Test
+    public void testWithPaging(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.PAGE_SIZE_PROPERTY)))
+        .thenReturn(new MockPropertyValue("3")); // in the real scenario, this should be 20 or bigger.
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+        String knownGroupName = getKnownTestGroupName();
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());
+        Assert.assertTrue(search.size() > 0);
+    }
+
+    @Test
+    public void testWithGroupFilterPrefix(){
+        // make sure to set up a test group name whose name length is longer than 5
+        String knownGroupName = getKnownTestGroupName();
+        String prefix = knownGroupName.substring(0, 2);
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_PREFIX_PROPERTY)))
+        .thenReturn(new MockPropertyValue(prefix));
+
+        setupTestingProvider();
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());
+        Assert.assertTrue(search.size() > 0);
+    }
+
+    @Test
+    public void testWithGroupFilterSuffix(){
+        // make sure to set up a test group name whose name length is longer than 5
+        String knownGroupName = getKnownTestGroupName();
+        String suffix = knownGroupName.substring(knownGroupName.length()-2);
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_SUFFIX_PROPERTY)))
+        .thenReturn(new MockPropertyValue(suffix));
+
+        setupTestingProvider();
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());
+        Assert.assertTrue(search.size() > 0);
+    }
+
+    @Test
+    public void testWithGroupFilterSubstring(){
+        // make sure to set up a test group name whose name length is longer than 5
+        String knownGroupName = getKnownTestGroupName();
+        String substring = knownGroupName.substring(1, knownGroupName.length()-1);
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_SUBSTRING_PROPERTY)))
+        .thenReturn(new MockPropertyValue(substring));
+
+        setupTestingProvider();
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());
+        Assert.assertTrue(search.size() > 0);
+    }
+
+    @Test
+    public void testWithGroupFilterOperatorAndListInclusion(){
+        // make sure to set up a test group name whose name length is longer than 5
+        String knownGroupName = getKnownTestGroupName();
+        String substring = knownGroupName.substring(1, knownGroupName.length()-1);
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_SUBSTRING_PROPERTY)))
+        .thenReturn(new MockPropertyValue(substring));

Review comment:
       ```suggestion
               .thenReturn(new MockPropertyValue(substring));
   ````

##########
File path: nifi-docs/src/main/asciidoc/administration-guide.adoc
##########
@@ -494,6 +494,46 @@ The ShellUserGroupProvider has the following properties:
 
 Like LdapUserGroupProvider, the ShellUserGroupProvider is commented out in the _authorizers.xml_ file.  Refer to that comment for usage examples.
 
+==== AzureGraphUserGroupProvider
+
+The AzureGraphUserGroupProvider fetches users and group details from Azure Activy Driectory (AAD) using Azure graph rest-api & SDK.
+
+This provider first fetches the list of the groups based on group filter specification (e.g. prefix, suffix, substring of group names, or list of group names).
+The group filter conditions are evaluated against _displayName_ property of Group directory object. From the list of groups, this provider loads member users of these groups.
+
+Pre-requsites of this provider on Azure Activy Driectory (AAD) side are (link:https://docs.microsoft.com/en-us/graph/auth-v2-service[from Azure graph doc^]):
+
+* Register your NiFi Application to AAD
+* Configure application type permissions for Microsoft Graph on your app (Group.Read.All,User.Read.All)
+* Grant admin consent for the configured permissions
+* Create a client secret for your application
+* Enable 'upn' optional claim for ID token
+
+Reference the following sample and example for detail instructions to configure pre-requsites:
+
+* link:https://docs.microsoft.com/en-us/azure/active-directory/develop/scenario-daemon-app-registration[sample^]
+* link:https://github.com/Azure-Samples/ms-identity-java-daemon[another-example^]
+
+To enable `upn` optional claim for ID token, follow the instructions in link:https://docs.microsoft.com/en-us/azure/active-directory/develop/active-directory-optional-claims#configuring-optional-claims[configuring-optional-claims^],
+disable 'Replace has marks' option, and set `nifi.security.user.oidc.claim.identifying.user=upn` in _nifi.properties_ file.
+
+The AzureGraphUserGroupProvider has the following properties:
+
+[options="header,footer"]
+|==================================================================================================================================================
+| Property Name | Description
+|`REFRESH_DELAY` | Duration of delay between each user and group refresh. Default is `5 mins`.
+|`AUTHORITY_ENDPOINT` | The endpoint of your AAD login. This can be found in your Azure AD application registration -> overview -> Endpoint. For example, global authority endpoint is https://login.microsoftonline.com, and US Gov cloud authority endpoint is https://login.microsoftonline.us.

Review comment:
       ```suggestion
   |`AUTHORITY_ENDPOINT` | The Azure AD login authority endpoint. This can be found in the Azure portal under Azure Active Directory -> App registrations -> Overview -> Endpoint. For example, the global authority endpoint is https://login.microsoftonline.com, and US Gov cloud authority endpoint is https://login.microsoftonline.us.
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/test/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProviderIT.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import static org.junit.Assert.fail;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.util.MockPropertyValue;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AzureGraphUserGroupProviderIT {
+    private static final Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProviderIT.class);
+
+    private static final Properties CONFIG;
+
+    private static final String CREDENTIALS_FILE = System.getProperty("user.home") + "/azure-aad-app-reg.PROPERTIES";
+
+    static {
+        CONFIG = new Properties();
+        try {
+            final FileInputStream fis = new FileInputStream(CREDENTIALS_FILE);
+            try {
+                CONFIG.load(fis);
+            } catch (IOException e) {
+                fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+            } finally {
+                FileUtils.closeQuietly(fis);
+            }
+        } catch (FileNotFoundException e) {
+            fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+        }
+    }
+
+    protected static String getAuthorityEndpoint() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY);
+    }
+
+    protected static String getTenantId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientSecret() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY);
+    }
+
+    protected static String getKnownTestUserName() {
+        return CONFIG.getProperty("KNOWN_TEST_USER");
+
+    }
+
+    protected static String getKnownTestGroupName() {
+        return CONFIG.getProperty("KNOWN_TEST_GROUP");
+    }
+
+    protected static String getGroupListInclusion() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY);
+    }
+
+    private AuthorizerConfigurationContext authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+    private AzureGraphUserGroupProvider testingProvider;
+    private UserGroupProviderInitializationContext initContext;
+
+    @Before
+    public void setup() throws IOException {
+        authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+        initContext = Mockito.mock(UserGroupProviderInitializationContext.class);
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAuthorityEndpoint()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getTenantId()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientId()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientSecret()));
+    }
+
+    private void setupTestingProvider() {
+        testingProvider = new AzureGraphUserGroupProvider();
+        try {
+            testingProvider.initialize(initContext);
+            testingProvider.onConfigured(authContext);
+        } catch (final Exception exc) {
+            logger.error("setup() exception: " + exc + "; tests cannot run on this system.");
+            return;
+        }
+    }
+
+
+    @After
+    public void tearDown() {
+        testingProvider.preDestruction();
+    }
+
+    @Test
+    public void testWithGroupListFilter(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+

Review comment:
       ```suggestion
   ```

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/test/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProviderIT.java
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.nifi.authorization.azure;
+
+import static org.junit.Assert.fail;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.nifi.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.authorization.Group;
+import org.apache.nifi.authorization.UserAndGroups;
+import org.apache.nifi.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.util.MockPropertyValue;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AzureGraphUserGroupProviderIT {
+    private static final Logger logger = LoggerFactory.getLogger(AzureGraphUserGroupProviderIT.class);
+
+    private static final Properties CONFIG;
+
+    private static final String CREDENTIALS_FILE = System.getProperty("user.home") + "/azure-aad-app-reg.PROPERTIES";
+
+    static {
+        CONFIG = new Properties();
+        try {
+            final FileInputStream fis = new FileInputStream(CREDENTIALS_FILE);
+            try {
+                CONFIG.load(fis);
+            } catch (IOException e) {
+                fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+            } finally {
+                FileUtils.closeQuietly(fis);
+            }
+        } catch (FileNotFoundException e) {
+            fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e.getLocalizedMessage());
+        }
+    }
+
+    protected static String getAuthorityEndpoint() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY);
+    }
+
+    protected static String getTenantId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientId() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY);
+    }
+
+    protected static String getAppRegClientSecret() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY);
+    }
+
+    protected static String getKnownTestUserName() {
+        return CONFIG.getProperty("KNOWN_TEST_USER");
+
+    }
+
+    protected static String getKnownTestGroupName() {
+        return CONFIG.getProperty("KNOWN_TEST_GROUP");
+    }
+
+    protected static String getGroupListInclusion() {
+        return CONFIG.getProperty(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY);
+    }
+
+    private AuthorizerConfigurationContext authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+    private AzureGraphUserGroupProvider testingProvider;
+    private UserGroupProviderInitializationContext initContext;
+
+    @Before
+    public void setup() throws IOException {
+        authContext = Mockito.mock(AuthorizerConfigurationContext.class);
+        initContext = Mockito.mock(UserGroupProviderInitializationContext.class);
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.AUTHORITY_ENDPOINT_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAuthorityEndpoint()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.TENANT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getTenantId()));
+
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_ID_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientId()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.APP_REG_CLIENT_SECRET_PROPERTY)))
+            .thenReturn(new MockPropertyValue(AzureGraphUserGroupProviderIT.getAppRegClientSecret()));
+    }
+
+    private void setupTestingProvider() {
+        testingProvider = new AzureGraphUserGroupProvider();
+        try {
+            testingProvider.initialize(initContext);
+            testingProvider.onConfigured(authContext);
+        } catch (final Exception exc) {
+            logger.error("setup() exception: " + exc + "; tests cannot run on this system.");
+            return;
+        }
+    }
+
+
+    @After
+    public void tearDown() {
+        testingProvider.preDestruction();
+    }
+
+    @Test
+    public void testWithGroupListFilter(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+    }
+
+    @Test
+    public void testWithPaging(){
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_LIST_PROPERTY)))
+            .thenReturn(new MockPropertyValue(getGroupListInclusion()));
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.PAGE_SIZE_PROPERTY)))
+        .thenReturn(new MockPropertyValue("3")); // in the real scenario, this should be 20 or bigger.
+
+        setupTestingProvider();
+
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        Assert.assertTrue(testingProvider.getUsers().size() > 0);
+        UserAndGroups uag  = testingProvider.getUserAndGroups(getKnownTestUserName());
+        Assert.assertTrue(uag.getUser() !=null);
+        Assert.assertTrue(uag.getGroups().size() > 0);
+
+        String knownGroupName = getKnownTestGroupName();
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());
+        Assert.assertTrue(search.size() > 0);
+    }
+
+    @Test
+    public void testWithGroupFilterPrefix(){
+        // make sure to set up a test group name whose name length is longer than 5
+        String knownGroupName = getKnownTestGroupName();
+        String prefix = knownGroupName.substring(0, 2);
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_PREFIX_PROPERTY)))
+        .thenReturn(new MockPropertyValue(prefix));
+
+        setupTestingProvider();
+        Assert.assertTrue(testingProvider.getGroups().size() > 0);
+        List<Group> search = testingProvider.getGroups().stream().filter( g-> g.getName().equals(knownGroupName)).collect(Collectors.toList());
+        Assert.assertTrue(search.size() > 0);
+    }
+
+    @Test
+    public void testWithGroupFilterSuffix(){
+        // make sure to set up a test group name whose name length is longer than 5
+        String knownGroupName = getKnownTestGroupName();
+        String suffix = knownGroupName.substring(knownGroupName.length()-2);
+        Mockito.when(authContext.getProperty(Mockito.eq(AzureGraphUserGroupProvider.GROUP_FILTER_SUFFIX_PROPERTY)))
+        .thenReturn(new MockPropertyValue(suffix));

Review comment:
       ```suggestion
               .thenReturn(new MockPropertyValue(suffix));
   ```

##########
File path: nifi-docs/src/main/asciidoc/administration-guide.adoc
##########
@@ -494,6 +494,46 @@ The ShellUserGroupProvider has the following properties:
 
 Like LdapUserGroupProvider, the ShellUserGroupProvider is commented out in the _authorizers.xml_ file.  Refer to that comment for usage examples.
 
+==== AzureGraphUserGroupProvider
+
+The AzureGraphUserGroupProvider fetches users and group details from Azure Activy Driectory (AAD) using Azure graph rest-api & SDK.
+
+This provider first fetches the list of the groups based on group filter specification (e.g. prefix, suffix, substring of group names, or list of group names).
+The group filter conditions are evaluated against _displayName_ property of Group directory object. From the list of groups, this provider loads member users of these groups.
+
+Pre-requsites of this provider on Azure Activy Driectory (AAD) side are (link:https://docs.microsoft.com/en-us/graph/auth-v2-service[from Azure graph doc^]):
+
+* Register your NiFi Application to AAD
+* Configure application type permissions for Microsoft Graph on your app (Group.Read.All,User.Read.All)
+* Grant admin consent for the configured permissions
+* Create a client secret for your application
+* Enable 'upn' optional claim for ID token
+
+Reference the following sample and example for detail instructions to configure pre-requsites:
+
+* link:https://docs.microsoft.com/en-us/azure/active-directory/develop/scenario-daemon-app-registration[sample^]
+* link:https://github.com/Azure-Samples/ms-identity-java-daemon[another-example^]
+
+To enable `upn` optional claim for ID token, follow the instructions in link:https://docs.microsoft.com/en-us/azure/active-directory/develop/active-directory-optional-claims#configuring-optional-claims[configuring-optional-claims^],
+disable 'Replace has marks' option, and set `nifi.security.user.oidc.claim.identifying.user=upn` in _nifi.properties_ file.
+
+The AzureGraphUserGroupProvider has the following properties:
+
+[options="header,footer"]
+|==================================================================================================================================================
+| Property Name | Description
+|`REFRESH_DELAY` | Duration of delay between each user and group refresh. Default is `5 mins`.
+|`AUTHORITY_ENDPOINT` | The endpoint of your AAD login. This can be found in your Azure AD application registration -> overview -> Endpoint. For example, global authority endpoint is https://login.microsoftonline.com, and US Gov cloud authority endpoint is https://login.microsoftonline.us.
+|`TENANT_ID` | Tenant ID or Directory ID. This can be found in your AAD application registration -> overview -> Directory (Tenant) ID
+|`APP_REG_CLIENT_ID` | Client ID or application ID of your Azure AD application registration
+|`APP_REG_CLIENT_SECRET` | Secret you have created and  saved in Azure AD application registration ->  Certificates & secrets
+|`GROUP_FILTER_PREFIX` | Prefix will match against the displayName property of groups and retrives only those matching groups
+|`GROUP_FILTER_SUFFIX` | Suffix will match against the displayName property of groups and retrives only those matching groups
+|`GROUP_FILTER_SUBSTRING` | Substring will match against the displayName property of groups and retrives only those groups that contain the substring
+|`GROUP_FILTER_LIST_INCLUSION` | Additional comma-separated list of groups that will be searched for users. If no string-based matching filter (i.e. prefix, suffix, and substring) is specified, make sure to set this property
+|`PAGE_SIZE` | Page size used when making rest-api calls to Azure Graph. Set to 0 if you want to disable paging graph rest api calls. Default: 50, Max: 999

Review comment:
       ```suggestion
   |`PAGE_SIZE` | Page size to use with Azure Active Directory Graph API. Set to 0 to disable paging API calls. Default: 50, Max: 999
   ```

##########
File path: nifi-docs/src/main/asciidoc/administration-guide.adoc
##########
@@ -494,6 +494,46 @@ The ShellUserGroupProvider has the following properties:
 
 Like LdapUserGroupProvider, the ShellUserGroupProvider is commented out in the _authorizers.xml_ file.  Refer to that comment for usage examples.
 
+==== AzureGraphUserGroupProvider
+
+The AzureGraphUserGroupProvider fetches users and group details from Azure Activy Driectory (AAD) using Azure graph rest-api & SDK.
+
+This provider first fetches the list of the groups based on group filter specification (e.g. prefix, suffix, substring of group names, or list of group names).
+The group filter conditions are evaluated against _displayName_ property of Group directory object. From the list of groups, this provider loads member users of these groups.
+
+Pre-requsites of this provider on Azure Activy Driectory (AAD) side are (link:https://docs.microsoft.com/en-us/graph/auth-v2-service[from Azure graph doc^]):
+
+* Register your NiFi Application to AAD
+* Configure application type permissions for Microsoft Graph on your app (Group.Read.All,User.Read.All)
+* Grant admin consent for the configured permissions
+* Create a client secret for your application
+* Enable 'upn' optional claim for ID token
+
+Reference the following sample and example for detail instructions to configure pre-requsites:
+
+* link:https://docs.microsoft.com/en-us/azure/active-directory/develop/scenario-daemon-app-registration[sample^]
+* link:https://github.com/Azure-Samples/ms-identity-java-daemon[another-example^]
+
+To enable `upn` optional claim for ID token, follow the instructions in link:https://docs.microsoft.com/en-us/azure/active-directory/develop/active-directory-optional-claims#configuring-optional-claims[configuring-optional-claims^],
+disable 'Replace has marks' option, and set `nifi.security.user.oidc.claim.identifying.user=upn` in _nifi.properties_ file.
+
+The AzureGraphUserGroupProvider has the following properties:
+
+[options="header,footer"]
+|==================================================================================================================================================
+| Property Name | Description
+|`REFRESH_DELAY` | Duration of delay between each user and group refresh. Default is `5 mins`.
+|`AUTHORITY_ENDPOINT` | The endpoint of your AAD login. This can be found in your Azure AD application registration -> overview -> Endpoint. For example, global authority endpoint is https://login.microsoftonline.com, and US Gov cloud authority endpoint is https://login.microsoftonline.us.
+|`TENANT_ID` | Tenant ID or Directory ID. This can be found in your AAD application registration -> overview -> Directory (Tenant) ID
+|`APP_REG_CLIENT_ID` | Client ID or application ID of your Azure AD application registration
+|`APP_REG_CLIENT_SECRET` | Secret you have created and  saved in Azure AD application registration ->  Certificates & secrets
+|`GROUP_FILTER_PREFIX` | Prefix will match against the displayName property of groups and retrives only those matching groups
+|`GROUP_FILTER_SUFFIX` | Suffix will match against the displayName property of groups and retrives only those matching groups
+|`GROUP_FILTER_SUBSTRING` | Substring will match against the displayName property of groups and retrives only those groups that contain the substring
+|`GROUP_FILTER_LIST_INCLUSION` | Additional comma-separated list of groups that will be searched for users. If no string-based matching filter (i.e. prefix, suffix, and substring) is specified, make sure to set this property

Review comment:
       ```suggestion
   |`GROUP_FILTER_LIST_INCLUSION` | Comma-separated list of groups to be searched for users. If no string-based matching filter (i.e., prefix, suffix, and substring) is specified, make sure to set this property to avoid fetching all groups and users in the Azure AD tenant.
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorizers.xml
##########
@@ -195,6 +195,64 @@
     </userGroupProvider>
     To enable the shell-user-group-provider remove 2 lines. This is 2 of 2. -->
 
+    <!--
+        ** Pre-requsites on Azure Activy Driectory (AAD) side: As described in https://docs.microsoft.com/en-us/graph/auth-v2-service,
+                1. Register your NiFi Application to AAD
+                2. Configure application type permissions for Microsoft Graph on your app
+                    - Group.Read.All
+                    - User.Read.All
+                3. Grant admin consent for the configured permissions
+                4. Create a client secret for your application
+            Reference the following document and sample to configure pre-requsites for detail instructions.
+                - https://docs.microsoft.com/en-us/azure/active-directory/develop/scenario-daemon-app-registration
+                - https://github.com/Azure-Samples/ms-identity-java-daemon
+        The AzureGraphUserGroupProvider provides support for retrieving users and groups from Azure Activy Driectory (AAD)
+        using graph rest-api & SDK. This providers will first search and filter groups based on a prefix, a suffix, and a substring filter,
+        if any specified. Comparsion is evaluated against 'displayName' property of Group directory objects.
+        Then, it will add optional list of groups, separated by comma, into the list of group.
+        If no prefix/suffix/substring group filter is specified as a group filter,
+        it will just use the optional list of groups for loading group directory info (displayName and id).
+        Finally, it will finds all the member users of these groups. Only those users with group membership will be loaded into this provider.
+
+
+        'REFRESH_DELAY'         - (Optional) duration of delay between subsequent refreshes.  Default is '5 mins'.

Review comment:
       See previous comment about naming these like "Refresh Delay" and "Authority Endpoint".

##########
File path: nifi-docs/src/main/asciidoc/administration-guide.adoc
##########
@@ -494,6 +494,46 @@ The ShellUserGroupProvider has the following properties:
 
 Like LdapUserGroupProvider, the ShellUserGroupProvider is commented out in the _authorizers.xml_ file.  Refer to that comment for usage examples.
 
+==== AzureGraphUserGroupProvider
+
+The AzureGraphUserGroupProvider fetches users and group details from Azure Activy Driectory (AAD) using Azure graph rest-api & SDK.
+
+This provider first fetches the list of the groups based on group filter specification (e.g. prefix, suffix, substring of group names, or list of group names).
+The group filter conditions are evaluated against _displayName_ property of Group directory object. From the list of groups, this provider loads member users of these groups.

Review comment:
       Can we make this reference the specific configuration names once we settle on them? For example:
   ```suggestion
   This provider fetches a subset of users and groups based on the filters configured in _authorizers.xml_.
   It first fetches a list of groups based on the configured group filters (`Group Filter Prefix`, `Group Filter Suffix`, `Group Filter Substring`, and `Group Filter List`).
   These filter conditions are evaluated against the _displayName_ property of Azure AD Group directory object. From the list of groups, it then loads member users of these groups.
   ```




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org