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 2014/12/23 21:37:34 UTC

[01/14] incubator-nifi git commit: NIFI-65: - Adding an authorizeDownload method to the UserService. - Removing an unused/deprecated unit test.

Repository: incubator-nifi
Updated Branches:
  refs/heads/develop 1374d1db7 -> b569a9d78


NIFI-65:
- Adding an authorizeDownload method to the UserService.
- Removing an unused/deprecated unit test.

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

Branch: refs/heads/develop
Commit: 1c9393032d4917bf4b9584a5066d84a85dc68a6b
Parents: df9529c
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Dec 23 09:30:29 2014 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Dec 23 09:30:29 2014 -0500

----------------------------------------------------------------------
 .../apache/nifi/admin/service/UserService.java  |  13 +
 .../service/action/AuthorizeDownloadAction.java |  54 ++++
 .../admin/service/impl/StandardUserService.java |  36 ++-
 .../AuthorityProviderFactoryBean.java           |  13 +
 .../java/org/apache/nifi/user/NiFiUser.java     |  10 +
 .../impl/NiFiAuthorizationServiceTest.java      | 284 -------------------
 6 files changed, 125 insertions(+), 285 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1c939303/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/UserService.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/UserService.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/UserService.java
index 76e54d6..86256fd 100644
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/UserService.java
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/UserService.java
@@ -17,8 +17,11 @@
 package org.apache.nifi.admin.service;
 
 import java.util.Collection;
+import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import org.apache.nifi.authorization.Authority;
+import org.apache.nifi.authorization.DownloadAuthorization;
 import org.apache.nifi.user.NiFiUser;
 import org.apache.nifi.user.NiFiUserGroup;
 
