You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jf...@apache.org on 2021/04/09 23:08:30 UTC

[nifi] branch main updated: NIFI-7573 Added Azure Active Directory User Group Provider

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

jfrazee pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/main by this push:
     new 365dde2  NIFI-7573 Added Azure Active Directory User Group Provider
365dde2 is described below

commit 365dde2d1151a9a3d5b0d7ef58e4e0b8726d802b
Author: sjyang18 <il...@hotmail.com>
AuthorDate: Thu Jun 25 19:44:06 2020 +0000

    NIFI-7573 Added Azure Active Directory User Group Provider
    
    This closes #4367
    
    Signed-off-by: Joey Frazee <jf...@apache.org>
---
 nifi-assembly/LICENSE                              |  23 ++
 .../src/main/asciidoc/administration-guide.adoc    |  33 ++
 .../pom.xml                                        |  47 ++-
 .../azure/AzureGraphUserGroupProvider.java         | 455 +++++++++++++++++++++
 .../azure/ClientCredentialAuthProvider.java        | 179 ++++++++
 .../azure/ImmutableAzureGraphUserGroup.java        | 158 +++++++
 ...org.apache.nifi.authorization.UserGroupProvider |  15 +
 .../azure/AzureGraphUserGroupProviderIT.java       | 227 ++++++++++
 .../nifi-azure-bundle/nifi-azure-nar/pom.xml       |   5 +
 .../src/main/resources/META-INF/LICENSE            |  47 ++-
 .../nifi-azure-processors/pom.xml                  |   2 +-
 nifi-nar-bundles/nifi-azure-bundle/pom.xml         |   2 +
 .../src/main/resources/conf/authorizers.xml        |  44 ++
 13 files changed, 1208 insertions(+), 29 deletions(-)

diff --git a/nifi-assembly/LICENSE b/nifi-assembly/LICENSE
index 3c6b18f..624948a 100644
--- a/nifi-assembly/LICENSE
+++ b/nifi-assembly/LICENSE
@@ -3075,3 +3075,26 @@ The binary distribution of this product bundles 'Enterprise Security API for Jav
     SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
     THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN
     ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+The binary distribution of this product bundles 'Microsoft Graph SDK for Java'
+which is available under an MIT license.
+
+    Copyright (c) 2018 Microsoft Graph
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to
+    deal in the Software without restriction, including without limitation the
+    rights to use, copy, modify, merge, publish, distribute, sublicense, and/or
+    sell copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+    FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+    IN THE SOFTWARE.
diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc
index 6fc66a9..9315845 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -522,6 +522,39 @@ 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 groups from Azure Active Directory (AAD) using the Microsoft Graph API.
+
+A subset of groups are fetched based on filter conditions (`Group Filter Prefix`, `Group Filter Suffix`, `Group Filter Substring`, and `Group Filter List Inclusion`) evaluated against the _displayName_ property of the Azure AD group. Member users are then loaded from these groups. At least one filter condition should be specified.
+
+This provider requires an Azure app registration with:
+
+* Microsoft Graph Group.Read.All and User.Read.All API permissions with admin consent
+* A client secret or application password
+* ID token claims for upn and/or email
+
+See link:https://docs.microsoft.com/en-us/graph/auth-v2-service[here^] and link:https://docs.microsoft.com/en-us/azure/active-directory/develop/scenario-daemon-app-registration[here^] for more information on how to create a valid app registration.
+
+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 the Azure AD login. This can be found in the Azure portal under Azure Active Directory -> App registrations -> [application name] -> Endpoints. For example, the global authority endpoint is https://login.microsoftonline.com.
+|`Directory ID` | Tenant ID or Directory ID of the Azure AD tenant. This can be found in the Azure portal under Azure Active Directory -> App registrations -> [application name] -> Directory (tenant) ID.
+|`Application ID` | Client ID or Application ID of the Azure app registration. This can be found in the Azure portal under Azure Active Directory -> App registrations -> [application name] -> Overview -> Application (client) ID.
+|`Client Secret` | A client secret from the Azure 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.
+|`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.
+|`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.
+|`Group Filter Substring` | Substring filter for Azure AD groups. Matches against the group displayName to retrieve only groups with names containing the provided substring.
+|`Group Filter List Inclusion` | Comma-separated list of Azure AD groups. If no string-based matching filter (i.e., prefix, suffix, and substring) is specified, set this property to avoid fetching all groups and users in the Azure AD tenant.
+|`Page Size` | Page size to use with the Microsoft Graph API. Set to 0 to disable paging API calls. Default: 50, Max: 999.
+|`Claim for Username` | The property of the user directory object mapped to the NiFi user name field. Default is 'upn'. 'email' is another option when `nifi.security.user.oidc.fallback.claims.identifying.user` is set to 'upn'.
+|==================================================================================================================================================
+
+Like LdapUserGroupProvider and ShellUserGroupProvider, the AzureGraphUserGroupProvider configuration is commented out in the _authorizers.xml_ file. Refer to the comment for a starter configuration.
 
 ==== Composite Implementations
 
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-nar/pom.xml b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/pom.xml
similarity index 56%
copy from nifi-nar-bundles/nifi-azure-bundle/nifi-azure-nar/pom.xml
copy to nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/pom.xml
index f944efe..a9a924e 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/pom.xml
@@ -13,39 +13,54 @@
   See the License for the specific language governing permissions and
   limitations under the License.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <parent>
