You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by mc...@apache.org on 2016/04/07 22:30:01 UTC

nifi git commit: NIFI-1553: - Implementing a file based authorizer. - Providing an example authorizations files. - Address comments from PR. - This closes #330

Repository: nifi
Updated Branches:
  refs/heads/master 3f4ac3156 -> 5de40ccec


NIFI-1553:
- Implementing a file based authorizer.
- Providing an example authorizations files.
- Address comments from PR.
- This closes #330


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/5de40cce
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/5de40cce
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/5de40cce

Branch: refs/heads/master
Commit: 5de40ccec3c330c097aa9ebc2d99cb01a1e55487
Parents: 3f4ac31
Author: Matt Gilman <ma...@gmail.com>
Authored: Thu Apr 7 16:11:07 2016 -0400
Committer: Matt Gilman <ma...@gmail.com>
Committed: Thu Apr 7 16:28:42 2016 -0400

----------------------------------------------------------------------
 .../authorization/AuthorizationRequest.java     |   4 +-
 .../nifi/authorization/AuthorizationResult.java |   2 +-
 .../AuthorizerConfigurationContext.java         |   4 +-
 .../nifi-framework/nifi-administration/pom.xml  |   4 +
 .../StandardAuthorizerConfigurationContext.java |   7 +-
 .../nifi-framework/nifi-file-authorizer/pom.xml |  94 +++++++
 .../nifi/authorization/FileAuthorizer.java      | 279 +++++++++++++++++++
 .../org.apache.nifi.authorization.Authorizer    |  15 +
 .../src/main/xsd/authorizations.xsd             |  61 ++++
 .../nifi/authorization/FileAuthorizerTest.java  | 198 +++++++++++++
 .../src/main/resources/conf/authorizations.xml  |  67 +++++
 .../nifi-framework/pom.xml                      |   1 +
 nifi-nar-bundles/nifi-framework-bundle/pom.xml  |   5 +
 13 files changed, 735 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/5de40cce/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorizationRequest.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorizationRequest.java b/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorizationRequest.java