@@ -44,6 +47,16 @@ public interface UserService {
     Boolean hasPendingUserAccount();
 
     /**
+     * Determines if the users in the dnChain are authorized to download content 
+     * with the specified attributes.
+     * 
+     * @param dnChain
+     * @param attributes
+     * @return 
+     */
+    DownloadAuthorization authorizeDownload(List<String> dnChain, Map<String, String> attributes);
+    
+    /**
      * Updates a user group using the specified group comprised of the specified
      * users. Returns all the users that are currently in the specified group.
      *

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1c939303/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeDownloadAction.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeDownloadAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeDownloadAction.java
new file mode 100644
index 0000000..d1b994c
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeDownloadAction.java
@@ -0,0 +1,54 @@
+/*
+ * 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.admin.service.action;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.nifi.admin.dao.DAOFactory;
+import org.apache.nifi.admin.service.AccountNotFoundException;
+import org.apache.nifi.admin.service.AdministrationException;
+import org.apache.nifi.authorization.AuthorityProvider;
+import org.apache.nifi.authorization.DownloadAuthorization;
+import org.apache.nifi.authorization.exception.AuthorityAccessException;
+import org.apache.nifi.authorization.exception.UnknownIdentityException;
+
+/**
+ * Attempts to obtain authorization to download the content with the specified
+ * attributes for the specified user.
+ */
+public class AuthorizeDownloadAction implements AdministrationAction<DownloadAuthorization> {
+
+    private final List<String> dnChain;
+    private final Map<String, String> attributes;
+
+    public AuthorizeDownloadAction(List<String> dnChain, Map<String, String> attributes) {
+        this.dnChain = dnChain;
+        this.attributes = attributes;
+    }
+
+    @Override
+    public DownloadAuthorization execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) {
+        try {
+            return authorityProvider.authorizeDownload(dnChain, attributes);
+        } catch (UnknownIdentityException uie) {
+            throw new AccountNotFoundException(uie.getMessage(), uie);
+        } catch (AuthorityAccessException aae) {
+            throw new AdministrationException(aae.getMessage(), aae);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1c939303/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardUserService.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardUserService.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardUserService.java
index 63aa93b..5c9af4b 100644
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardUserService.java
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardUserService.java
@@ -18,6 +18,8 @@ package org.apache.nifi.admin.service.impl;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -27,6 +29,7 @@ import org.apache.nifi.admin.service.AccountDisabledException;
 import org.apache.nifi.admin.service.AccountPendingException;
 import org.apache.nifi.admin.service.AdministrationException;
 import org.apache.nifi.admin.service.UserService;
+import org.apache.nifi.admin.service.action.AuthorizeDownloadAction;
 import org.apache.nifi.admin.service.action.AuthorizeUserAction;
 import org.apache.nifi.admin.service.action.DeleteUserAction;
 import org.apache.nifi.admin.service.action.DisableUserAction;
@@ -48,6 +51,7 @@ import org.apache.nifi.admin.service.transaction.Transaction;
 import org.apache.nifi.admin.service.transaction.TransactionBuilder;
 import org.apache.nifi.admin.service.transaction.TransactionException;
 import org.apache.nifi.authorization.Authority;
+import org.apache.nifi.authorization.DownloadAuthorization;
 import org.apache.nifi.user.NiFiUser;
 import org.apache.nifi.user.NiFiUserGroup;
 import org.apache.nifi.util.FormatUtils;
@@ -440,7 +444,7 @@ public class StandardUserService implements UserService {
      * modifying a user account. This method should only be invoked from within
      * a write lock.
      *
-     * @param id
+     * @param group
      */
     @Override
     public void invalidateUserGroupAccount(String group) {
@@ -501,6 +505,36 @@ public class StandardUserService implements UserService {
     }
 
     @Override
+    public DownloadAuthorization authorizeDownload(final List<String> dnChain, final Map<String, String> attributes) {
+        Transaction transaction = null;
+
+        readLock.lock();
+        try {
+            // start the transaction
+            transaction = transactionBuilder.start();
+
+            // authorize the download
+            AuthorizeDownloadAction authorizeDownload = new AuthorizeDownloadAction(dnChain, attributes);
+            DownloadAuthorization downloadAuthorization = transaction.execute(authorizeDownload);
+
+            // commit the transaction
+            transaction.commit();
+
+            // return the authorization
+            return downloadAuthorization;
+        } catch (TransactionException | DataAccessException te) {
+            rollback(transaction);
+            throw new AdministrationException(te);
+        } catch (Throwable t) {
+            rollback(transaction);
+            throw t;
+        } finally {
+            closeQuietly(transaction);
+            readLock.unlock();
+        }
+    }
+    
+    @Override
     public Collection<NiFiUser> getUsers() {
         Transaction transaction = null;
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1c939303/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/AuthorityProviderFactoryBean.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/AuthorityProviderFactoryBean.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/AuthorityProviderFactoryBean.java
index b05d32f..31a01be 100644
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/AuthorityProviderFactoryBean.java
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/AuthorityProviderFactoryBean.java
@@ -24,6 +24,7 @@ import java.lang.reflect.Method;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import javax.xml.XMLConstants;
@@ -366,6 +367,11 @@ public class AuthorityProviderFactoryBean implements FactoryBean, ApplicationCon
             }
 
             @Override
+            public DownloadAuthorization authorizeDownload(List<String> dnChain, Map<String, String> attributes) throws UnknownIdentityException, AuthorityAccessException {
+                return DownloadAuthorization.approved();
+            }
+
+            @Override
             public void initialize(AuthorityProviderInitializationContext initializationContext) throws ProviderCreationException {
             }
 
@@ -466,6 +472,13 @@ public class AuthorityProviderFactoryBean implements FactoryBean, ApplicationCon
             }
 
             @Override
+            public DownloadAuthorization authorizeDownload(List<String> dnChain, Map<String, String> attributes) throws UnknownIdentityException, AuthorityAccessException {
+                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
+                    return baseProvider.authorizeDownload(dnChain, attributes);
+                }
+            }
+
+            @Override
             public void initialize(AuthorityProviderInitializationContext initializationContext) throws ProviderCreationException {
                 try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
                     baseProvider.initialize(initializationContext);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1c939303/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUser.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUser.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUser.java
index 984a572..415160a 100644
--- a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUser.java
+++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUser.java
@@ -43,6 +43,8 @@ public class NiFiUser implements Serializable {
 
     private AccountStatus status;
     private EnumSet<Authority> authorities;
+    
+    private NiFiUser chain;
 
     /* getters / setters */
     public Date getCreation() {
@@ -117,6 +119,14 @@ public class NiFiUser implements Serializable {
         this.lastAccessed = lastAccessed;
     }
 
+    public NiFiUser getChain() {
+        return chain;
+    }
+
+    public void setChain(NiFiUser chain) {
+        this.chain = chain;
+    }
+
     public Set<Authority> getAuthorities() {
         if (authorities == null) {
             authorities = EnumSet.noneOf(Authority.class);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1c939303/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/impl/NiFiAuthorizationServiceTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/impl/NiFiAuthorizationServiceTest.java b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/impl/NiFiAuthorizationServiceTest.java
deleted file mode 100644
index 5c8b75a..0000000
--- a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/impl/NiFiAuthorizationServiceTest.java
+++ /dev/null
@@ -1,284 +0,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.
- */
-package org.apache.nifi.admin.service.impl;
-
-import org.junit.Ignore;
-
-/**
- *
- */
-@Ignore
-public class NiFiAuthorizationServiceTest {
-
-//    private static final String UNKNOWN_USER_IN_CACHE_DN = "unknown-user-in-cache-dn";
-//    private static final String PENDING_USER_DN = "pending-user-dn";
-//    private static final String DISABLED_USER_DN = "disabled-user-dn";
-//    private static final String UNKNOWN_USER_IN_IDENTITY_PROVIDER_DN = "unknown-user-in-identity-provider-dn";
-//    private static final String ACCESS_EXCEPTION_IN_IDENTITY_PROVIDER_DN = "access-exception-in-identity-provider-dn";
-//    private static final String UNABLE_TO_UPDATE_CACHE_DN = "unable-to-update-cache-dn";
-//    private static final String VERIFICATION_REQUIRED_DN = "verification-required-dn";
-//    private static final String VERIFICATION_NOT_REQUIRED_DN = "verification-not-required-dn";
-//    private static final String NEW_USER_DN = "new-user-dn";
-//    
-//    private UserService userService;
-//    private AuthorityProvider authorityProvider;
-//    private UserDAO userDAO;
-//
-//    @Before
-//    public void setup() throws Exception {
-//        // mock the web security properties
-//        NiFiProperties properties = Mockito.mock(NiFiProperties.class);
-//        Mockito.when(properties.getSupportNewAccountRequests()).thenReturn(Boolean.TRUE);
-//        Mockito.when(properties.getUserCredentialCacheDurationSeconds()).thenReturn(60);
-//
-//        // mock the authority provider
-//
-//        // mock the admin service
-//        userDAO = Mockito.mock(UserDAO.class);
-//        Mockito.doAnswer(new Answer() {
-//
-//            @Override
-//            public Object answer(InvocationOnMock invocation) throws Throwable {
-//                Object[] args = invocation.getArguments();
-//                String dn = (String) args[0];
-//
-//                NiFiUser user = null;
-//                switch (dn) {
-//                    case PENDING_USER_DN:
-//                        user = new NiFiUser();
-//                        user.setDn(dn);
-//                        user.setStatus(AccountStatus.PENDING);
-//                        break;
-//                    case DISABLED_USER_DN:
-//                        user = new NiFiUser();
-//                        user.setDn(dn);
-//                        user.setStatus(AccountStatus.DISABLED);
-//                        break;
-//                    case UNKNOWN_USER_IN_IDENTITY_PROVIDER_DN:
-//                    case UNABLE_TO_UPDATE_CACHE_DN:
-//                    case ACCESS_EXCEPTION_IN_IDENTITY_PROVIDER_DN:
-//                        user = new NiFiUser();
-//                        user.setDn(dn);
-//                        user.setStatus(AccountStatus.ACTIVE);
-//                        break;
-//                    case VERIFICATION_REQUIRED_DN: {
-//                        Calendar calendar = Calendar.getInstance();
-//                        calendar.add(Calendar.SECOND, -65);
-//                        user = new NiFiUser();
-//                        user.setDn(dn);
-//                        user.setStatus(AccountStatus.ACTIVE);
-//                        user.setLastVerified(calendar.getTime());
-//                        user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_ADMIN, Authority.ROLE_DFM));
-//                        break;
-//                    }
-//                    case VERIFICATION_NOT_REQUIRED_DN: {
-//                        Calendar calendar = Calendar.getInstance();
-//                        calendar.add(Calendar.SECOND, -5);
-//                        user = new NiFiUser();
-//                        user.setDn(dn);
-//                        user.setStatus(AccountStatus.ACTIVE);
-//                        user.setLastVerified(calendar.getTime());
-//                        user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_ADMIN, Authority.ROLE_DFM));
-//                        break;
-//                    }
-//                }
-//                return user;
-//            }
-//        }).when(userDAO).getUser(Mockito.anyString());
-//        Mockito.doAnswer(new Answer() {
-//
-//            @Override
-//            public Object answer(InvocationOnMock invocation) throws Throwable {
-//                Object[] args = invocation.getArguments();
-//                NiFiUser user = (NiFiUser) args[0];
-//
-//                if (UNABLE_TO_UPDATE_CACHE_DN.equals(user.getDn())) {
-//                    throw new AdministrationException();
-//                }
-//                return user;
-//            }
-//        }).when(userDAO).updateUser(Mockito.any(NiFiUser.class));
-//        Mockito.doNothing().when(userDAO).createUser(Mockito.any(NiFiUser.class));
-//
-//        // mock the authority provider
-//        authorityProvider = Mockito.mock(AuthorityProvider.class);
-//        Mockito.doAnswer(new Answer() {
-//
-//            @Override
-//            public Object answer(InvocationOnMock invocation) throws Throwable {
-//                Object[] args = invocation.getArguments();
-//                String dn = (String) args[0];
-//
-//                boolean hasDn = false;
-//                if (VERIFICATION_REQUIRED_DN.equals(dn) || NEW_USER_DN.equals(dn)) {
-//                    hasDn = true;
-//                }
-//                return hasDn;
-//            }
-//        }).when(authorityProvider).doesDnExist(Mockito.anyString());
-//        Mockito.doAnswer(new Answer() {
-//
-//            @Override
-//            public Object answer(InvocationOnMock invocation) throws Throwable {
-//                Object[] args = invocation.getArguments();
-//                String dn = (String) args[0];
-//
-//                Set<String> authorities = null;
-//                switch (dn) {
-//                    case VERIFICATION_REQUIRED_DN:
-//                    case NEW_USER_DN:
-//                        authorities = new HashSet<>();
-//                        authorities.add("ROLE_MONITOR");
-//                        break;
-//                    case DISABLED_USER_DN:
-//                        throw new UnknownIdentityException("Unable to find user");
-//                }
-//                return authorities;
-//            }
-//        }).when(authorityProvider).getAuthorities(Mockito.anyString());
-//
-//        // create an instance of the authorization service
-//        userService = new UserServiceImpl();
-//        ((UserServiceImpl) userService).setAuthorityProvider(authorityProvider);
-//        ((UserServiceImpl) userService).set(authorityProvider);
-//        
-////        authorizationService.setIdentityProvider(identityProvider);
-////        authorizationService.setAuthorityProvider(authorityProvider);
-////        authorizationService.setProperties(properties);
-//    }
-//
-//    /**
-//     * Ensures the authorization service correctly handles users who are
-//     * unknown.
-//     *
-//     * @throws Exception
-//     */
-//    @Test(expected = org.springframework.security.core.userdetails.UsernameNotFoundException.class)
-//    public void testUnknownUserInCache() throws Exception {
-//        authorizationService.loadUserByUsername(WebUtils.formatProxyDn(UNKNOWN_USER_IN_CACHE_DN));
-//    }
-//
-//    /**
-//     * Ensures the authorization service correctly handles users whose accounts
-//     * are PENDING.
-//     *
-//     * @throws Exception
-//     */
-//    @Test(expected = nifi.admin.service.AccountPendingException.class)
-//    public void testPendingUser() throws Exception {
-//        authorizationService.loadUserByUsername(WebUtils.formatProxyDn(PENDING_USER_DN));
-//    }
-//
-//    /**
-//     * Ensures the authorization service correctly handles users whose accounts
-//     * are DISABLED.
-//     *
-//     * @throws Exception
-//     */
-//    @Test(expected = org.springframework.security.authentication.DisabledException.class)
-//    public void testDisabledUser() throws Exception {
-//        authorizationService.loadUserByUsername(WebUtils.formatProxyDn(DISABLED_USER_DN));
-//    }
-//
-//    /**
-//     * Ensures the authorization service correctly handles users whose are in
-//     * the cache but have been removed from the identity provider.
-//     *
-//     * @throws Exception
-//     */
-//    @Test(expected = org.springframework.security.authentication.DisabledException.class)
-//    public void testUnknownUserInIdentityProvider() throws Exception {
-//        authorizationService.loadUserByUsername(WebUtils.formatProxyDn(UNKNOWN_USER_IN_IDENTITY_PROVIDER_DN));
-//    }
-//
-//    /**
-//     * Ensures the authorization service correctly handles cases when the cache
-//     * is unable to be updated.
-//     *
-//     * @throws Exception
-//     */
-//    @Test(expected = org.springframework.security.authentication.AuthenticationServiceException.class)
-//    public void testUnableToUpdateCache() throws Exception {
-//        authorizationService.loadUserByUsername(WebUtils.formatProxyDn(UNABLE_TO_UPDATE_CACHE_DN));
-//    }
-//
-//    /**
-//     * Ensures the authorization service correctly handles cases when the
-//     * identity provider has an access exception.
-//     *
-//     * @throws Exception
-//     */
-//    @Test(expected = org.springframework.security.authentication.AuthenticationServiceException.class)
-//    public void testUnableToAccessIdentity() throws Exception {
-//        authorizationService.loadUserByUsername(WebUtils.formatProxyDn(ACCESS_EXCEPTION_IN_IDENTITY_PROVIDER_DN));
-//    }
-//
-//    /**
-//     * Ensures that user authorities are properly loaded from the authority
-//     * provider.
-//     *
-//     * @throws Exception
-//     */
-//    @Test
-//    public void testVerificationRequiredUser() throws Exception {
-//        NiFiUserDetails userDetails = (NiFiUserDetails) authorizationService.loadUserByUsername(WebUtils.formatProxyDn(VERIFICATION_REQUIRED_DN));
-//        NiFiUser user = userDetails.getNiFiUser();
-//        Mockito.verify(authorityProvider).getAuthorities(VERIFICATION_REQUIRED_DN);
-//
-//        // ensure the user details
-//        Assert.assertEquals(VERIFICATION_REQUIRED_DN, user.getDn());
-//        Assert.assertEquals(1, user.getAuthorities().size());
-//        Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_MONITOR));
-//    }
-//
-//    /**
-//     * Ensures that user authorities are not loaded when the cache is still
-//     * valid.
-//     *
-//     * @throws Exception
-//     */
-//    @Test
-//    public void testVerificationNotRequiredUser() throws Exception {
-//        NiFiUserDetails userDetails = (NiFiUserDetails) authorizationService.loadUserByUsername(WebUtils.formatProxyDn(VERIFICATION_NOT_REQUIRED_DN));
-//        NiFiUser user = userDetails.getNiFiUser();
-//        Mockito.verify(authorityProvider, Mockito.never()).getAuthorities(VERIFICATION_NOT_REQUIRED_DN);
-//
-//        // ensure the user details
-//        Assert.assertEquals(VERIFICATION_NOT_REQUIRED_DN, user.getDn());
-//        Assert.assertEquals(2, user.getAuthorities().size());
-//        Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_ADMIN));
-//        Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_DFM));
-//    }
-//
-//    /**
-//     * Ensures that new users are automatically created when the authority
-//     * provider has their authorities.
-//     *
-//     * @throws Exception
-//     */
-//    @Test
-//    public void testNewUser() throws Exception {
-//        NiFiUserDetails userDetails = (NiFiUserDetails) authorizationService.loadUserByUsername(WebUtils.formatProxyDn(NEW_USER_DN));
-//        NiFiUser user = userDetails.getNiFiUser();
-//        Mockito.verify(authorityProvider).getAuthorities(NEW_USER_DN);
-//
-//        // ensure the user details
-//        Assert.assertEquals(NEW_USER_DN, user.getDn());
-//        Assert.assertEquals(1, user.getAuthorities().size());
-//        Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_MONITOR));
-//    }
-}


[08/14] incubator-nifi git commit: NIFI-65: - Adding methods to the AuthorityProvider to authorize the downloading of content.

Posted by mc...@apache.org.
NIFI-65:
- Adding methods to the AuthorityProvider to authorize the downloading of content.

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

Branch: refs/heads/develop
Commit: 71ab084d399bd09dd273942f42bceda66cf410d6
Parents: e67eb4f
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Dec 23 09:46:35 2014 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Dec 23 09:46:35 2014 -0500

----------------------------------------------------------------------
 .../cluster/authorization/NodeAuthorizationProvider.java     | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/71ab084d/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/NodeAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/NodeAuthorizationProvider.java b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/NodeAuthorizationProvider.java
index c81e9d0..840422f 100644
--- a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/NodeAuthorizationProvider.java
+++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/NodeAuthorizationProvider.java
@@ -22,12 +22,15 @@ import org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.Socket;
+import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import org.apache.nifi.authorization.Authority;
 import org.apache.nifi.authorization.AuthorityProvider;
 import org.apache.nifi.authorization.AuthorityProviderConfigurationContext;
 import org.apache.nifi.authorization.AuthorityProviderInitializationContext;
+import org.apache.nifi.authorization.DownloadAuthorization;
 import org.apache.nifi.authorization.annotation.AuthorityProviderContext;
 import org.apache.nifi.authorization.exception.AuthorityAccessException;
 import org.apache.nifi.authorization.exception.IdentityAlreadyExistsException;
@@ -283,6 +286,11 @@ public class NodeAuthorizationProvider implements AuthorityProvider, Application
     }
 
     @Override
+    public DownloadAuthorization authorizeDownload(List<String> dnChain, Map<String, String> attributes) throws UnknownIdentityException, AuthorityAccessException {
+        return DownloadAuthorization.approved();
+    }
+
+    @Override
     public String getGroupForUser(String dn) throws UnknownIdentityException, AuthorityAccessException {
         // create message
         final GetGroupForUserMessage msg = new GetGroupForUserMessage();


[06/14] incubator-nifi git commit: NIFI-65: - Adding methods to the AuthorityProvider to authorize the downloading of content.

Posted by mc...@apache.org.
NIFI-65:
- Adding methods to the AuthorityProvider to authorize the downloading of content.

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

Branch: refs/heads/develop
Commit: 1c7208125796ff3c2dcc540ecf7f9c973396664c
Parents: 2fed138
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Dec 23 09:33:17 2014 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Dec 23 09:33:17 2014 -0500

----------------------------------------------------------------------
 .../nifi/integration/util/NiFiTestAuthorizationProvider.java  | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1c720812/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestAuthorizationProvider.java b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestAuthorizationProvider.java
index d1e7512..48f18a9 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestAuthorizationProvider.java
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestAuthorizationProvider.java
@@ -19,6 +19,7 @@ package org.apache.nifi.integration.util;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import org.apache.nifi.authorization.Authority;
@@ -29,6 +30,7 @@ import org.apache.nifi.authorization.exception.AuthorityAccessException;
 import org.apache.nifi.authorization.exception.ProviderCreationException;
 import org.apache.nifi.authorization.exception.UnknownIdentityException;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.authorization.DownloadAuthorization;
 
 /**
  *
@@ -169,4 +171,9 @@ public class NiFiTestAuthorizationProvider implements AuthorityProvider {
     public void ungroup(String group) throws UnknownIdentityException, AuthorityAccessException {
     }
 
+    @Override
+    public DownloadAuthorization authorizeDownload(List<String> dnChain, Map<String, String> attributes) throws UnknownIdentityException, AuthorityAccessException {
+        return DownloadAuthorization.approved();
+    }
+
 }


[13/14] incubator-nifi git commit: NIFI-65: - Addressed issues submitting replays while clustered.

Posted by mc...@apache.org.
NIFI-65:
- Addressed issues submitting replays while clustered.

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

Branch: refs/heads/develop
Commit: 1a34e75c5b9cd29cf4496a4ff0d10b2b0519dc76
Parents: 2436b53
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Dec 23 14:56:08 2014 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Dec 23 14:56:08 2014 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/nifi/web/api/ProvenanceResource.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1a34e75c/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.java b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.java
index 24bdf0c..f0b38a1 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.java
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.java
@@ -166,7 +166,7 @@ public class ProvenanceResource extends ApplicationResource {
     public Response submitReplay(
             @Context HttpServletRequest httpServletRequest,
             @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
-            @QueryParam("clusterNodeId") String clusterNodeId,
+            @FormParam("clusterNodeId") String clusterNodeId,
             @FormParam("eventId") LongParameter eventId) {
 
         // ensure the event id is specified
@@ -190,7 +190,7 @@ public class ProvenanceResource extends ApplicationResource {
                 targetNodes.add(targetNode.getNodeId());
 
                 // replicate the request to the specific node
-                return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders(), targetNodes).getResponse();
+                return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), getRequestParameters(true), getHeaders(), targetNodes).getResponse();
             }
         }
 


[02/14] incubator-nifi git commit: NIFI-65: - Renaming artifact name.

Posted by mc...@apache.org.
NIFI-65:
- Renaming artifact name.

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

Branch: refs/heads/develop
Commit: 2df03143b9c2e419788a80271fceb3c3f5aa9aa9
Parents: 1c93930
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Dec 23 09:30:47 2014 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Dec 23 09:30:47 2014 -0500

----------------------------------------------------------------------
 .../framework-bundle/framework/file-authorization-provider/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2df03143/nar-bundles/framework-bundle/framework/file-authorization-provider/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/file-authorization-provider/pom.xml b/nar-bundles/framework-bundle/framework/file-authorization-provider/pom.xml
index c96f739..6912108 100644
--- a/nar-bundles/framework-bundle/framework/file-authorization-provider/pom.xml
+++ b/nar-bundles/framework-bundle/framework/file-authorization-provider/pom.xml
@@ -23,7 +23,7 @@
 
     <artifactId>file-authorization-provider</artifactId>
     <version>0.0.1-SNAPSHOT</version>
-    <name>Authorization Provider: File</name>
+    <name>NiFi File Authorization Provider</name>
 
     <build>
         <resources>


[07/14] incubator-nifi git commit: NIFI-65: - Setting the dn chain on the current NiFiUser. - Adding unit tests.

Posted by mc...@apache.org.
NIFI-65:
- Setting the dn chain on the current NiFiUser.
- Adding unit tests.

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

Branch: refs/heads/develop
Commit: e67eb4f53bca1b63326efc35b86b2f7a36318d7e
Parents: 1c72081
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Dec 23 09:33:43 2014 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Dec 23 09:33:43 2014 -0500

----------------------------------------------------------------------
 .../authorization/NiFiAuthorizationService.java |  16 ++
 .../NiFiAuthorizationServiceTest.java           | 250 +++++++++++++++++++
 2 files changed, 266 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e67eb4f5/nar-bundles/framework-bundle/framework/web/web-security/src/main/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationService.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/web-security/src/main/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationService.java b/nar-bundles/framework-bundle/framework/web/web-security/src/main/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationService.java
index 6417a05..33c4eb6 100644
--- a/nar-bundles/framework-bundle/framework/web/web-security/src/main/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationService.java
+++ b/nar-bundles/framework-bundle/framework/web/web-security/src/main/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationService.java
@@ -72,6 +72,8 @@ public class NiFiAuthorizationService implements UserDetailsService {
             throw new UntrustedProxyException("Malformed proxy chain.");
         }
 
+        NiFiUser proxy = null;
+        
         // process each part of the proxy chain
         for (final Iterator<String> dnIter = dnList.iterator(); dnIter.hasNext();) {
             final String dn = dnIter.next();
@@ -88,6 +90,14 @@ public class NiFiAuthorizationService implements UserDetailsService {
                         logger.warn(String.format("Proxy '%s' must have '%s' authority. Current authorities: %s", dn, Authority.ROLE_PROXY.toString(), StringUtils.join(user.getAuthorities(), ", ")));
                         throw new UntrustedProxyException(String.format("Untrusted proxy '%s' must be authorized with '%s'.", dn, Authority.ROLE_PROXY.toString()));
                     }
+                    
+                    // if we've already encountered a proxy, update the chain
+                    if (proxy != null) {
+                        user.setChain(proxy);
+                    }
+                    
+                    // record this user as the proxy for the next user in the chain
+                    proxy = user;
                 } catch (UsernameNotFoundException unfe) {
                     // if this proxy is a new user, conditionally create a new account automatically
                     if (properties.getSupportNewAccountRequests()) {
@@ -112,6 +122,12 @@ public class NiFiAuthorizationService implements UserDetailsService {
                 }
             } else {
                 userDetails = getNiFiUserDetails(dn);
+                
+                // if we've already encountered a proxy, update the chain
+                if (proxy != null) {
+                    final NiFiUser user = userDetails.getNiFiUser();
+                    user.setChain(proxy);
+                }
             }
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e67eb4f5/nar-bundles/framework-bundle/framework/web/web-security/src/test/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationServiceTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/web-security/src/test/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationServiceTest.java b/nar-bundles/framework-bundle/framework/web/web-security/src/test/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationServiceTest.java
new file mode 100644
index 0000000..057a854
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/web/web-security/src/test/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationServiceTest.java
@@ -0,0 +1,250 @@
+/*
+ * 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.web.security.authorization;
+
+import org.apache.nifi.admin.service.AccountDisabledException;
+import org.apache.nifi.admin.service.AccountNotFoundException;
+import org.apache.nifi.admin.service.AccountPendingException;
+import org.apache.nifi.admin.service.AdministrationException;
+import org.apache.nifi.admin.service.UserService;
+import org.apache.nifi.authorization.Authority;
+import org.apache.nifi.user.NiFiUser;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.web.security.DnUtils;
+import org.apache.nifi.web.security.UntrustedProxyException;
+import org.apache.nifi.web.security.user.NiFiUserDetails;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.springframework.security.authentication.AccountStatusException;
+import org.springframework.security.authentication.AuthenticationServiceException;
+import org.springframework.security.core.userdetails.UsernameNotFoundException;
+
+
+/**
+ * Test case for NiFiAuthorizationService.
+ */
+public class NiFiAuthorizationServiceTest {
+    
+    private static final String USER = "user";
+    private static final String PROXY = "proxy";
+    private static final String PROXY_PROXY = "proxy-proxy";
+    private static final String USER_NOT_FOUND = "user-not-found";
+    private static final String USER_DISABLED = "user-disabled";
+    private static final String USER_PENDING = "user-pending";
+    private static final String USER_ADMIN_EXCEPTION = "user-admin-exception";
+    private static final String PROXY_NOT_FOUND = "proxy-not-found";
+    
+    private NiFiAuthorizationService authorizationService;
+    private UserService userService;
+    
+    @Before
+    public void setup() throws Exception {
+        // mock the web security properties
+        final NiFiProperties properties = Mockito.mock(NiFiProperties.class);
+        Mockito.when(properties.getSupportNewAccountRequests()).thenReturn(Boolean.TRUE);
+        
+        userService = Mockito.mock(UserService.class);
+        Mockito.doReturn(null).when(userService).createPendingUserAccount(Mockito.anyString(), Mockito.anyString());
+        Mockito.doAnswer(new Answer() {
+            @Override
+            public Object answer(InvocationOnMock invocation) throws Throwable {
+                Object[] args = invocation.getArguments();
+                String dn = (String) args[0];
+                
+                if (null != dn) {
+                    switch (dn) {
+                        case USER_NOT_FOUND:
+                        case PROXY_NOT_FOUND:
+                            throw new AccountNotFoundException("");
+                        case USER_DISABLED:
+                            throw new AccountDisabledException("");
+                        case USER_PENDING:
+                            throw new AccountPendingException("");
+                        case USER_ADMIN_EXCEPTION:
+                            throw new AdministrationException();
+                        case USER:
+                            final NiFiUser monitor = new NiFiUser();
+                            monitor.setDn(dn);
+                            monitor.getAuthorities().add(Authority.ROLE_MONITOR);
+                            return monitor;
+                        case PROXY:
+                        case PROXY_PROXY:
+                            final NiFiUser proxy = new NiFiUser();
+                            proxy.setDn(dn);
+                            proxy.getAuthorities().add(Authority.ROLE_PROXY);
+                            return proxy;
+                    }
+                }
+                
+                return null;
+            }
+        }).when(userService).checkAuthorization(Mockito.anyString());
+        
+        // create the authorization service
+        authorizationService = new NiFiAuthorizationService();
+        authorizationService.setProperties(properties);
+        authorizationService.setUserService(userService);
+    }
+    
+    /**
+     * Ensures the authorization service correctly handles users invalid dn chain.
+     *
+     * @throws Exception
+     */
+    @Test(expected = UntrustedProxyException.class)
+    public void testInvalidDnChain() throws Exception {
+        authorizationService.loadUserByUsername(USER);
+    }
+    
+    /**
+     * Ensures the authorization service correctly handles account not found.
+     *
+     * @throws Exception
+     */
+    @Test(expected = UsernameNotFoundException.class)
+    public void testAccountNotFound() throws Exception {
+        authorizationService.loadUserByUsername(DnUtils.formatProxyDn(USER_NOT_FOUND));
+    }
+    
+    /**
+     * Ensures the authorization service correctly handles account disabled.
+     *
+     * @throws Exception
+     */
+    @Test(expected = AccountStatusException.class)
+    public void testAccountDisabled() throws Exception {
+        authorizationService.loadUserByUsername(DnUtils.formatProxyDn(USER_DISABLED));
+    }
+    
+    /**
+     * Ensures the authorization service correctly handles account pending.
+     *
+     * @throws Exception
+     */
+    @Test(expected = AccountStatusException.class)
+    public void testAccountPending() throws Exception {
+        authorizationService.loadUserByUsername(DnUtils.formatProxyDn(USER_PENDING));
+    }
+    
+    /**
+     * Ensures the authorization service correctly handles account administration exception.
+     *
+     * @throws Exception
+     */
+    @Test(expected = AuthenticationServiceException.class)
+    public void testAccountAdminException() throws Exception {
+        authorizationService.loadUserByUsername(DnUtils.formatProxyDn(USER_ADMIN_EXCEPTION));
+    }
+    
+    /**
+     * Tests the case when there is no proxy.
+     * 
+     * @throws Exception 
+     */
+    @Test
+    public void testNoProxy() throws Exception {
+        final NiFiUserDetails details = (NiFiUserDetails) authorizationService.loadUserByUsername(DnUtils.formatProxyDn(USER));
+        final NiFiUser user = details.getNiFiUser();
+        
+        Assert.assertEquals(USER, user.getDn());
+        Assert.assertNull(user.getChain());
+    }
+    
+    /**
+     * Tests the case when the proxy does not have ROLE_PROXY.
+     * 
+     * @throws Exception 
+     */
+    @Test(expected = UntrustedProxyException.class)
+    public void testInvalidProxy() throws Exception {
+        final String dnChain = DnUtils.formatProxyDn(USER) + DnUtils.formatProxyDn(USER);
+        authorizationService.loadUserByUsername(dnChain);
+    }
+    
+    /**
+     * Ensures the authorization service correctly handles proxy not found by attempting
+     * to create an account request for the proxy.
+     *
+     * @throws Exception
+     */
+    @Test(expected = UsernameNotFoundException.class)
+    public void testProxyNotFound() throws Exception {
+        try {
+            final String dnChain = DnUtils.formatProxyDn(USER) + DnUtils.formatProxyDn(PROXY_NOT_FOUND);
+            authorizationService.loadUserByUsername(DnUtils.formatProxyDn(dnChain));
+        } finally {
+            Mockito.verify(userService).createPendingUserAccount(Mockito.eq(PROXY_NOT_FOUND), Mockito.anyString());
+        }
+    }
+    
+    /**
+     * Tests the case when there is a proxy.
+     * 
+     * @throws Exception 
+     */
+    @Test
+    public void testProxy() throws Exception {
+        final String dnChain = DnUtils.formatProxyDn(USER) + DnUtils.formatProxyDn(PROXY);
+        final NiFiUserDetails details = (NiFiUserDetails) authorizationService.loadUserByUsername(dnChain);
+        final NiFiUser user = details.getNiFiUser();
+        
+        // verify the user
+        Assert.assertEquals(USER, user.getDn());
+        Assert.assertNotNull(user.getChain());
+        
+        // get the proxy
+        final NiFiUser proxy = user.getChain();
+        
+        // verify the proxy
+        Assert.assertEquals(PROXY, proxy.getDn());
+        Assert.assertNull(proxy.getChain());
+    }
+    
+    /**
+     * Tests the case when there is are multiple proxies.
+     * 
+     * @throws Exception 
+     */
+    @Test
+    public void testProxyProxy() throws Exception {
+        final String dnChain = DnUtils.formatProxyDn(USER) + DnUtils.formatProxyDn(PROXY) + DnUtils.formatProxyDn(PROXY_PROXY);
+        final NiFiUserDetails details = (NiFiUserDetails) authorizationService.loadUserByUsername(dnChain);
+        final NiFiUser user = details.getNiFiUser();
+        
+        // verify the user
+        Assert.assertEquals(USER, user.getDn());
+        Assert.assertNotNull(user.getChain());
+        
+        // get the proxy
+        NiFiUser proxy = user.getChain();
+        
+        // verify the proxy
+        Assert.assertEquals(PROXY, proxy.getDn());
+        Assert.assertNotNull(proxy.getChain());
+        
+        // get the proxies proxy
+        proxy = proxy.getChain();
+        
+        // verify the proxies proxy
+        Assert.assertEquals(PROXY_PROXY, proxy.getDn());
+        Assert.assertNull(proxy.getChain());
+    }
+}


[03/14] incubator-nifi git commit: NIFI-65: - Adding methods to the AuthorityProvider to authorize the downloading of content.

Posted by mc...@apache.org.
NIFI-65:
- Adding methods to the AuthorityProvider to authorize the downloading of content.

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

Branch: refs/heads/develop
Commit: 418d6b03b2a2c81597954b6c29469e180c3e780e
Parents: 2df0314
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Dec 23 09:31:46 2014 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Dec 23 09:31:46 2014 -0500

----------------------------------------------------------------------
 .../nifi/authorization/AuthorityProvider.java   | 18 +++++
 .../authorization/DownloadAuthorization.java    | 85 ++++++++++++++++++++
 2 files changed, 103 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/418d6b03/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorityProvider.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorityProvider.java b/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorityProvider.java
index cd96a99..da536c2 100644
--- a/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorityProvider.java
+++ b/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorityProvider.java
@@ -16,6 +16,8 @@
  */
 package org.apache.nifi.authorization;
 
+import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import org.apache.nifi.authorization.exception.AuthorityAccessException;
 import org.apache.nifi.authorization.exception.IdentityAlreadyExistsException;
@@ -137,6 +139,22 @@ public interface AuthorityProvider {
     void ungroup(String group) throws AuthorityAccessException;
 
     /**
+     * Determines whether the user in the specified dnChain should be able to 
+     * download the content for the flowfile with the specified attributes.
+     * 
+     * The last dn in the chain is the end user that the request was issued on 
+     * behalf of. The previous dn's in the chain represent entities proxying the 
+     * user's request.
+     * 
+     * @param dnChain
+     * @param attributes
+     * @return
+     * @throws UnknownIdentityException
+     * @throws AuthorityAccessException 
+     */
+    DownloadAuthorization authorizeDownload(List<String> dnChain, Map<String, String> attributes) throws UnknownIdentityException, AuthorityAccessException;
+    
+    /**
      * Called immediately after instance creation for implementers to perform
      * additional setup
      *

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/418d6b03/nifi-api/src/main/java/org/apache/nifi/authorization/DownloadAuthorization.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/authorization/DownloadAuthorization.java b/nifi-api/src/main/java/org/apache/nifi/authorization/DownloadAuthorization.java
new file mode 100644
index 0000000..3855817
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/authorization/DownloadAuthorization.java
@@ -0,0 +1,85 @@
+/*
+ * 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;
+
+/**
+ * Represents a decision whether authorization is granted to download content.
+ */
+public class DownloadAuthorization {
+
+    private static enum Result {
+        Approved,
+        Denied;
+    };
+
+    private static final DownloadAuthorization APPROVED = new DownloadAuthorization(Result.Approved, null);
+
+    private final Result result;
+    private final String explanation;
+
+    /**
+     * Creates a new DownloadAuthorization with the specified result and explanation.
+     * 
+     * @param result
+     * @param explanation 
+     */
+    private DownloadAuthorization(Result result, String explanation) {
+        if (Result.Denied.equals(result) && explanation == null) {
+            throw new IllegalArgumentException("An explanation is request when the download request is denied.");
+        }
+
+        this.result = result;
+        this.explanation = explanation;
+    }
+
+    /**
+     * Whether or not the download request is approved.
+     * 
+     * @return 
+     */
+    public boolean isApproved() {
+        return Result.Approved.equals(result);
+    }
+
+    /**
+     * If the download request is denied, the reason why. Null otherwise.
+     * 
+     * @return 
+     */
+    public String getExplanation() {
+        return explanation;
+    }
+
+    /**
+     * Creates a new approved DownloadAuthorization.
+     * 
+     * @return 
+     */
+    public static DownloadAuthorization approved() {
+        return APPROVED;
+    }
+
+    /**
+     * Creates a new denied DownloadAuthorization with the specified exlanation.
+     * 
+     * @param explanation
+     * @return 
+     */
+    public static DownloadAuthorization denied(String explanation) {
+        return new DownloadAuthorization(Result.Denied, explanation);
+    }
+}


[11/14] incubator-nifi git commit: NIFI-65: - Code clean up. - Updating authorizeDownload(...) to accept the dnChain in the appropriate order.

Posted by mc...@apache.org.
NIFI-65:
- Code clean up.
- Updating authorizeDownload(...) to accept the dnChain in the appropriate order.

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

Branch: refs/heads/develop
Commit: 469502f30ca01953bceabca590bea8e183ed2c7d
Parents: 7a29166
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Dec 23 13:00:38 2014 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Dec 23 13:00:38 2014 -0500

----------------------------------------------------------------------
 .../java/org/apache/nifi/web/controller/ControllerFacade.java | 5 ++---
 .../java/org/apache/nifi/authorization/AuthorityProvider.java | 7 ++++---
 .../org/apache/nifi/authorization/DownloadAuthorization.java  | 5 +++--
 3 files changed, 9 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/469502f3/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
index 99440bc..07e3150 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
@@ -26,7 +26,6 @@ import java.util.Comparator;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -796,13 +795,13 @@ public class ControllerFacade implements ControllerServiceProvider {
             final Map<String, String> attributes = event.getAttributes();
 
             // calculate the dn chain
-            final LinkedList<String> dnChain = new LinkedList<>();
+            final List<String> dnChain = new ArrayList<>();
 
             // build the dn chain
             NiFiUser chainedUser = user;
             do {
                 // add the entry for this user
-                dnChain.push(chainedUser.getDn());
+                dnChain.add(chainedUser.getDn());
 
                 // go to the next user in the chain
                 chainedUser = chainedUser.getChain();

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/469502f3/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorityProvider.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorityProvider.java b/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorityProvider.java
index da536c2..7754c35 100644
--- a/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorityProvider.java
+++ b/nifi-api/src/main/java/org/apache/nifi/authorization/AuthorityProvider.java
@@ -142,9 +142,10 @@ public interface AuthorityProvider {
      * Determines whether the user in the specified dnChain should be able to 
      * download the content for the flowfile with the specified attributes.
      * 
-     * The last dn in the chain is the end user that the request was issued on 
-     * behalf of. The previous dn's in the chain represent entities proxying the 
-     * user's request.
+     * The first dn in the chain is the end user that the request was issued on 
+     * behalf of. The subsequent dn's in the chain represent entities proxying
+     * the user's request with the last being the proxy that sent the current
+     * request.
      * 
      * @param dnChain
      * @param attributes

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/469502f3/nifi-api/src/main/java/org/apache/nifi/authorization/DownloadAuthorization.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/authorization/DownloadAuthorization.java b/nifi-api/src/main/java/org/apache/nifi/authorization/DownloadAuthorization.java
index 3855817..08695fa 100644
--- a/nifi-api/src/main/java/org/apache/nifi/authorization/DownloadAuthorization.java
+++ b/nifi-api/src/main/java/org/apache/nifi/authorization/DownloadAuthorization.java
@@ -39,7 +39,7 @@ public class DownloadAuthorization {
      */
     private DownloadAuthorization(Result result, String explanation) {
         if (Result.Denied.equals(result) && explanation == null) {
-            throw new IllegalArgumentException("An explanation is request when the download request is denied.");
+            throw new IllegalArgumentException("An explanation is required when the download request is denied.");
         }
 
         this.result = result;
@@ -74,10 +74,11 @@ public class DownloadAuthorization {
     }
 
     /**
-     * Creates a new denied DownloadAuthorization with the specified exlanation.
+     * Creates a new denied DownloadAuthorization with the specified explanation.
      * 
      * @param explanation
      * @return 
+     * @throws IllegalArgumentException     if explanation is null
      */
     public static DownloadAuthorization denied(String explanation) {
         return new DownloadAuthorization(Result.Denied, explanation);


[12/14] incubator-nifi git commit: NIFI-65: - Code clean up.

Posted by mc...@apache.org.
NIFI-65:
- Code clean up.

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

Branch: refs/heads/develop
Commit: 2436b5338ecf1752c0f5b51646f59c9a985ceb9c
Parents: 469502f
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Dec 23 13:46:48 2014 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Dec 23 13:46:48 2014 -0500

----------------------------------------------------------------------
 .../authorization/NiFiAuthorizationService.java  | 19 +++++++++++++------
 1 file changed, 13 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2436b533/nar-bundles/framework-bundle/framework/web/web-security/src/main/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationService.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/web-security/src/main/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationService.java b/nar-bundles/framework-bundle/framework/web/web-security/src/main/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationService.java
index 33c4eb6..dec7c4b 100644
--- a/nar-bundles/framework-bundle/framework/web/web-security/src/main/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationService.java
+++ b/nar-bundles/framework-bundle/framework/web/web-security/src/main/java/org/apache/nifi/web/security/authorization/NiFiAuthorizationService.java
@@ -51,10 +51,12 @@ public class NiFiAuthorizationService implements UserDetailsService {
     private NiFiProperties properties;
 
     /**
-     * Loads the user details for the specified dn. Method must be synchronized
-     * since multiple requests from the same user may be sent simultaneously.
-     * Since we don't want to run the account verification process multiple for
-     * the same user, we treat each request atomically.
+     * Loads the user details for the specified dn. 
+     * 
+     * Synchronizing because we want each request to be authorized atomically since
+     * each may contain any number of DNs. We wanted an access decision made
+     * for each individual request as a whole (without other request potentially
+     * impacting it).
      *
      * @param rawProxyChain
      * @return
@@ -111,6 +113,12 @@ public class NiFiAuthorizationService implements UserDetailsService {
                             throw new UsernameNotFoundException(String.format("An account request was generated for the proxy '%s'.", dn));
                         } catch (AdministrationException ae) {
                             throw new AuthenticationServiceException(String.format("Unable to create an account request for '%s': %s", dn, ae.getMessage()), ae);
+                        } catch (IllegalArgumentException iae) {
+                            // check then modified... account didn't exist when getting the user details but did when
+                            // attempting to auto create the user account request
+                            final String message = String.format("Account request was already submitted for '%s'", dn);
+                            logger.warn(message);
+                            throw new AccountStatusException(message) {};
                         }
                     } else {
                         logger.warn(String.format("Untrusted proxy '%s' must be authorized with '%s' authority: %s", dn, Authority.ROLE_PROXY.toString(), unfe.getMessage()));
@@ -147,8 +155,7 @@ public class NiFiAuthorizationService implements UserDetailsService {
         } catch (AdministrationException ase) {
             throw new AuthenticationServiceException(String.format("An error occurred while accessing the user credentials for '%s': %s", dn, ase.getMessage()), ase);
         } catch (AccountDisabledException | AccountPendingException e) {
-            throw new AccountStatusException(e.getMessage(), e) {
-            };
+            throw new AccountStatusException(e.getMessage(), e) {};
         } catch (AccountNotFoundException anfe) {
             throw new UsernameNotFoundException(anfe.getMessage());
         }


[05/14] incubator-nifi git commit: NIFI-65: - Calling the userService to verify the user has authorization to download content.

Posted by mc...@apache.org.
NIFI-65:
- Calling the userService to verify the user has authorization to download content.

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

Branch: refs/heads/develop
Commit: 2fed138888fbc4d4ae4c93f13c9f9cf81dfcae92
Parents: e1ffbdf
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Dec 23 09:32:54 2014 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Dec 23 09:32:54 2014 -0500

----------------------------------------------------------------------
 .../nifi/web/controller/ControllerFacade.java   | 31 ++++++++++++++++++++
 .../src/main/resources/nifi-web-api-context.xml |  1 +
 2 files changed, 32 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2fed1388/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
index 2c2d4dc..99440bc 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
@@ -26,6 +26,7 @@ import java.util.Comparator;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -111,8 +112,11 @@ import org.apache.nifi.web.util.DownloadableContent;
 
 import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.admin.service.UserService;
+import org.apache.nifi.authorization.DownloadAuthorization;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.springframework.security.access.AccessDeniedException;
 
 /**
  *
@@ -124,6 +128,7 @@ public class ControllerFacade implements ControllerServiceProvider {
     // nifi components
     private FlowController flowController;
     private FlowService flowService;
+    private UserService userService;
 
     // properties
     private NiFiProperties properties;
@@ -787,6 +792,28 @@ public class ControllerFacade implements ControllerServiceProvider {
                 throw new ResourceNotFoundException("Unable to find the specified event.");
             }
 
+            // get the flowfile attributes
+            final Map<String, String> attributes = event.getAttributes();
+
+            // calculate the dn chain
+            final LinkedList<String> dnChain = new LinkedList<>();
+
+            // build the dn chain
+            NiFiUser chainedUser = user;
+            do {
+                // add the entry for this user
+                dnChain.push(chainedUser.getDn());
+
+                // go to the next user in the chain
+                chainedUser = chainedUser.getChain();
+            } while (chainedUser != null);
+
+            // ensure the users in this chain are allowed to download this content
+            final DownloadAuthorization downloadAuthorization = userService.authorizeDownload(dnChain, attributes);
+            if (!downloadAuthorization.isApproved()) {
+                throw new AccessDeniedException(downloadAuthorization.getExplanation());
+            }
+            
             // get the filename and fall back to the idnetifier (should never happen)
             String filename = event.getAttributes().get(CoreAttributes.FILENAME.key());
             if (filename == null) {
@@ -1329,6 +1356,10 @@ public class ControllerFacade implements ControllerServiceProvider {
         this.properties = properties;
     }
 
+    public void setUserService(UserService userService) {
+        this.userService = userService;
+    }
+
     public void setFlowService(FlowService flowService) {
         this.flowService = flowService;
     }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2fed1388/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/resources/nifi-web-api-context.xml b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
index 484ceff..39677ca 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
@@ -80,6 +80,7 @@
         <property name="properties" ref="nifiProperties"/>
         <property name="flowController" ref="flowController"/>
         <property name="flowService" ref="flowService"/>
+        <property name="userService" ref="userService"/>
         <property name="dtoFactory" ref="dtoFactory"/>
     </bean>
     <bean id="serviceFacade" class="org.apache.nifi.web.StandardNiFiServiceFacade">


[09/14] incubator-nifi git commit: NIFI-65: - Only requiring ROLE_PROVENANCE for downloading content as the AuthorityProvider is able to make authorization decisions based on FlowFile attributes. - Requiring ROLE_PROVENANCE and ROLE_DFM to submit a repla

Posted by mc...@apache.org.
NIFI-65:
- Only requiring ROLE_PROVENANCE for downloading content as the AuthorityProvider is able to make authorization decisions based on FlowFile attributes.
- Requiring ROLE_PROVENANCE and ROLE_DFM to submit a replay request.

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

Branch: refs/heads/develop
Commit: 1f4ae7306584d3cd524c08c19886001456df5fce
Parents: 71ab084
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Dec 23 10:27:09 2014 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Dec 23 10:27:09 2014 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/nifi/web/api/ProvenanceResource.java  | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1f4ae730/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.java b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.java
index 2da7828..24bdf0c 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.java
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.java
@@ -160,7 +160,7 @@ public class ProvenanceResource extends ApplicationResource {
      */
     @POST
     @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-    @PreAuthorize("hasRole('ROLE_PROVENANCE')")
+    @PreAuthorize("hasRole('ROLE_PROVENANCE') and hasRole('ROLE_DFM')")
     @Path("/replays")
     @TypeHint(ProvenanceEventEntity.class)
     public Response submitReplay(
@@ -231,7 +231,7 @@ public class ProvenanceResource extends ApplicationResource {
     @GET
     @Produces(MediaType.APPLICATION_OCTET_STREAM)
     @Path("/events/{id}/content/input")
-    @PreAuthorize("hasRole('ROLE_PROVENANCE') and hasRole('ROLE_DFM')")
+    @PreAuthorize("hasRole('ROLE_PROVENANCE')")
     public Response getInputContent(
             @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
             @QueryParam("clusterNodeId") String clusterNodeId,
@@ -305,7 +305,7 @@ public class ProvenanceResource extends ApplicationResource {
     @GET
     @Produces(MediaType.APPLICATION_OCTET_STREAM)
     @Path("/events/{id}/content/output")
-    @PreAuthorize("hasRole('ROLE_PROVENANCE') and hasRole('ROLE_DFM')")
+    @PreAuthorize("hasRole('ROLE_PROVENANCE')")
     public Response getOutputContent(
             @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
             @QueryParam("clusterNodeId") String clusterNodeId,


[04/14] incubator-nifi git commit: NIFI-65: - Adding methods to the AuthorityProvider to authorize the downloading of content.

Posted by mc...@apache.org.
NIFI-65:
- Adding methods to the AuthorityProvider to authorize the downloading of content.

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

Branch: refs/heads/develop
Commit: e1ffbdf466b6684b9dec57da2f17849b9bc45c72
Parents: 418d6b0
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Dec 23 09:31:55 2014 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Dec 23 09:31:55 2014 -0500

----------------------------------------------------------------------
 .../authorization/FileAuthorizationProvider.java    | 16 ++++++++++++++++
 1 file changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e1ffbdf4/nar-bundles/framework-bundle/framework/file-authorization-provider/src/main/java/org/apache/nifi/authorization/FileAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/file-authorization-provider/src/main/java/org/apache/nifi/authorization/FileAuthorizationProvider.java b/nar-bundles/framework-bundle/framework/file-authorization-provider/src/main/java/org/apache/nifi/authorization/FileAuthorizationProvider.java
index beedd8e..5657369 100644
--- a/nar-bundles/framework-bundle/framework/file-authorization-provider/src/main/java/org/apache/nifi/authorization/FileAuthorizationProvider.java
+++ b/nar-bundles/framework-bundle/framework/file-authorization-provider/src/main/java/org/apache/nifi/authorization/FileAuthorizationProvider.java
@@ -21,6 +21,8 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import javax.xml.XMLConstants;
 import javax.xml.bind.JAXBContext;
@@ -493,6 +495,20 @@ public class FileAuthorizationProvider implements AuthorityProvider {
     }
 
     /**
+     * Grants access to download content regardless of FlowFile attributes.
+     * 
+     * @param dnChain
+     * @param attributes
+     * @return
+     * @throws UnknownIdentityException
+     * @throws AuthorityAccessException 
+     */
+    @Override
+    public DownloadAuthorization authorizeDownload(List<String> dnChain, Map<String, String> attributes) throws UnknownIdentityException, AuthorityAccessException {
+        return DownloadAuthorization.approved();
+    }
+
+    /**
      * Locates the user with the specified DN.
      *
      * @param dn


[14/14] incubator-nifi git commit: Merge branch 'NIFI-65' into develop

Posted by mc...@apache.org.
Merge branch 'NIFI-65' into develop


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

Branch: refs/heads/develop
Commit: b569a9d7847c90ce89468a21bdcd0e9ae080f433
Parents: 1374d1d 1a34e75
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Dec 23 15:36:37 2014 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Dec 23 15:36:37 2014 -0500

----------------------------------------------------------------------
 .../apache/nifi/admin/service/UserService.java  |  13 +
 .../service/action/AuthorizeDownloadAction.java |  54 ++++
 .../admin/service/impl/StandardUserService.java |  36 ++-
 .../AuthorityProviderFactoryBean.java           |  13 +
 .../java/org/apache/nifi/user/NiFiUser.java     |  10 +
 .../impl/NiFiAuthorizationServiceTest.java      | 284 -------------------
 .../NodeAuthorizationProvider.java              |   8 +
 .../file-authorization-provider/pom.xml         |   2 +-
 .../FileAuthorizationProvider.java              |  16 ++
 .../apache/nifi/web/api/ProvenanceResource.java |  10 +-
 .../nifi/web/controller/ControllerFacade.java   |  30 ++
 .../src/main/resources/nifi-web-api-context.xml |   1 +
 .../util/NiFiTestAuthorizationProvider.java     |   7 +
 .../js/nf/provenance/nf-provenance-table.js     | 107 ++++---
 .../js/nf/templates/nf-templates-table.js       |   2 +-
 .../authorization/NiFiAuthorizationService.java |  35 ++-
 .../NiFiAuthorizationServiceTest.java           | 250 ++++++++++++++++
 .../nifi/authorization/AuthorityProvider.java   |  19 ++
 .../authorization/DownloadAuthorization.java    |  86 ++++++
 19 files changed, 630 insertions(+), 353 deletions(-)
----------------------------------------------------------------------



[10/14] incubator-nifi git commit: NIFI-65: - Replacing calls to nf.Common.submit with window.open where appropriate.

Posted by mc...@apache.org.
NIFI-65:
- Replacing calls to nf.Common.submit with window.open where appropriate.

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

Branch: refs/heads/develop
Commit: 7a29166d9cf009e7c259ddf48641f45737b4c76b
Parents: 1f4ae73
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Dec 23 10:28:41 2014 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Dec 23 10:28:41 2014 -0500

----------------------------------------------------------------------
 .../js/nf/provenance/nf-provenance-table.js     | 107 +++++++++----------
 .../js/nf/templates/nf-templates-table.js       |   2 +-
 2 files changed, 53 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/7a29166d/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-table.js
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-table.js b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-table.js
index bcd0fce..759bcda 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-table.js
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/provenance/nf-provenance-table.js
@@ -84,17 +84,18 @@ nf.ProvenanceTable = (function () {
     var downloadContent = function (direction) {
         var eventId = $('#provenance-event-id').text();
 
-        // build the parameters
-        var parameters = {};
+        // build the url
+        var url = config.urls.provenance + '/events/' + encodeURIComponent(eventId) + '/content/' + encodeURIComponent(direction);
 
         // conditionally include the cluster node id
         var clusterNodeId = $('#provenance-event-cluster-node-id').text();
         if (!nf.Common.isBlank(clusterNodeId)) {
-            parameters['clusterNodeId'] = clusterNodeId;
+            window.open(url + '?' + $.param({
+                'clusterNodeId': clusterNodeId
+            }));
+        } else {
+            window.open(url);
         }
-
-        // get the content
-        nf.Common.submit('GET', config.urls.provenance + '/events/' + encodeURIComponent(eventId) + '/content/' + encodeURIComponent(direction), parameters);
     };
 
     /**
@@ -191,6 +192,29 @@ nf.ProvenanceTable = (function () {
             }
         });
 
+        // input download
+        $('#input-content-download').on('click', function () {
+            downloadContent('input');
+        });
+
+        // output download
+        $('#output-content-download').on('click', function () {
+            downloadContent('output');
+        });
+
+        // if a content viewer url is specified, use it
+        if (isContentViewConfigured()) {
+            // input view
+            $('#input-content-view').on('click', function () {
+                viewContent('input');
+            });
+
+            // output view
+            $('#output-content-view').on('click', function () {
+                viewContent('output');
+            });
+        }
+
         // handle the replay and downloading
         if (nf.Common.isDFM()) {
             // replay
@@ -220,29 +244,6 @@ nf.ProvenanceTable = (function () {
                 $('#event-details-dialog').modal('hide');
             });
 
-            // input download
-            $('#input-content-download').on('click', function () {
-                downloadContent('input');
-            });
-
-            // output download
-            $('#output-content-download').on('click', function () {
-                downloadContent('output');
-            });
-
-            // if a content viewer url is specified, use it
-            if (isContentViewConfigured()) {
-                // input view
-                $('#input-content-view').on('click', function () {
-                    viewContent('input');
-                });
-
-                // output view
-                $('#output-content-view').on('click', function () {
-                    viewContent('output');
-                });
-            }
-
             // show the replay panel
             $('#replay-details').show();
         }
@@ -1281,19 +1282,17 @@ nf.ProvenanceTable = (function () {
 
                 $('#output-content-download').hide();
 
-                if (nf.Common.isDFM()) {
-                    if (event.inputContentAvailable === true) {
-                        $('#input-content-download').show();
+                if (event.inputContentAvailable === true) {
+                    $('#input-content-download').show();
 
-                        if (isContentViewConfigured()) {
-                            $('#input-content-view').show();
-                        } else {
-                            $('#input-content-view').hide();
-                        }
+                    if (isContentViewConfigured()) {
+                        $('#input-content-view').show();
                     } else {
-                        $('#input-content-download').hide();
                         $('#input-content-view').hide();
                     }
+                } else {
+                    $('#input-content-download').hide();
+                    $('#input-content-view').hide();
                 }
             } else {
                 $('#output-content-details').show();
@@ -1327,32 +1326,30 @@ nf.ProvenanceTable = (function () {
                     outputContentSize.attr('title', nf.Common.formatInteger(event.outputContentClaimFileSizeBytes) + ' bytes');
                 }
 
-                if (nf.Common.isDFM()) {
-                    if (event.inputContentAvailable === true) {
-                        $('#input-content-download').show();
+                if (event.inputContentAvailable === true) {
+                    $('#input-content-download').show();
 
-                        if (isContentViewConfigured()) {
-                            $('#input-content-view').show();
-                        } else {
-                            $('#input-content-view').hide();
-                        }
+                    if (isContentViewConfigured()) {
+                        $('#input-content-view').show();
                     } else {
-                        $('#input-content-download').hide();
                         $('#input-content-view').hide();
                     }
+                } else {
+                    $('#input-content-download').hide();
+                    $('#input-content-view').hide();
+                }
 
-                    if (event.outputContentAvailable === true) {
-                        $('#output-content-download').show();
+                if (event.outputContentAvailable === true) {
+                    $('#output-content-download').show();
 
-                        if (isContentViewConfigured()) {
-                            $('#output-content-view').show();
-                        } else {
-                            $('#output-content-view').hide();
-                        }
+                    if (isContentViewConfigured()) {
+                        $('#output-content-view').show();
                     } else {
-                        $('#output-content-download').hide();
                         $('#output-content-view').hide();
                     }
+                } else {
+                    $('#output-content-download').hide();
+                    $('#output-content-view').hide();
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/7a29166d/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/templates/nf-templates-table.js
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/templates/nf-templates-table.js b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/templates/nf-templates-table.js
index 7f486de..1756207 100644
--- a/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/templates/nf-templates-table.js
+++ b/nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/templates/nf-templates-table.js
@@ -270,7 +270,7 @@ nf.TemplatesTable = (function () {
             if (nf.Common.isDefinedAndNotNull(grid)) {
                 var data = grid.getData();
                 var item = data.getItem(row);
-                nf.Common.submit('GET', config.urls.templates + '/' + encodeURIComponent(item.id));
+                window.open(config.urls.templates + '/' + encodeURIComponent(item.id));
             }
         },