-        <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-azure-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
         <version>1.14.0-SNAPSHOT</version>
     </parent>
-
-    <artifactId>nifi-azure-nar</artifactId>
-    <packaging>nar</packaging>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>nifi-azure-graph-authorizer</artifactId>
+    <packaging>jar</packaging>
     <properties>
-        <maven.javadoc.skip>true</maven.javadoc.skip>
-        <source.skip>true</source.skip>
+        <microsoft-graph.version>2.10.0</microsoft-graph.version>
+        <msal4j.version>1.6.0</msal4j.version>
     </properties>
-
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-azure-processors</artifactId>
-            <version>1.14.0-SNAPSHOT</version>
+            <artifactId>nifi-framework-api</artifactId>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-azure-reporting-task</artifactId>
+            <artifactId>nifi-mock</artifactId>
             <version>1.14.0-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-azure-services-api-nar</artifactId>
+            <artifactId>nifi-utils</artifactId>
             <version>1.14.0-SNAPSHOT</version>
-            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>com.microsoft.graph</groupId>
+            <artifactId>microsoft-graph</artifactId>
+            <version>${microsoft-graph.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.azure</groupId>
+            <artifactId>azure-identity</artifactId>
+            <version>${azure.identity.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.azure</groupId>
+                    <artifactId>azure-core</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>com.microsoft.azure</groupId>
+            <artifactId>msal4j</artifactId>
+            <version>${msal4j.version}</version>
         </dependency>
     </dependencies>
-
 </project>
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
new file mode 100644
index 0000000..fd74b4a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProvider.java
@@ -0,0 +1,455 @@
+/*
+ * 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.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+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 String claimForUserName;
+
+    private ScheduledExecutorService scheduler;
+
+    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 = "Directory ID";
+    public static final String APP_REG_CLIENT_ID_PROPERTY = "Application ID";
+    public static final String APP_REG_CLIENT_SECRET_PROPERTY = "Client Secret";
+    // comma separated 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). possible 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_REFRESH_DELAY = "5 mins";
+    public static final String DEFAULT_PAGE_SIZE = "50";
+    public static final String DEFAULT_CLAIM_FOR_USERNAME = "upn";
+    public static final int MAX_PAGE_SIZE = 999;
+    public static final String AZURE_PUBLIC_CLOUD = "https://login.microsoftonline.com/";
+    static final List<String> REST_CALL_KEYWORDS = Arrays.asList("$select", "$top", "$expand", "$search", "$filter", "$format", "$count", "$skip", "$orderby");
+
+
+    private ClientCredentialAuthProvider authProvider;
+    private IGraphServiceClient graphClient;
+    private final 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 {
+        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;
+            }
+        });
+    }
+
+    private String getProperty(AuthorizerConfigurationContext authContext, String propertyName, String defaultValue) {
+        final PropertyValue property = authContext.getProperty(propertyName);
+
+        if (property != null && property.isSet()) {
+            final String value = property.getValue();
+            if (StringUtils.isNotBlank(value)) {
+                return value;
+            }
+        }
+
+        return defaultValue;
+    }
+
+    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;
+    }
+
+    private boolean hasReservedKeyword(String prefix) {
+        return REST_CALL_KEYWORDS.contains(prefix);
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        final long fixedDelay = getDelayProperty(configurationContext, REFRESH_DELAY_PROPERTY, DEFAULT_REFRESH_DELAY);
+        final String authorityEndpoint = getProperty(configurationContext, AUTHORITY_ENDPOINT_PROPERTY, AZURE_PUBLIC_CLOUD);
+        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);
+        int pageSize = Integer.parseInt(getProperty(configurationContext, PAGE_SIZE_PROPERTY, DEFAULT_PAGE_SIZE));
+        this.claimForUserName = getProperty(configurationContext, CLAIM_FOR_USERNAME, DEFAULT_CLAIM_FOR_USERNAME);
+        final String providerClassName = getClass().getSimpleName();
+        if (StringUtils.isBlank(tenantId)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for %s", TENANT_ID_PROPERTY, providerClassName));
+        }
+        if (StringUtils.isBlank(clientId)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for %s", APP_REG_CLIENT_ID_PROPERTY, providerClassName));
+        }
+        if (StringUtils.isBlank(clientSecret)) {
+            throw new AuthorizerCreationException(String.format("%s is a required field for %s", APP_REG_CLIENT_SECRET_PROPERTY, providerClassName));
+        }
+        if (pageSize > MAX_PAGE_SIZE) {
+            throw new AuthorizerCreationException(String.format("Max page size for Microsoft Graph is %d.", MAX_PAGE_SIZE));
+        }
+
+        try {
+            authProvider = new ClientCredentialAuthProvider.Builder()
+                .authorityEndpoint(authorityEndpoint)
+                .tenantId(tenantId)
+                .clientId(clientId)
+                .clientSecret(clientSecret)
+                .build();
+            graphClient = GraphServiceClient.builder().authenticationProvider(authProvider).buildClient();
+        } catch (final ClientException e) {
+            throw new AuthorizerCreationException(String.format("Failed to create a GraphServiceClient due to %s", e.getMessage()), e);
+        }
+
+        // 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(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, providerClassName));
+        }
+        // make sure prefix shouldn't have any reserved keywords
+        if (hasReservedKeyword(prefix)) {
+            throw new AuthorizerCreationException(String.format("Prefix shouldn't have any reserved keywords ([%s])", StringUtils.join(REST_CALL_KEYWORDS, ",")));
+        }
+
+        try {
+            refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+        } catch (final IOException | ClientException e) {
+            throw new AuthorizerCreationException(String.format("Failed to load UserGroup due to %s", e.getMessage()), e);
+        }
+        scheduler.scheduleWithFixedDelay(() -> {
+            try {
+                refreshUserGroup(groupFilterList, prefix, suffix, substring, pageSize);
+            } catch (final Throwable t) {
+                logger.error("Error refreshing user groups due to {}", t.getMessage(), t);
+            }
+        }, fixedDelay, fixedDelay, TimeUnit.MILLISECONDS);
+    }
+
+    private void refreshUserGroup(String groupFilterList, String prefix, String suffix, String substring, int pageSize) throws IOException, ClientException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("Refreshing user groups");
+        }
+        final StopWatch stopWatch = new StopWatch(true);
+        final Set<String> groupDisplayNames = getGroupsWith(groupFilterList, prefix, suffix, substring, pageSize);
+        refreshUserGroupData(groupDisplayNames, pageSize);
+        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 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)
+                    .filter(s-> !s.isEmpty())
+                    .collect(Collectors.toList())
+            );
+        }
+        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<>();
+        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 = Arrays.asList(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) {
+                boolean 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, int pageSize) 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;
+                    if (!jsonUser.get("id").isJsonNull()) {
+                        idUser = jsonUser.get("id").getAsString();
+                    } else {
+                        idUser = "";
+                    }
+                    // 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);
+        }
+        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, int pageSize) throws IOException, ClientException {
+        Objects.requireNonNull(groupDisplayNames);
+
+        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, pageSize);
+            if (queryResult != null) {
+                groups.add(queryResult.getGroup());
+                users.addAll(queryResult.getUsers());
+            }
+        }
+        final ImmutableAzureGraphUserGroup azureGraphUserGroup = ImmutableAzureGraphUserGroup.newInstance(users, groups);
+        azureGraphUserGroupRef.set(azureGraphUserGroup);
+    }
+
+    @Override
+    public void preDestruction() throws AuthorizerDestructionException {
+        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);
+        }
+    }
+
+    private static class UserGroupQueryResult {
+        private final Group group;
+        private final Set<User> users;
+
+        public UserGroupQueryResult(Group group, Set<User> users) {
+            this.group = group;
+            this.users = users;
+        }
+
+        public Group getGroup() {
+            return this.group;
+        }
+
+        public Set<User> getUsers() {
+            return this.users;
+        }
+    }
+
+}
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java
new file mode 100644
index 0000000..552bac9
--- /dev/null
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ClientCredentialAuthProvider.java
@@ -0,0 +1,179 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+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 {
+
+    private final String authorityEndpoint;
+    private final String tenantId;
+    private final String clientId;
+    private final String clientSecret;
+    private LocalDateTime tokenExpiresOnDate;
+    private String lastAcessToken;
+    private static final String GRAPH_DEFAULT_SCOPE = "https://graph.microsoft.com/.default";
+    private static final 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 + "'" +
+            "}";
+    }
+
+    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();
+        ClientCredentialParameters clientCredentialParam = ClientCredentialParameters.builder(
+                Collections.singleton(GRAPH_DEFAULT_SCOPE))
+                .build();
+
+        CompletableFuture<IAuthenticationResult> future = app.acquireToken(clientCredentialParam);
+        return future.get();
+    }
+
+    private LocalDateTime convertToLocalDateTime(Date dateToConvert) {
+        return Instant.ofEpochMilli(dateToConvert.getTime())
+          .atZone(ZoneId.systemDefault())
+          .toLocalDateTime();
+    }
+
+    private String getAccessToken() {
+        if ((lastAcessToken != null) && (tokenExpiresOnDate != null) && (tokenExpiresOnDate.isAfter(LocalDateTime.now().plusMinutes(1)))) {
+            return lastAcessToken;
+        } else {
+            try {
+                IAuthenticationResult result = getAccessTokenByClientCredentialGrant();
+                tokenExpiresOnDate = convertToLocalDateTime(result.expiresOnDate());
+                lastAcessToken = result.accessToken();
+            } catch(final Exception e) {
+                logger.error("Failed to get access token due to {}", e.getMessage(), e);
+            }
+            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 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);
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ImmutableAzureGraphUserGroup.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ImmutableAzureGraphUserGroup.java
new file mode 100644
index 0000000..0aab274
--- /dev/null
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/java/org/apache/nifi/authorization/azure/ImmutableAzureGraphUserGroup.java
@@ -0,0 +1,158 @@
+/*
+ * 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) {
+        final User user = getUserByPrincipalName(principalName);
+        if (user != null) {
+            final String objectId = user.getIdentifier();
+            return getUserAndGroupsByUserObjectId(objectId);
+        } else {
+            // this covers the certificate-based authentication path
+            // this path may be called when CompositeUserGroupProvider is used
+            return new UserAndGroups() {
+                @Override
+                public User getUser() {
+                    return null;
+                }
+                @Override
+                public Set<Group> getGroups() {
+                    return null;
+                }
+            };
+        }
+    }
+
+    public Set<Group> getGroups() {
+        return groups;
+    }
+
+    public Group getGroup(final String objectId) {
+        return groupsByObjectId.get(objectId);
+    }
+
+    public Group getGroupByDisplayName(final String displayName) {
+        return groupsByDisplayName.get(displayName);
+    }
+
+    public static ImmutableAzureGraphUserGroup newInstance(final Set<User> users, final Set<Group> groups) {
+        final Map<String, User> usersByObjectId = new HashMap<>();
+        final Map<String, User> usersByPrincipalName = new HashMap<>();
+
+        users.forEach(user -> {
+            usersByObjectId.put(user.getIdentifier(), user);
+            usersByPrincipalName.put(user.getIdentity(), user);
+        });
+
+        final Map<String, Group> groupsByObjectId = new HashMap<>();
+        final Map<String, Group> groupsByDisplayName = new HashMap<>();
+        final Map<String, Set<Group>> groupsByUserObjectId =
+            users.stream().collect(toMap(User::getIdentifier, user -> {
+                return new HashSet<Group>();
+            }));
+
+        groups.forEach(group -> {
+            groupsByObjectId.put(group.getIdentifier(), group);
+            groupsByDisplayName.put(group.getName(), group);
+            group.getUsers().forEach(user -> {
+                groupsByUserObjectId.get(user).add(group);
+            });
+        });
+
+        final Map<String, UserAndGroups> usersAndGroupsByUserObjectId =
+            groupsByUserObjectId.entrySet()
+                .stream()
+                .collect(toMap(Map.Entry::getKey, e -> {
+                    return new UserAndGroups() {
+                        @Override public User getUser() {
+                            return usersByObjectId.get(e.getKey());
+                        }
+                        @Override public Set<Group> getGroups() {
+                            return e.getValue();
+                        }
+                    };
+                }));
+
+        return new ImmutableAzureGraphUserGroup(
+            users,
+            usersByObjectId,
+            usersByPrincipalName,
+            usersAndGroupsByUserObjectId,
+            groups,
+            groupsByObjectId,
+            groupsByDisplayName
+        );
+    }
+}
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/resources/META-INF/services/org.apache.nifi.authorization.UserGroupProvider b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/resources/META-INF/services/org.apache.nifi.authorization.UserGroupProvider
new file mode 100755
index 0000000..d64a7d0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/main/resources/META-INF/services/org.apache.nifi.authorization.UserGroupProvider
@@ -0,0 +1,15 @@
+# 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.
+org.apache.nifi.authorization.azure.AzureGraphUserGroupProvider
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/test/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProviderIT.java b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-graph-authorizer/src/test/java/org/apache/nifi/authorization/azure/AzureGraphUserGroupProviderIT.java
new file mode 100644
index 0000000..8be6b30
--- /dev/null
+++ b/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("AUTHORITY_ENDPOINT");
+    }
+
+    protected static String getTenantId() {
+        return CONFIG.getProperty("TENANT_ID");
+    }
+
+    protected static String getAppRegClientId() {
+        return CONFIG.getProperty("APP_REG_CLIENT_ID");
+    }
+
+    protected static String getAppRegClientSecret() {
+        return CONFIG.getProperty("APP_REG_CLIENT_SECRET");
+    }
+
+    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("GROUP_FILTER_LIST_INCLUSION");
+    }
+
+    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("Error during setup; 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.assertNotNull(uag.getUser());
+        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.assertNotNull(uag.getUser());
+        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.assertEquals(search.size(), testingProvider.getGroups().size());
+    }
+}
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-nar/pom.xml b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-nar/pom.xml
index f944efe..0de1267 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-nar/pom.xml
@@ -42,6 +42,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-azure-graph-authorizer</artifactId>
+            <version>1.14.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-azure-services-api-nar</artifactId>
             <version>1.14.0-SNAPSHOT</version>
             <type>nar</type>
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-nar/src/main/resources/META-INF/LICENSE
index cc58371..7b49df7 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-nar/src/main/resources/META-INF/LICENSE
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-nar/src/main/resources/META-INF/LICENSE
@@ -207,19 +207,19 @@
   The source is available under an MIT LICENSE.
 
 	Copyright (c) 2016 Microsoft Azure
-	
-	Permission is hereby granted, free of charge, to any person obtaining a copy of this software and 
-	associated documentation files (the Software), to deal in the Software without restriction, including 
-	without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or 
-	sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject 
+
+	Permission is hereby granted, free of charge, to any person obtaining a copy of this software and
+	associated documentation files (the Software), to deal in the Software without restriction, including
+	without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or
+	sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject
 	to the following conditions:
-	
+
 	The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
-	
-	THE SOFTWARE IS PROVIDED *AS IS*, WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT 
-	LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN 
-	NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, 
-	WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE 
+
+	THE SOFTWARE IS PROVIDED *AS IS*, WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT
+	LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN
+	NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
+	WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE
 	SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
 
 
@@ -406,4 +406,27 @@
     INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
     SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
     THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN
-    ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
+    ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+  The binary distribution of this product bundles 'Microsoft Graph SDK for Java'
+  which is available under an MIT license.
+
+    Copyright (c) 2018 Microsoft Graph
+
+    Permission is hereby granted, free of charge, to any person obtaining a copy
+    of this software and associated documentation files (the "Software"), to
+    deal in the Software without restriction, including without limitation the
+    rights to use, copy, modify, merge, publish, distribute, sublicense, and/or
+    sell copies of the Software, and to permit persons to whom the Software is
+    furnished to do so, subject to the following conditions:
+
+    The above copyright notice and this permission notice shall be included in
+    all copies or substantial portions of the Software.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+    IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+    AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+    LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+    FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+    IN THE SOFTWARE.
diff --git a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/pom.xml b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/pom.xml
index 186ea75..b79a632 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/pom.xml
@@ -70,7 +70,7 @@
         <dependency>
             <groupId>com.azure</groupId>
             <artifactId>azure-identity</artifactId>
-            <version>1.0.6</version>
+            <version>${azure.identity.version}</version>
             <exclusions>
                 <exclusion>
                     <groupId>com.azure</groupId>
diff --git a/nifi-nar-bundles/nifi-azure-bundle/pom.xml b/nifi-nar-bundles/nifi-azure-bundle/pom.xml
index 2bfaed8..48fcc36 100644
--- a/nifi-nar-bundles/nifi-azure-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-azure-bundle/pom.xml
@@ -28,12 +28,14 @@
     <properties>
         <azure-storage.version>8.4.0</azure-storage.version>
         <azure.core.version>1.6.0</azure.core.version>
+        <azure.identity.version>1.0.6</azure.identity.version>
         <azure-cosmos.version>4.2.0</azure-cosmos.version>
     </properties>
 
     <modules>
         <module>nifi-azure-processors</module>
         <module>nifi-azure-reporting-task</module>
+        <module>nifi-azure-graph-authorizer</module>
         <module>nifi-azure-nar</module>
         <module>nifi-azure-services-api</module>
         <module>nifi-azure-services-api-nar</module>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorizers.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorizers.xml
index f7a6752..f0e8e58 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorizers.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorizers.xml
@@ -196,6 +196,50 @@
     To enable the shell-user-group-provider remove 2 lines. This is 2 of 2. -->
 
     <!--
+        The AzureGraphUserGroupProvider fetches users and groups from Azure Active Directory (AAD) using the Microsoft Graph API.
+
+        'Refresh Delay'                 - (Optional) Duration of delay between each user and group refresh. Default is `5 mins`.
+        'Authority Endpoint'            - The endpoint of the Azure AD login. This can be found in the Azure portal under
+            Azure Active Directory -> App registrations -> [application name] -> Endpoints.
+        'Directory ID'                  - Tenant ID or Directory ID. This can be found in the Azural portal under
+            Azure Active Directory -> App registrations -> [application name] -> Directory (tenant) ID.
+        'Application ID'                - Client ID or Application ID of the Azure app registration. This can be found in the
+            Azure portal under Azure Active Directory -> App registrations -> [application name] -> Overview -> Application (client) ID.
+        'Client Secret'                 - A client secret from the Azure 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.
+        'Group Filter Prefix'           - (Optional) Prefix filter for Azure AD groups. Matches against the group displayName
+            to retrieve only groups with names starting with the provided prefix.
+        'Group Filter Suffix'           - (Optional) Suffix filter for Azure AD groups. Matches against the group displayName
+            to retrieve only groups with names ending with the provided suffix.
+        'Group Filter Substring'        - (Optional) Substring filter for Azure AD groups. Matches against the group displayName
+            to retrieve only groups with names containing the provided substring.
+        'Group Filter List Inclusion'   - (Optional) Comma-separated list of Azure AD groups. If no string-based matching filter
+            (i.e., prefix, suffix, and substring) is specified, set this property to avoid fetching all groups and users in the
+            Azure AD tenant.
+        'Page Size'                     - (Optional) Page size to use with the Microsoft Graph API. Set to 0 to disable paging API
+            calls. Default: 50, Max: 999.
+        'Claim for Username'            - (Optional) The property of the user directory object mapped to the NiFi user name field.
+            Default is 'upn'. 'email' is another option when nifi.security.user.oidc.fallback.claims.identifying.user is set to 'upn'.
+    -->
+    <!-- To enable the azure-graph-user-group-provider remove 2 lines. This is 1 of 2.
+    <userGroupProvider>
+        <identifier>azure-graph-user-group-provider</identifier>
+        <class>org.apache.nifi.authorization.azure.AzureGraphUserGroupProvider</class>
+        <property name="Refresh Delay">5 mins</property>
+        <property name="Authority Endpoint">https://login.microsoftonline.com</property>
+        <property name="Directory ID">XXXXXXXX-XXXX-XXXX-XXXX-XXXXXXXX</property>
+        <property name="Application ID">XXXXXXXX-XXXX-XXXX-XXXX-XXXXXXXX</property>
+        <property name="Client Secret">XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX</property>
+        <property name="Group Filter Prefix"></property>
+        <property name="Group Filter Suffix"></property>
+        <property name="Group Filter Substring"></property>
+        <property name="Group Filter List Inclusion"></property>
+        <property name="Page Size"></property>
+        <property name="Claim for Username">upn</property>
+    </userGroupProvider>
+    To enable the azure-graph-user-group-provider remove 2 lines. This is 2 of 2. -->
+
+    <!--
         The CompositeUserGroupProvider will provide support for retrieving users and groups from multiple sources.
 
         - User Group Provider [unique key] - The identifier of user group providers to load from. The name of