index 38c9e26..9e50e62 100644
--- a/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorizationRequest.java
+++ b/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorizationRequest.java
@@ -40,8 +40,8 @@ public class AuthorizationRequest {
         this.resource = builder.resource;
         this.identity = builder.identity;
         this.action = builder.action;
-        this.context = Collections.unmodifiableMap(builder.context);
-        this.eventAttributes = Collections.unmodifiableMap(builder.eventAttributes);
+        this.context = builder.context == null ? null : Collections.unmodifiableMap(builder.context);
+        this.eventAttributes = builder.context == null ? null : Collections.unmodifiableMap(builder.eventAttributes);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/5de40cce/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorizationResult.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorizationResult.java b/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorizationResult.java
index acbbbe2..a3f520c 100644
--- a/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorizationResult.java
+++ b/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorizationResult.java
@@ -21,7 +21,7 @@ package org.apache.nifi.authorization;
  */
 public class AuthorizationResult {
 
-    private enum Result {
+    public enum Result {
         Approved,
         Denied,
         ResourceNotFound

http://git-wip-us.apache.org/repos/asf/nifi/blob/5de40cce/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorizerConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorizerConfigurationContext.java b/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorizerConfigurationContext.java
index b2b6b3a..3721ab4 100644
--- a/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorizerConfigurationContext.java
+++ b/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorizerConfigurationContext.java
@@ -16,6 +16,8 @@
  */
 package org.apache.nifi.authorization;
 
+import org.apache.nifi.components.PropertyValue;
+
 import java.util.Map;
 
 /**
@@ -44,5 +46,5 @@ public interface AuthorizerConfigurationContext {
      * PropertyDescriptor. This method does not substitute default
      * PropertyDescriptor values, so the value returned will be null if not set
      */
-    String getProperty(String property);
+    PropertyValue getProperty(String property);
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5de40cce/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/pom.xml
index 2fef0c4..c9a9c0e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/pom.xml
@@ -116,5 +116,9 @@
             <groupId>org.apache.commons</groupId>
             <artifactId>commons-collections4</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-expression-language</artifactId>
+        </dependency>
     </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/5de40cce/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/StandardAuthorizerConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/StandardAuthorizerConfigurationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/StandardAuthorizerConfigurationContext.java
index 946da96..3010c92 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/StandardAuthorizerConfigurationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/authorization/StandardAuthorizerConfigurationContext.java
@@ -16,6 +16,9 @@
  */
 package org.apache.nifi.authorization;
 
+import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
+import org.apache.nifi.components.PropertyValue;
+
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
@@ -44,8 +47,8 @@ public class StandardAuthorizerConfigurationContext implements AuthorizerConfigu
     }
 
     @Override
-    public String getProperty(String property) {
-        return properties.get(property);
+    public PropertyValue getProperty(String property) {
+        return new StandardPropertyValue(properties.get(property), null);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5de40cce/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/pom.xml
new file mode 100644
index 0000000..53f35f4
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/pom.xml
@@ -0,0 +1,94 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<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">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-framework</artifactId>
+        <version>1.0.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-file-authorizer</artifactId>
+    <build>
+        <resources>
+            <resource>
+                <directory>src/main/resources</directory>
+            </resource>
+            <resource>
+                <directory>src/main/xsd</directory>
+            </resource>
+        </resources>
+        <plugins>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>jaxb2-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>xjc</id>
+                        <goals>
+                            <goal>xjc</goal>
+                        </goals>
+                        <configuration>
+                            <packageName>org.apache.nifi.authorization.generated</packageName>
+                        </configuration>
+                    </execution>
+                </executions>
+                <configuration>
+                    <outputDirectory>${project.build.directory}/generated-sources/jaxb</outputDirectory>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-checkstyle-plugin</artifactId>
+                <configuration>
+                    <excludes>**/authorization/generated/*.java</excludes>
+                </configuration>
+            </plugin>            
+
+        </plugins>
+    </build>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-properties</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-authorization</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-codec</groupId>
+            <artifactId>commons-codec</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-expression-language</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/5de40cce/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAuthorizer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAuthorizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAuthorizer.java
new file mode 100644
index 0000000..174e501
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/java/org/apache/nifi/authorization/FileAuthorizer.java
@@ -0,0 +1,279 @@
+/*
+ * 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;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.authorization.annotation.AuthorizerContext;
+import org.apache.nifi.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.authorization.exception.ProviderCreationException;
+import org.apache.nifi.authorization.generated.Authorization;
+import org.apache.nifi.authorization.generated.Resource;
+import org.apache.nifi.authorization.generated.Resources;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.util.file.FileUtils;
+import org.apache.nifi.util.file.monitor.MD5SumMonitor;
+import org.apache.nifi.util.file.monitor.SynchronousFileWatcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.xml.sax.SAXException;
+
+import javax.xml.XMLConstants;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBElement;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Unmarshaller;
+import javax.xml.transform.stream.StreamSource;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+import java.io.File;
+import java.io.IOException;
+import java.util.Date;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Map;
+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;
+
+/**
+ * Provides identity checks and grants authorities.
+ */
+public class FileAuthorizer implements Authorizer {
+
+    private static final Logger logger = LoggerFactory.getLogger(FileAuthorizer.class);
+    private static final String READ_CODE = "R";
+    private static final String WRITE_CODE = "W";
+    private static final String USERS_XSD = "/authorizations.xsd";
+    private static final String JAXB_GENERATED_PATH = "org.apache.nifi.authorization.generated";
+    private static final JAXBContext JAXB_CONTEXT = initializeJaxbContext();
+
+    /**
+     * Load the JAXBContext.
+     */
+    private static JAXBContext initializeJaxbContext() {
+        try {
+            return JAXBContext.newInstance(JAXB_GENERATED_PATH, FileAuthorizer.class.getClassLoader());
+        } catch (JAXBException e) {
+            throw new RuntimeException("Unable to create JAXBContext.");
+        }
+    }
+
+    private NiFiProperties properties;
+    private File authorizationsFile;
+    private File restoreAuthorizationsFile;
+    private SynchronousFileWatcher fileWatcher;
+    private ScheduledExecutorService fileWatcherExecutorService;
+
+    private final AtomicReference<Map<String, Map<String, Set<RequestAction>>>> authorizations = new AtomicReference<>();
+
+    @Override
+    public void initialize(final AuthorizerInitializationContext initializationContext) throws ProviderCreationException {
+    }
+
+    @Override
+    public void onConfigured(final AuthorizerConfigurationContext configurationContext) throws ProviderCreationException {
+        try {
+            final PropertyValue authorizationsPath = configurationContext.getProperty("Authorizations File");
+            if (StringUtils.isBlank(authorizationsPath.getValue())) {
+                throw new ProviderCreationException("The authorizations file must be specified.");
+            }
+
+            // get the authorizations file and ensure it exists
+            authorizationsFile = new File(authorizationsPath.getValue());
+            if (!authorizationsFile.exists()) {
+                throw new ProviderCreationException("The authorizations file must exist.");
+            }
+
+            final File authorizationsFileDirectory = authorizationsFile.getAbsoluteFile().getParentFile();
+
+            // the restore directory is optional and may be null
+            final File restoreDirectory = properties.getRestoreDirectory();
+            if (restoreDirectory != null) {
+                // sanity check that restore directory is a directory, creating it if necessary
+                FileUtils.ensureDirectoryExistAndCanAccess(restoreDirectory);
+
+                // check that restore directory is not the same as the primary directory
+                if (authorizationsFileDirectory.getAbsolutePath().equals(restoreDirectory.getAbsolutePath())) {
+                    throw new ProviderCreationException(String.format("Authorizations file directory '%s' is the same as restore directory '%s' ",
+                            authorizationsFileDirectory.getAbsolutePath(), restoreDirectory.getAbsolutePath()));
+                }
+
+                // the restore copy will have same file name, but reside in a different directory
+                restoreAuthorizationsFile = new File(restoreDirectory, authorizationsFile.getName());
+
+                try {
+                    // sync the primary copy with the restore copy
+                    FileUtils.syncWithRestore(authorizationsFile, restoreAuthorizationsFile, logger);
+                } catch (final IOException | IllegalStateException ioe) {
+                    throw new ProviderCreationException(ioe);
+                }
+            }
+
+            final PropertyValue rawReloadInterval = configurationContext.getProperty("Reload Interval");
+
+            long reloadInterval;
+            try {
+                reloadInterval = rawReloadInterval.asTimePeriod(TimeUnit.MILLISECONDS);
+            } catch (final Exception iae) {
+                logger.info(String.format("Unable to interpret reload interval '%s'. Using default of 30 seconds.", rawReloadInterval));
+                reloadInterval = 30000L;
+            }
+
+            // reload the authorizations
+            reload();
+
+            // watch the file for modifications
+            fileWatcher = new SynchronousFileWatcher(authorizationsFile.toPath(), new MD5SumMonitor());
+            fileWatcherExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() {
+                @Override
+                public Thread newThread(final Runnable r) {
+                    return new Thread(r, "Authorization File Reload Thread");
+                }
+            });
+            fileWatcherExecutorService.scheduleWithFixedDelay(new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        if (fileWatcher.checkAndReset()) {
+                            reload();
+                        }
+                    } catch (final Exception e) {
+                        logger.warn("Unable to reload Authorizations file do to: " + e, e);
+                    }
+                }
+            }, reloadInterval, reloadInterval, TimeUnit.MILLISECONDS);
+        } catch (IOException | ProviderCreationException | SAXException | JAXBException | IllegalStateException e) {
+            throw new ProviderCreationException(e);
+        }
+
+    }
+
+    @Override
+    public AuthorizationResult authorize(final AuthorizationRequest request) throws AuthorizationAccessException {
+        // get the current authorizations
+        final Map<String, Map<String, Set<RequestAction>>> currentAuthorizations = authorizations.get();
+
+        // get the requested resource
+        final org.apache.nifi.authorization.Resource requestedResource = request.getResource();
+
+        // get the authorizations for the requested resources
+        final Map<String, Set<RequestAction>> resourceAuthorizations = currentAuthorizations.get(requestedResource.getIdentifier());
+
+        // ensure the resource has authorizations
+        if (resourceAuthorizations == null) {
+            return AuthorizationResult.resourceNotFound();
+        }
+
+        // get the user authorizations
+        final Set<RequestAction> userAuthorizations = resourceAuthorizations.get(request.getIdentity());
+
+        // ensure the user has authorizations
+        if (userAuthorizations == null) {
+            return AuthorizationResult.denied();
+        }
+
+        // ensure the appropriate response
+        if (userAuthorizations.contains(request.getAction())) {
+            return AuthorizationResult.approved();
+        } else {
+            return AuthorizationResult.denied();
+        }
+    }
+
+    /**
+     * Reloads the authorized users file.
+     *
+     * @throws SAXException             Unable to reload the authorized users file
+     * @throws JAXBException            Unable to reload the authorized users file
+     * @throws IOException              Unable to sync file with restore
+     * @throws IllegalStateException    Unable to sync file with restore
+     */
+    private void reload() throws SAXException, JAXBException, IOException, IllegalStateException {
+        // find the schema
+        final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
+        final Schema schema = schemaFactory.newSchema(FileAuthorizer.class.getResource(USERS_XSD));
+
+        // attempt to unmarshal
+        final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
+        unmarshaller.setSchema(schema);
+        final JAXBElement<Resources> element = unmarshaller.unmarshal(new StreamSource(authorizationsFile), Resources.class);
+        final Resources resources = element.getValue();
+
+        // new authorizations
+        final Map<String, Map<String, Set<RequestAction>>> newAuthorizations = new HashMap<>();
+
+        // load the new authorizations
+        for (final Resource authorizedResource : resources.getResource()) {
+            final String identifier = authorizedResource.getIdentifier();
+
+            // ensure the entry exists
+            if (!newAuthorizations.containsKey(identifier)) {
+                newAuthorizations.put(identifier, new HashMap<String, Set<RequestAction>>());
+            }
+
+            // go through each authorization
+            for (final Authorization authorization : authorizedResource.getAuthorization()) {
+                final String identity = authorization.getIdentity();
+
+                // get the authorizations for this resource
+                final Map<String, Set<RequestAction>> resourceAuthorizations = newAuthorizations.get(identifier);
+
+                // ensure the entry exists
+                if (!resourceAuthorizations.containsKey(identity)) {
+                    resourceAuthorizations.put(identity, EnumSet.noneOf(RequestAction.class));
+                }
+
+                final Set<RequestAction> authorizedActions = resourceAuthorizations.get(identity);
+                final String authorizationCode = authorization.getAction();
+
+                // updated the actions for this identity
+                if (authorizationCode.contains(READ_CODE)) {
+                    authorizedActions.add(RequestAction.READ);
+                }
+                if (authorizationCode.contains(WRITE_CODE)) {
+                    authorizedActions.add(RequestAction.WRITE);
+                }
+            }
+        }
+
+        // set the new authorizations
+        authorizations.set(newAuthorizations);
+
+        // if we've copied a the authorizations file to a restore directory synchronize it
+        if (restoreAuthorizationsFile != null) {
+            FileUtils.copyFile(authorizationsFile, restoreAuthorizationsFile, false, false, logger);
+        }
+
+        logger.info(String.format("Authorizations file loaded at %s", new Date().toString()));
+    }
+
+    @AuthorizerContext
+    public void setNiFiProperties(NiFiProperties properties) {
+        this.properties = properties;
+    }
+
+    @Override
+    public void preDestruction() {
+        if (fileWatcherExecutorService != null) {
+            fileWatcherExecutorService.shutdown();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5de40cce/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/resources/META-INF/services/org.apache.nifi.authorization.Authorizer
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/resources/META-INF/services/org.apache.nifi.authorization.Authorizer b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/resources/META-INF/services/org.apache.nifi.authorization.Authorizer
new file mode 100755
index 0000000..da9c2ee
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/resources/META-INF/services/org.apache.nifi.authorization.Authorizer
@@ -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.FileAuthorizer

http://git-wip-us.apache.org/repos/asf/nifi/blob/5de40cce/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/xsd/authorizations.xsd
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/xsd/authorizations.xsd b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/xsd/authorizations.xsd
new file mode 100644
index 0000000..dc17265
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/main/xsd/authorizations.xsd
@@ -0,0 +1,61 @@
+<?xml version="1.0"?>
+<!--
+  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.
+-->
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema">
+    <!-- authorization -->
+    <xs:complexType name="Authorization">
+        <xs:attribute name="identity">
+            <xs:simpleType>
+                <xs:restriction base="xs:string">
+                    <xs:minLength value="1"/>
+                    <xs:pattern value=".*[^\s].*"/>
+                </xs:restriction>
+            </xs:simpleType>
+        </xs:attribute>
+        <xs:attribute name="action">
+            <xs:simpleType>
+                <xs:restriction base="xs:string">
+                    <xs:enumeration value="R"/>
+                    <xs:enumeration value="W"/>
+                    <xs:enumeration value="RW"/>
+                </xs:restriction>
+            </xs:simpleType>
+        </xs:attribute>
+    </xs:complexType>
+
+    <!-- resource -->
+    <xs:complexType name="Resource">
+        <xs:sequence>
+            <xs:element name="authorization" type="Authorization" minOccurs="0" maxOccurs="unbounded"/>
+        </xs:sequence>
+        <xs:attribute name="identifier">
+            <xs:simpleType>
+                <xs:restriction base="xs:string">
+                    <xs:minLength value="1"/>
+                    <xs:pattern value=".*[^\s].*"/>
+                </xs:restriction>
+            </xs:simpleType>
+        </xs:attribute>
+    </xs:complexType>
+
+    <!-- resources -->
+    <xs:element name="resources">
+        <xs:complexType>
+            <xs:sequence>
+                <xs:element name="resource" type="Resource" minOccurs="0" maxOccurs="unbounded"/>
+            </xs:sequence>
+        </xs:complexType>
+    </xs:element>
+</xs:schema>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/5de40cce/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
new file mode 100644
index 0000000..359d45b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorizer/src/test/java/org/apache/nifi/authorization/FileAuthorizerTest.java
@@ -0,0 +1,198 @@
+/*
+ * 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;
+
+import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
+import org.apache.nifi.authorization.AuthorizationResult.Result;
+import org.apache.nifi.authorization.exception.ProviderCreationException;
+import org.apache.nifi.authorization.resource.ResourceFactory;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class FileAuthorizerTest {
+
+    private static final String EMPTY_AUTHORIZATIONS_CONCISE =
+        "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"yes\"?>"
+        + "<resources/>";
+
+    private static final String EMPTY_AUTHORIZATIONS =
+        "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"yes\"?>"
+        + "<resources>"
+        + "</resources>";
+
+    private static final String BAD_SCHEMA_AUTHORIZATIONS =
+        "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"yes\"?>"
+        + "<resource>"
+        + "</resource>";
+
+    private static final String AUTHORIZATIONS =
+        "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"yes\"?>"
+        + "<resources>"
+            + "<resource identifier=\"/flow\">"
+                + "<authorization identity=\"user-1\" action=\"R\"/>"
+            + "</resource>"
+        + "</resources>";
+
+    private static final String UPDATED_AUTHORIZATIONS =
+        "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"yes\"?>"
+        + "<resources>"
+            + "<resource identifier=\"/flow\">"
+                + "<authorization identity=\"user-1\" action=\"RW\"/>"
+            + "</resource>"
+        + "</resources>";
+
+    private FileAuthorizer authorizer;
+    private File primary;
+    private File restore;
+
+    private AuthorizerConfigurationContext configurationContext;
+
+    @Before
+    public void setup() throws IOException {
+        // primary authorizations
+        primary = new File("target/primary/authorizations.xml");
+        FileUtils.ensureDirectoryExistAndCanAccess(primary.getParentFile());
+
+        // restore authorizations
+        restore = new File("target/restore/authorizations.xml");
+        FileUtils.ensureDirectoryExistAndCanAccess(restore.getParentFile());
+
+        final NiFiProperties properties = mock(NiFiProperties.class);
+        when(properties.getRestoreDirectory()).thenReturn(restore.getParentFile());
+
+        configurationContext = mock(AuthorizerConfigurationContext.class);
+        when(configurationContext.getProperty(Mockito.eq("Authorizations File"))).thenReturn(new StandardPropertyValue(primary.getPath(), null));
+
+        authorizer = new FileAuthorizer();
+        authorizer.setNiFiProperties(properties);
+        authorizer.initialize(null);
+    }
+
+    @After
+    public void cleanup() throws Exception {
+        deleteFile(primary);
+        deleteFile(restore);
+    }
+
+    @Test
+    public void testPostConstructionWhenRestoreDoesNotExist() throws Exception {
+        writeAuthorizationsFile(primary, EMPTY_AUTHORIZATIONS_CONCISE);
+        authorizer.onConfigured(configurationContext);
+
+        assertEquals(primary.length(), restore.length());
+    }
+
+    @Test(expected = ProviderCreationException.class)
+    public void testPostConstructionWhenPrimaryDoesNotExist() throws Exception {
+        writeAuthorizationsFile(restore, EMPTY_AUTHORIZATIONS_CONCISE);
+        authorizer.onConfigured(configurationContext);
+    }
+
+    @Test(expected = ProviderCreationException.class)
+    public void testPostConstructionWhenPrimaryDifferentThanRestore() throws Exception {
+        writeAuthorizationsFile(primary, EMPTY_AUTHORIZATIONS);
+        writeAuthorizationsFile(restore, EMPTY_AUTHORIZATIONS_CONCISE);
+        authorizer.onConfigured(configurationContext);
+    }
+
+    @Test(expected = ProviderCreationException.class)
+    public void testBadSchema() throws Exception {
+        writeAuthorizationsFile(primary, BAD_SCHEMA_AUTHORIZATIONS);
+        authorizer.onConfigured(configurationContext);
+    }
+
+    @Test
+    public void testAuthorizedUserAction() throws Exception {
+        writeAuthorizationsFile(primary, AUTHORIZATIONS);
+        authorizer.onConfigured(configurationContext);
+
+        final AuthorizationRequest request = new AuthorizationRequest.Builder().resource(ResourceFactory.getFlowResource()).identity("user-1").action(RequestAction.READ).build();
+        final AuthorizationResult result = authorizer.authorize(request);
+        assertTrue(Result.Approved.equals(result.getResult()));
+    }
+
+    @Test
+    public void testUnauthorizedUser() throws Exception {
+        writeAuthorizationsFile(primary, AUTHORIZATIONS);
+        authorizer.onConfigured(configurationContext);
+
+        final AuthorizationRequest request = new AuthorizationRequest.Builder().resource(ResourceFactory.getFlowResource()).identity("user-2").action(RequestAction.READ).build();
+        final AuthorizationResult result = authorizer.authorize(request);
+        assertFalse(Result.Approved.equals(result.getResult()));
+    }
+
+    @Test
+    public void testUnauthorizedAction() throws Exception {
+        writeAuthorizationsFile(primary, AUTHORIZATIONS);
+        authorizer.onConfigured(configurationContext);
+
+        final AuthorizationRequest request = new AuthorizationRequest.Builder().resource(ResourceFactory.getFlowResource()).identity("user-1").action(RequestAction.WRITE).build();
+        final AuthorizationResult result = authorizer.authorize(request);
+        assertFalse(Result.Approved.equals(result.getResult()));
+    }
+
+    @Test
+    public void testReloadAuthorizations() throws Exception {
+        writeAuthorizationsFile(primary, AUTHORIZATIONS);
+        when(configurationContext.getProperty(Mockito.eq("Reload Interval"))).thenReturn(new StandardPropertyValue("1 sec", null));
+        authorizer.onConfigured(configurationContext);
+
+        // ensure the user currently does not have write access
+        final AuthorizationRequest request = new AuthorizationRequest.Builder().resource(ResourceFactory.getFlowResource()).identity("user-1").action(RequestAction.WRITE).build();
+        AuthorizationResult result = authorizer.authorize(request);
+        assertFalse(Result.Approved.equals(result.getResult()));
+
+        // add write access for the user
+        writeAuthorizationsFile(primary, UPDATED_AUTHORIZATIONS);
+
+        // wait at least one second for the file to be stale
+        Thread.sleep(4000L);
+
+        // ensure the user does have write access now using the same request
+        result = authorizer.authorize(request);
+        assertTrue(Result.Approved.equals(result.getResult()));
+    }
+
+    private static void writeAuthorizationsFile(final File file, final String content) throws Exception {
+        byte[] bytes = content.getBytes(StandardCharsets.UTF_8);
+        try (final FileOutputStream fos = new FileOutputStream(file)) {
+            fos.write(bytes);
+        }
+    }
+
+    private static boolean deleteFile(final File file) {
+        if (file.isDirectory()) {
+            FileUtils.deleteFilesInDir(file, null, null, true, true);
+        }
+        return FileUtils.deleteFile(file, null, 10);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5de40cce/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorizations.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorizations.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorizations.xml
new file mode 100644
index 0000000..21815c0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/authorizations.xml
@@ -0,0 +1,67 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+  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.
+-->
+<!--
+    This file lists all authorizations for this NiFi instance. Refer to the properties file and authorizers.xml for configuration details.
+    
+    Available resources:
+        /flow                       - READ - allows user/entity to load the UI and see the flow structure
+                                    - WRITE - NA
+        /resource                   - READ - allows user/entity to retrieve the available resources
+                                    - WRITE - NA
+        /system                     - READ - allows user/entity to retrieve system level diagnostics (CPU load, disk utilization, etc)
+                                    - WRITE - NA
+        /controller                 - READ - allows user/entity to retrieve configuration details for the controller (controller bulletins, thread pool, reporting tasks, etc)
+                                    - WRITE - allows user/entity to modify configuration details for the controller
+        /provenance                 - READ - allows user/entity to perform provenance requests. results will be filtered based on access to provenance data per component
+                                    - WRITE - NA
+        /token                      - READ - NA
+                                    - WRITE - allows user/entity to create a token for access the REST API
+        /site-to-site               - READ - allows user/entity to retrieve configuration details for performing site to site data transfers with this NiFi
+                                    - WRITE - NA
+        /proxy                      - READ - NA
+                                    - WRITE - allows user/entity to create a proxy request on behalf of another user
+        /process-groups/{id}        - READ - allows user/entity to retrieve configuration details for the process group and all descendant components without explicit access policies
+                                    - WRITE - allows user/entity to create/update/delete configuration details for the process group and all descendant components without explicit access policies
+        /processors/{id}            - READ - allows user/entity to retrieve configuration details for the processor overriding any inherited authorizations from an ancestor process group
+                                    - WRITE - allows user/entity to update/delete the processor overriding any inherited authorizations from an ancestor process group
+        /input-ports/{id}           - READ - allows user/entity to retrieve configuration details for the input port overriding any inherited authorizations from an ancestor process group
+                                    - WRITE - allows user/entity to update/delete the input port overriding any inherited authorizations from an ancestor process group
+        /output-ports/{id}          - READ - allows user/entity to retrieve configuration details for the output port overriding any inherited authorizations from an ancestor process group
+                                    - WRITE - allows user/entity to update/delete the output port overriding any inherited authorizations from an ancestor process group
+        /labels/{id}                - READ - allows user/entity to retrieve configuration details for the label overriding any inherited authorizations from an ancestor process group
+                                    - WRITE - allows user/entity to update/delete the label overriding any inherited authorizations from an ancestor process group
+        /connections/{id}           - READ - allows user/entity to retrieve configuration details for the connection overriding any inherited authorizations from an ancestor process group
+                                    - WRITE - allows user/entity to update/delete the label overriding any inherited authorizations from an ancestor process group
+        /remote-process-groups/{id} - READ - allows user/entity to retrieve configuration details for the remote process group overriding any inherited authorizations from an ancestor process group
+                                    - WRITE - allows user/entity to update/delete the remote process group overriding any inherited authorizations from an ancestor process group
+        /templates/{id}             - READ - allows user/entity to retrieve configuration details for the template overriding any inherited authorizations from an ancestor process group
+                                    - WRITE - allows user/entity to create/update/delete the template overriding any inherited authorizations from an ancestor process group
+        /controller-services/{id}   - READ - allows user/entity to retrieve configuration details for the controller service overriding any inherited authorizations from an ancestor process group
+                                    - WRITE - allows user/entity to update/delete the controller service overriding any inherited authorizations from an ancestor process group
+        /reporting-tasks/{id}       - READ - allows user/entity to retrieve configuration details for the reporting tasks overriding any inherited authorizations from the controller
+                                    - WRITE - allows user/entity to create/update/delete the reporting tasks overriding any inherited authorizations from the controller
+        /{type}/{id}/provenance     - READ - allows user/entity to view provenance data from the underlying component
+                                    - WRITE - NA
+-->
+<resources>
+    <!--
+    <resource identifier="/flow">
+        <authorization identity="user-identity-1" action="R"></authorization>
+        <authorization identity="user-identity-2" action="W"></authorization>
+        <authorization identity="user-identity-3" action="RW"></authorization>
+    </resource>
+    -->
+</resources>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/5de40cce/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
index 7faf517..771a258 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
@@ -37,6 +37,7 @@
         <module>nifi-cluster-authorization-provider</module>
         <module>nifi-user-actions</module>
         <module>nifi-framework-authorization</module>
+        <module>nifi-file-authorizer</module>
         <module>nifi-administration</module>
         <module>nifi-web</module>
         <module>nifi-resources</module>

http://git-wip-us.apache.org/repos/asf/nifi/blob/5de40cce/nifi-nar-bundles/nifi-framework-bundle/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/pom.xml
index d1b1422..b4f1c55 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/pom.xml
@@ -90,6 +90,11 @@
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-framework-authorization</artifactId>
+                <version>1.0.0-SNAPSHOT</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-user-actions</artifactId>
                 <version>1.0.0-SNAPSHOT</version>
             </dependency>