NIFI-3050:

- Introducing a Restricted annotation for components that require elevated privileges to use.
- Updating the new Processor, Controller Service, and Reporting Task dialogs to include these details and prevent unauthorized selection.
- Including the Restricted description in the generated component documentation.
- Updating processor access control integration test to verify restricted component creation.
- Updating the developer, user, and admin guide to include the restricted component policy.
This commit is contained in:
Matt Gilman 2016-11-18 16:51:13 -05:00
parent 1be0871473
commit f487682419
64 changed files with 1090 additions and 357 deletions

View File

@ -0,0 +1,51 @@
/*
* 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.annotation.behavior;
import java.lang.annotation.Documented;
import java.lang.annotation.ElementType;
import java.lang.annotation.Inherited;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
/**
* <p>
* Marks the usage of a component as restricted to users with elevated privileges.
* </p>
* <p>
* A Restricted component is one that can be used to execute arbitrary unsanitized
* code provided by the operator through the NiFi REST API/UI or can be used to obtain
* or alter data on the NiFi host system using the NiFi OS credentials. These components
* could be used by an otherwise authorized NiFi user to go beyond the intended use of
* the application, escalate privilege, or could expose data about the internals of the
* NiFi process or the host system. All of these capabilities should be considered
* privileged, and admins should be aware of these capabilities and explicitly enable
* them for a subset of trusted users.
* </p>
*/
@Documented
@Target({ElementType.TYPE})
@Retention(RetentionPolicy.RUNTIME)
@Inherited
public @interface Restricted {
/**
* Provides a description of why the component usage is restricted
*/
String value();
}

View File

@ -478,6 +478,7 @@ Here is a summary of policies assigned to each legacy role if the NiFi instance
|view policies |* | | | | |
|modify policies |* | | | | |
|query provenance | | | |* | |
|access restricted components | |* | | | |
|view the data | |* | |* | |*
|modify the data | |* | | | |*
|retrieve site-to-site details | | | | |* |
@ -581,6 +582,10 @@ Global access policies govern the following system level authorizations:
|Allows users to submit a Provenance Search and request Event Lineage
|Data Provenance
|access restricted components
|Allows users to create/modify restricted components assuming otherwise sufficient permissions
|N/A
|access all policies
|Allows users to view/modify the policies for all components
|Policies

View File

@ -565,6 +565,23 @@ for instance, they should not be
relied upon for critical business logic.
[[restricted]]
=== Restricted
A Restricted component is one that can be used to execute arbitrary unsanitized code provided by the operator
through the NiFi REST API/UI or can be used to obtain or alter data on the NiFi host system using the NiFi OS
credentials. These components could be used by an otherwise authorized NiFi user to go beyond the intended use of
the application, escalate privilege, or could expose data about the internals of the NiFi process or the host
system. All of these capabilities should be considered privileged, and admins should be aware of these
capabilities and explicitly enable them for a subset of trusted users.
A Processor, Controller Service, or Reporting Task can be marked with the @Restricted annotation. This
will result in the component being treated as restricted and will require a user to be explicitly added to the
list of users who can access restricted components. Once a user is permitted to access restricted components,
they will be allowed to create and modify those components assuming all other permissions are permitted.
Without access to restricted components, a user will be still be aware these types of components exist but will
be unable to create or modify them even with otherwise sufficient permissions.
[[state_manager]]
=== State Manager

Binary file not shown.

After

Width:  |  Height:  |  Size: 3.5 KiB

View File

@ -40,15 +40,13 @@ Browser Support
|Browser |Version
|Chrome |Current and Current - 1
|FireFox |Current and Current - 1
|Edge |Current
|Edge |Current and Current - 1
|Safari |Current and Current - 1
|======================
Current and Current - 1 indicates that the UI is supported in the current stable release of that browser and the preceding one. For instance, if
the current stable release is 45.X then the officially supported versions will be 45.X and 44.X.
Current indicates that the UI is supported in the current stable release of that browser.
The supported browser versions are driven by the capabilities the UI employs and the dependencies it uses. UI features will be developed and tested
against the supported browsers. Any problem using a supported browser should be reported to Apache NiFi.
@ -195,6 +193,7 @@ The available global access policies are:
|view the UI |Allows users to view the UI
|access the controller |Allows users to view and modify the controller including reporting tasks, Controller Services, and nodes in the cluster
|query provenance |Allows users to submit a provenance search and request even lineage
|access restricted components |Allows users to create/modify restricted components assuming otherwise sufficient permissions
|access all policies |Allows users to view and modify the policies for all components
|access users/groups |Allows users view and modify the users and user groups
|retrieve site-to-site details | Allows other NiFi instances to retrieve Site-To-Site details
@ -270,6 +269,17 @@ Processors that allow us to ingest data via HTTP, we can select both the `http`
image::add-processor-with-tag-cloud.png["Add Processor with Tag Cloud"]
Restricted components will be marked with a
image:restricted.png["Restricted"]
icon next to their name. These are components that can be used to execute arbitrary unsanitized code provided by the operator
through the NiFi REST API/UI or can be used to obtain or alter data on the NiFi host system using the NiFi OS credentials.
These components could be used by an otherwise authorized NiFi user to go beyond the intended use of the application, escalate
privilege, or could expose data about the internals of the NiFi process or the host system. All of these capabilities should
be considered privileged, and admins should be aware of these capabilities and explicitly enable them for a subset of trusted users.
Before a user is allowed to create and modify restricted components they must be granted access to restricted components. Refer to
<<UI-with-multi-tenant-authorization,multi-tenant>> documentation.
Clicking the `Add` button or double-clicking on a Processor Type will add the selected Processor to the canvas at the
location that it was dropped.

View File

@ -16,14 +16,15 @@
*/
package org.apache.nifi.processors.flume;
import java.util.List;
import java.util.Set;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import org.apache.flume.EventDeliveryException;
import org.apache.flume.Sink;
import org.apache.flume.conf.Configurables;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.behavior.TriggerSerially;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
@ -38,9 +39,8 @@ import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import java.util.List;
import java.util.Set;
/**
* This processor runs a Flume sink
@ -49,6 +49,7 @@ import com.google.common.collect.ImmutableSet;
@Tags({"flume", "hadoop", "put", "sink"})
@InputRequirement(Requirement.INPUT_REQUIRED)
@CapabilityDescription("Execute a Flume sink. Each input FlowFile is converted into a Flume Event for processing by the sink.")
@Restricted("Provides operator the ability to execute arbitrary Flume configurations assuming all permissions that NiFi has.")
public class ExecuteFlumeSink extends AbstractFlumeProcessor {
public static final PropertyDescriptor SINK_TYPE = new PropertyDescriptor.Builder()

View File

@ -16,10 +16,9 @@
*/
package org.apache.nifi.processors.flume;
import java.util.List;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import org.apache.flume.EventDeliveryException;
import org.apache.flume.EventDrivenSource;
import org.apache.flume.PollableSource;
@ -29,6 +28,7 @@ import org.apache.flume.conf.Configurables;
import org.apache.flume.source.EventDrivenSourceRunner;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.behavior.TriggerSerially;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
@ -44,9 +44,9 @@ import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
/**
* This processor runs a Flume source
@ -55,6 +55,7 @@ import com.google.common.collect.ImmutableSet;
@Tags({"flume", "hadoop", "get", "source"})
@InputRequirement(Requirement.INPUT_FORBIDDEN)
@CapabilityDescription("Execute a Flume source. Each Flume Event is sent to the success relationship as a FlowFile")
@Restricted("Provides operator the ability to execute arbitrary Flume configurations assuming all permissions that NiFi has.")
public class ExecuteFlumeSource extends AbstractFlumeProcessor {
public static final PropertyDescriptor SOURCE_TYPE = new PropertyDescriptor.Builder()

View File

@ -17,8 +17,9 @@
package org.apache.nifi.web.api.dto;
import com.wordnik.swagger.annotations.ApiModelProperty;
import java.util.Set;
import javax.xml.bind.annotation.XmlType;
import java.util.Set;
/**
* Class used for providing documentation of a specified type.
@ -28,6 +29,7 @@ public class DocumentedTypeDTO {
private String type;
private String description;
private String usageRestriction;
private Set<String> tags;
/**
@ -44,6 +46,20 @@ public class DocumentedTypeDTO {
this.description = description;
}
/**
* @return An optional description of why the usage of this component is restricted
*/
@ApiModelProperty(
value = "The description of why the usage of this component is restricted."
)
public String getUsageRestriction() {
return usageRestriction;
}
public void setUsageRestriction(String usageRestriction) {
this.usageRestriction = usageRestriction;
}
/**
* @return The type is the fully-qualified name of a Java class
*/

View File

@ -36,6 +36,7 @@ public class CurrentUserEntity extends Entity {
private PermissionsDTO controllerPermissions;
private PermissionsDTO policiesPermissions;
private PermissionsDTO systemPermissions;
private PermissionsDTO restrictedComponentsPermissions;
/**
* @return the user identity being serialized
@ -132,4 +133,16 @@ public class CurrentUserEntity extends Entity {
public void setSystemPermissions(PermissionsDTO systemPermissions) {
this.systemPermissions = systemPermissions;
}
/**
* @return permissions for accessing the restricted components
*/
@ApiModelProperty("Permissions for accessing restricted components. Note: the read permission are not used and will always be false.")
public PermissionsDTO getRestrictedComponentsPermissions() {
return restrictedComponentsPermissions;
}
public void setRestrictedComponentsPermissions(PermissionsDTO restrictedComponentsPermissions) {
this.restrictedComponentsPermissions = restrictedComponentsPermissions;
}
}

View File

@ -16,19 +16,9 @@
*/
package org.apache.nifi.documentation.html;
import java.io.IOException;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import javax.xml.stream.FactoryConfigurationError;
import javax.xml.stream.XMLOutputFactory;
import javax.xml.stream.XMLStreamException;
import javax.xml.stream.XMLStreamWriter;
import org.apache.nifi.annotation.behavior.DynamicProperties;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.behavior.Stateful;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
@ -40,6 +30,16 @@ import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.documentation.DocumentationWriter;
import org.apache.nifi.nar.ExtensionManager;
import javax.xml.stream.FactoryConfigurationError;
import javax.xml.stream.XMLOutputFactory;
import javax.xml.stream.XMLStreamException;
import javax.xml.stream.XMLStreamWriter;
import java.io.IOException;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
/**
* Generates HTML documentation for a ConfigurableComponent. This class is used
* to generate documentation for ControllerService and ReportingTask because
@ -129,6 +129,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter {
writeDynamicProperties(configurableComponent, xmlStreamWriter);
writeAdditionalBodyInfo(configurableComponent, xmlStreamWriter);
writeStatefulInfo(configurableComponent, xmlStreamWriter);
writeRestrictedInfo(configurableComponent, xmlStreamWriter);
writeSeeAlso(configurableComponent, xmlStreamWriter);
xmlStreamWriter.writeEndElement();
}
@ -165,6 +166,24 @@ public class HtmlDocumentationWriter implements DocumentationWriter {
}
}
/**
* Write the description of the Restricted annotation if provided in this component.
*
* @param configurableComponent the component to describe
* @param xmlStreamWriter the stream writer to use
* @throws XMLStreamException thrown if there was a problem writing the XML
*/
private void writeRestrictedInfo(ConfigurableComponent configurableComponent, XMLStreamWriter xmlStreamWriter)
throws XMLStreamException {
final Restricted restricted = configurableComponent.getClass().getAnnotation(Restricted.class);
writeSimpleElement(xmlStreamWriter, "h3", "Restricted: ");
if(restricted != null) {
writeSimpleElement(xmlStreamWriter, "td", restricted.value());
}
}
/**
* Writes the list of components that may be linked from this component.
*

View File

@ -16,10 +16,7 @@
*/
package org.apache.nifi.documentation.example;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnRemoved;
@ -29,8 +26,13 @@ import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.processor.util.StandardValidators;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
@CapabilityDescription("A documented controller service that can help you do things")
@Tags({ "one", "two", "three" })
@Restricted("controller service restriction description")
public class FullyDocumentedControllerService extends AbstractControllerService implements SampleService {
public static final PropertyDescriptor KEYSTORE = new PropertyDescriptor.Builder().name("Keystore Filename").description("The fully-qualified filename of the Keystore").defaultValue(null)

View File

@ -16,15 +16,10 @@
*/
package org.apache.nifi.documentation.example;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.DynamicRelationship;
import org.apache.nifi.annotation.behavior.ReadsAttribute;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.behavior.Stateful;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
@ -44,6 +39,12 @@ import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@Tags({"one", "two", "three"})
@CapabilityDescription("This is a processor that is used to test documentation.")
@WritesAttributes({
@ -54,6 +55,7 @@ import org.apache.nifi.processor.util.StandardValidators;
@DynamicProperty(name = "Relationship Name", supportsExpressionLanguage = true, value = "some XPath", description = "Routes FlowFiles to relationships based on XPath")
@DynamicRelationship(name = "name from dynamic property", description = "all files that match the properties XPath")
@Stateful(scopes = {Scope.CLUSTER, Scope.LOCAL}, description = "state management description")
@Restricted("processor restriction description")
public class FullyDocumentedProcessor extends AbstractProcessor {
public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder().name("Input Directory")

View File

@ -16,20 +16,22 @@
*/
package org.apache.nifi.documentation.example;
import java.util.ArrayList;
import java.util.List;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnRemoved;
import org.apache.nifi.annotation.lifecycle.OnShutdown;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.reporting.AbstractReportingTask;
import org.apache.nifi.reporting.ReportingContext;
import java.util.ArrayList;
import java.util.List;
@CapabilityDescription("A helper reporting task to do...")
@Tags({"first", "second", "third"})
@Restricted("reporting task restriction description")
public class FullyDocumentedReportingTask extends AbstractReportingTask {
public static final PropertyDescriptor SHOW_DELTAS = new PropertyDescriptor.Builder()

View File

@ -16,12 +16,6 @@
*/
package org.apache.nifi.documentation.html;
import static org.apache.nifi.documentation.html.XmlValidator.assertContains;
import static org.junit.Assert.assertEquals;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.documentation.DocumentationWriter;
import org.apache.nifi.documentation.example.ControllerServiceWithLogger;
@ -37,6 +31,12 @@ import org.apache.nifi.reporting.ReportingTask;
import org.junit.Assert;
import org.junit.Test;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import static org.apache.nifi.documentation.html.XmlValidator.assertContains;
import static org.junit.Assert.assertEquals;
public class HtmlDocumentationWriterTest {
@Test
@ -77,6 +77,9 @@ public class HtmlDocumentationWriterTest {
assertContains(results, "PKCS12");
assertContains(results, "Sensitive Property: true");
// restricted
assertContains(results, "controller service restriction description");
// verify the right OnRemoved and OnShutdown methods were called
Assert.assertEquals(0, controllerService.getOnRemovedArgs());
Assert.assertEquals(0, controllerService.getOnRemovedNoArgs());
@ -114,6 +117,9 @@ public class HtmlDocumentationWriterTest {
assertContains(results, "true");
assertContains(results, "false");
// restricted
assertContains(results, "reporting task restriction description");
// verify the right OnRemoved and OnShutdown methods were called
Assert.assertEquals(0, reportingTask.getOnRemovedArgs());
Assert.assertEquals(0, reportingTask.getOnRemovedNoArgs());

View File

@ -16,12 +16,6 @@
*/
package org.apache.nifi.documentation.html;
import static org.apache.nifi.documentation.html.XmlValidator.assertContains;
import static org.apache.nifi.documentation.html.XmlValidator.assertNotContains;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.documentation.DocumentationWriter;
import org.apache.nifi.documentation.example.FullyDocumentedProcessor;
@ -31,6 +25,12 @@ import org.apache.nifi.documentation.init.ProcessorInitializer;
import org.junit.Assert;
import org.junit.Test;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import static org.apache.nifi.documentation.html.XmlValidator.assertContains;
import static org.apache.nifi.documentation.html.XmlValidator.assertNotContains;
public class ProcessorDocumentationWriterTest {
@Test
@ -69,6 +69,8 @@ public class ProcessorDocumentationWriterTest {
assertContains(results, "CLUSTER, LOCAL");
assertContains(results, "state management description");
assertContains(results, "processor restriction description");
assertNotContains(results, "iconSecure.png");
assertContains(results, FullyDocumentedProcessor.class.getAnnotation(CapabilityDescription.class)
.value());

View File

@ -333,6 +333,9 @@ public class FileAuthorizer extends AbstractPolicyBasedAuthorizer {
addAccessPolicy(authorizations, ResourceType.ProcessGroup.getValue() + "/" + rootGroupId, adminUser.getIdentifier(), WRITE_CODE);
}
// grant the user write to restricted components
addAccessPolicy(authorizations, ResourceType.RestrictedComponents.getValue(), adminUser.getIdentifier(), WRITE_CODE);
// grant the user read/write access to the /tenants resource
addAccessPolicy(authorizations, ResourceType.Tenant.getValue(), adminUser.getIdentifier(), READ_CODE);
addAccessPolicy(authorizations, ResourceType.Tenant.getValue(), adminUser.getIdentifier(), WRITE_CODE);

View File

@ -72,6 +72,7 @@ public final class RoleAccessPolicy {
dfmPolicies.add(new RoleAccessPolicy(ResourceType.Controller.getValue(), READ_ACTION));
dfmPolicies.add(new RoleAccessPolicy(ResourceType.Controller.getValue(), WRITE_ACTION));
dfmPolicies.add(new RoleAccessPolicy(ResourceType.System.getValue(), READ_ACTION));
dfmPolicies.add(new RoleAccessPolicy(ResourceType.RestrictedComponents.getValue(), WRITE_ACTION));
if (rootGroupId != null) {
dfmPolicies.add(new RoleAccessPolicy(ResourceType.ProcessGroup.getValue() + "/" + rootGroupId, READ_ACTION));
dfmPolicies.add(new RoleAccessPolicy(ResourceType.ProcessGroup.getValue() + "/" + rootGroupId, WRITE_ACTION));

View File

@ -312,7 +312,7 @@ public class FileAuthorizerTest {
// verify user3's policies
final Map<String,Set<RequestAction>> user3Policies = getResourceActions(policies, user3);
assertEquals(5, user3Policies.size());
assertEquals(6, user3Policies.size());
assertTrue(user3Policies.containsKey(ResourceType.Flow.getValue()));
assertEquals(1, user3Policies.get(ResourceType.Flow.getValue()).size());
@ -502,7 +502,7 @@ public class FileAuthorizerTest {
assertEquals(adminIdentity, adminUser.getIdentity());
final Set<AccessPolicy> policies = authorizer.getAccessPolicies();
assertEquals(11, policies.size());
assertEquals(12, policies.size());
final String rootGroupResource = ResourceType.ProcessGroup.getValue() + "/" + ROOT_GROUP_ID;
@ -540,7 +540,7 @@ public class FileAuthorizerTest {
assertEquals(adminIdentity, adminUser.getIdentity());
final Set<AccessPolicy> policies = authorizer.getAccessPolicies();
assertEquals(7, policies.size());
assertEquals(8, policies.size());
final String rootGroupResource = ResourceType.ProcessGroup.getValue() + "/" + ROOT_GROUP_ID;
@ -578,7 +578,7 @@ public class FileAuthorizerTest {
assertEquals(adminIdentity, adminUser.getIdentity());
final Set<AccessPolicy> policies = authorizer.getAccessPolicies();
assertEquals(7, policies.size());
assertEquals(8, policies.size());
final String rootGroupResource = ResourceType.ProcessGroup.getValue() + "/" + ROOT_GROUP_ID;

View File

@ -142,6 +142,18 @@ public final class ResourceFactory {
}
};
private final static Resource RESTRICTED_COMPONENTS_RESOURCE = new Resource() {
@Override
public String getIdentifier() {
return ResourceType.RestrictedComponents.getValue();
}
@Override
public String getName() {
return "Restricted Components";
}
};
private final static Resource TENANT_RESOURCE = new Resource() {
@Override
public String getIdentifier() {
@ -241,6 +253,15 @@ public final class ResourceFactory {
return SYSTEM_RESOURCE;
}
/**
* Gets the Resource for accessing restricted components.
*
* @return The restricted components resource
*/
public static Resource getRestrictedComponentsResource() {
return RESTRICTED_COMPONENTS_RESOURCE;
}
/**
* Gets the Resource for accessing Tenants which includes creating, modifying, and deleting Users and UserGroups.
*

View File

@ -37,6 +37,7 @@ public enum ResourceType {
SiteToSite("/site-to-site"),
DataTransfer("/data-transfer"),
System("/system"),
RestrictedComponents("/restricted-components"),
Template("/templates"),
Tenant("/tenants");

View File

@ -0,0 +1,33 @@
/*
* 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.resource;
import org.apache.nifi.authorization.Resource;
public class RestrictedComponentsAuthorizable implements Authorizable {
@Override
public Authorizable getParentAuthorizable() {
return null;
}
@Override
public Resource getResource() {
return ResourceFactory.getRestrictedComponentsResource();
}
}

View File

@ -16,7 +16,14 @@
*/
package org.apache.nifi.controller;
import org.apache.nifi.authorization.AccessDeniedException;
import org.apache.nifi.authorization.AuthorizationResult;
import org.apache.nifi.authorization.AuthorizationResult.Result;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.ComponentAuthorizable;
import org.apache.nifi.authorization.resource.RestrictedComponentsAuthorizable;
import org.apache.nifi.authorization.user.NiFiUser;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationResult;
@ -58,4 +65,37 @@ public interface ConfiguredComponent extends ComponentAuthorizable {
*/
String getCanonicalClassName();
/**
* @return whether or not the underlying implementation is restricted
*/
boolean isRestricted();
@Override
default AuthorizationResult checkAuthorization(Authorizer authorizer, RequestAction action, NiFiUser user, Map<String, String> resourceContext) {
// if this is a modification request and the reporting task is restricted ensure the user has elevated privileges. if this
// is not a modification request, we just want to use the normal rules
if (RequestAction.WRITE.equals(action) && isRestricted()) {
final RestrictedComponentsAuthorizable restrictedComponentsAuthorizable = new RestrictedComponentsAuthorizable();
final AuthorizationResult result = restrictedComponentsAuthorizable.checkAuthorization(authorizer, RequestAction.WRITE, user, resourceContext);
if (Result.Denied.equals(result.getResult())) {
return result;
}
}
// defer to the base authorization check
return ComponentAuthorizable.super.checkAuthorization(authorizer, action, user, resourceContext);
}
@Override
default void authorize(Authorizer authorizer, RequestAction action, NiFiUser user, Map<String, String> resourceContext) throws AccessDeniedException {
// if this is a modification request and the reporting task is restricted ensure the user has elevated privileges. if this
// is not a modification request, we just want to use the normal rules
if (RequestAction.WRITE.equals(action) && isRestricted()) {
final RestrictedComponentsAuthorizable restrictedComponentsAuthorizable = new RestrictedComponentsAuthorizable();
restrictedComponentsAuthorizable.authorize(authorizer, RequestAction.WRITE, user, resourceContext);
}
// defer to the base authorization check
ComponentAuthorizable.super.authorize(authorizer, action, user, resourceContext);
}
}

View File

@ -21,6 +21,7 @@ import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.nifi.annotation.behavior.EventDriven;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.behavior.SideEffectFree;
import org.apache.nifi.annotation.behavior.SupportsBatching;
import org.apache.nifi.annotation.behavior.TriggerSerially;
@ -56,8 +57,8 @@ import org.apache.nifi.processor.Processor;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.SimpleProcessLogger;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.scheduling.SchedulingStrategy;
import org.apache.nifi.scheduling.ExecutionNode;
import org.apache.nifi.scheduling.SchedulingStrategy;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.ReflectionUtils;
@ -236,6 +237,11 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
return ResourceFactory.getComponentResource(ResourceType.Processor, getIdentifier(), getName());
}
@Override
public boolean isRestricted() {
return getProcessor().getClass().isAnnotationPresent(Restricted.class);
}
/**
* Provides and opportunity to retain information about this particular
* processor instance

View File

@ -16,6 +16,7 @@
*/
package org.apache.nifi.controller.reporting;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.authorization.Resource;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.authorization.resource.ResourceFactory;
@ -57,6 +58,11 @@ public class StandardReportingTaskNode extends AbstractReportingTaskNode impleme
return ResourceFactory.getComponentResource(ResourceType.ReportingTask, getIdentifier(), getName());
}
@Override
public boolean isRestricted() {
return getReportingTask().getClass().isAnnotationPresent(Restricted.class);
}
@Override
public ReportingContext getReportingContext() {
return new StandardReportingContext(flowController, flowController.getBulletinRepository(), getProperties(), flowController, getReportingTask(), getVariableRegistry());

View File

@ -17,6 +17,7 @@
package org.apache.nifi.controller.service;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.lifecycle.OnDisabled;
import org.apache.nifi.annotation.lifecycle.OnEnabled;
import org.apache.nifi.authorization.Resource;
@ -120,6 +121,11 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
return ResourceFactory.getComponentResource(ResourceType.ControllerService, getIdentifier(), getName());
}
@Override
public boolean isRestricted() {
return getControllerServiceImplementation().getClass().isAnnotationPresent(Restricted.class);
}
@Override
public ControllerService getProxiedControllerService() {
return proxedControllerService;

View File

@ -34,7 +34,7 @@ public interface AuthorizableLookup {
* @param id processor id
* @return authorizable
*/
ControllerServiceReferencingComponentAuthorizable getProcessor(String id);
ConfigurableComponentAuthorizable getProcessor(String id);
/**
* Get the authorizable for this Processor. This will create a dummy instance of the
@ -44,7 +44,7 @@ public interface AuthorizableLookup {
* @param type processor type
* @return authorizable
*/
ControllerServiceReferencingComponentAuthorizable getProcessorByType(String type);
ConfigurableComponentAuthorizable getProcessorByType(String type);
/**
* Get the authorizable for querying Provenance.
@ -156,7 +156,7 @@ public interface AuthorizableLookup {
* @param id controller service id
* @return authorizable
*/
ControllerServiceReferencingComponentAuthorizable getControllerService(String id);
ConfigurableComponentAuthorizable getControllerService(String id);
/**
* Get the authorizable for this Controller Service. This will create a dummy instance of the
@ -166,7 +166,7 @@ public interface AuthorizableLookup {
* @param type processor type
* @return authorizable
*/
ControllerServiceReferencingComponentAuthorizable getControllerServiceByType(String type);
ConfigurableComponentAuthorizable getControllerServiceByType(String type);
/**
* Get the authorizable referencing component.
@ -183,7 +183,7 @@ public interface AuthorizableLookup {
* @param id reporting task id
* @return authorizable
*/
ControllerServiceReferencingComponentAuthorizable getReportingTask(String id);
ConfigurableComponentAuthorizable getReportingTask(String id);
/**
* Get the authorizable for this Reporting Task. This will create a dummy instance of the
@ -193,7 +193,7 @@ public interface AuthorizableLookup {
* @param type processor type
* @return authorizable
*/
ControllerServiceReferencingComponentAuthorizable getReportingTaskByType(String type);
ConfigurableComponentAuthorizable getReportingTaskByType(String type);
/**
* Get the authorizable Template.
@ -271,4 +271,10 @@ public interface AuthorizableLookup {
*/
Authorizable getSystem();
/**
* Get the authorizable for accessing restricted components.
*
* @return authorizable
*/
Authorizable getRestrictedComponents();
}

View File

@ -38,7 +38,7 @@ public final class AuthorizeControllerServiceReference {
* @param authorizer authorizer
* @param lookup lookup
*/
public static void authorizeControllerServiceReferences(final Map<String, String> proposedProperties, final ControllerServiceReferencingComponentAuthorizable authorizable,
public static void authorizeControllerServiceReferences(final Map<String, String> proposedProperties, final ConfigurableComponentAuthorizable authorizable,
final Authorizer authorizer, final AuthorizableLookup lookup) {
// only attempt to authorize if properties are changing

View File

@ -22,7 +22,7 @@ import org.apache.nifi.components.PropertyDescriptor;
/**
* Authorizable for a component that references a ControllerService.
*/
public interface ControllerServiceReferencingComponentAuthorizable {
public interface ConfigurableComponentAuthorizable {
/**
* Returns the base authorizable for this ControllerServiceReference. Non null
*
@ -30,6 +30,13 @@ public interface ControllerServiceReferencingComponentAuthorizable {
*/
Authorizable getAuthorizable();
/**
* Returns whether or not the underlying configurable component is restricted.
*
* @return whether or not the underlying configurable component is restricted
*/
boolean isRestricted();
/**
* Returns the property descriptor for the specified property.
*

View File

@ -23,6 +23,7 @@ import org.apache.nifi.authorization.resource.DataAuthorizable;
import org.apache.nifi.authorization.resource.DataTransferAuthorizable;
import org.apache.nifi.authorization.resource.ResourceFactory;
import org.apache.nifi.authorization.resource.ResourceType;
import org.apache.nifi.authorization.resource.RestrictedComponentsAuthorizable;
import org.apache.nifi.authorization.resource.TenantAuthorizable;
import org.apache.nifi.authorization.user.NiFiUser;
import org.apache.nifi.components.PropertyDescriptor;
@ -62,6 +63,8 @@ import java.util.Set;
class StandardAuthorizableLookup implements AuthorizableLookup {
private static final TenantAuthorizable TENANT_AUTHORIZABLE = new TenantAuthorizable();
private static final Authorizable RESTRICTED_COMPONENTS_AUTHORIZABLE = new RestrictedComponentsAuthorizable();
private static final Authorizable POLICIES_AUTHORIZABLE = new Authorizable() {
@Override
public Authorizable getParentAuthorizable() {
@ -134,14 +137,19 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
}
@Override
public ControllerServiceReferencingComponentAuthorizable getProcessor(final String id) {
public ConfigurableComponentAuthorizable getProcessor(final String id) {
final ProcessorNode processorNode = processorDAO.getProcessor(id);
return new ControllerServiceReferencingComponentAuthorizable() {
return new ConfigurableComponentAuthorizable() {
@Override
public Authorizable getAuthorizable() {
return processorNode;
}
@Override
public boolean isRestricted() {
return processorNode.isRestricted();
}
@Override
public String getValue(PropertyDescriptor propertyDescriptor) {
return processorNode.getProperty(propertyDescriptor);
@ -155,15 +163,20 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
}
@Override
public ControllerServiceReferencingComponentAuthorizable getProcessorByType(String type) {
public ConfigurableComponentAuthorizable getProcessorByType(String type) {
try {
final ProcessorNode processorNode = controllerFacade.createTemporaryProcessor(type);
return new ControllerServiceReferencingComponentAuthorizable() {
return new ConfigurableComponentAuthorizable() {
@Override
public Authorizable getAuthorizable() {
return processorNode;
}
@Override
public boolean isRestricted() {
return processorNode.isRestricted();
}
@Override
public String getValue(PropertyDescriptor propertyDescriptor) {
return processorNode.getProperty(propertyDescriptor);
@ -328,14 +341,19 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
}
@Override
public ControllerServiceReferencingComponentAuthorizable getControllerService(final String id) {
public ConfigurableComponentAuthorizable getControllerService(final String id) {
final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(id);
return new ControllerServiceReferencingComponentAuthorizable() {
return new ConfigurableComponentAuthorizable() {
@Override
public Authorizable getAuthorizable() {
return controllerService;
}
@Override
public boolean isRestricted() {
return controllerService.isRestricted();
}
@Override
public String getValue(PropertyDescriptor propertyDescriptor) {
return controllerService.getProperty(propertyDescriptor);
@ -349,15 +367,20 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
}
@Override
public ControllerServiceReferencingComponentAuthorizable getControllerServiceByType(String type) {
public ConfigurableComponentAuthorizable getControllerServiceByType(String type) {
try {
final ControllerServiceNode controllerService = controllerFacade.createTemporaryControllerService(type);
return new ControllerServiceReferencingComponentAuthorizable() {
return new ConfigurableComponentAuthorizable() {
@Override
public Authorizable getAuthorizable() {
return controllerService;
}
@Override
public boolean isRestricted() {
return controllerService.isRestricted();
}
@Override
public String getValue(PropertyDescriptor propertyDescriptor) {
return controllerService.getProperty(propertyDescriptor);
@ -417,14 +440,19 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
}
@Override
public ControllerServiceReferencingComponentAuthorizable getReportingTask(final String id) {
public ConfigurableComponentAuthorizable getReportingTask(final String id) {
final ReportingTaskNode reportingTaskNode = reportingTaskDAO.getReportingTask(id);
return new ControllerServiceReferencingComponentAuthorizable() {
return new ConfigurableComponentAuthorizable() {
@Override
public Authorizable getAuthorizable() {
return reportingTaskNode;
}
@Override
public boolean isRestricted() {
return reportingTaskNode.isRestricted();
}
@Override
public String getValue(PropertyDescriptor propertyDescriptor) {
return reportingTaskNode.getProperty(propertyDescriptor);
@ -438,15 +466,20 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
}
@Override
public ControllerServiceReferencingComponentAuthorizable getReportingTaskByType(String type) {
public ConfigurableComponentAuthorizable getReportingTaskByType(String type) {
try {
final ReportingTaskNode reportingTask = controllerFacade.createTemporaryReportingTask(type);
return new ControllerServiceReferencingComponentAuthorizable() {
return new ConfigurableComponentAuthorizable() {
@Override
public Authorizable getAuthorizable() {
return reportingTask;
}
@Override
public boolean isRestricted() {
return reportingTask.isRestricted();
}
@Override
public String getValue(PropertyDescriptor propertyDescriptor) {
return reportingTask.getProperty(propertyDescriptor);
@ -603,17 +636,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
authorizable = getController();
break;
case Counters:
authorizable = new Authorizable() {
@Override
public Authorizable getParentAuthorizable() {
return null;
}
@Override
public Resource getResource() {
return ResourceFactory.getCountersResource();
}
};
authorizable = getCounters();
break;
case Flow:
authorizable = new Authorizable() {
@ -629,17 +652,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
};
break;
case Provenance:
authorizable = new Authorizable() {
@Override
public Authorizable getParentAuthorizable() {
return null;
}
@Override
public Resource getResource() {
return ResourceFactory.getProvenanceResource();
}
};
authorizable = getProvenance();
break;
case Proxy:
authorizable = new Authorizable() {
@ -655,7 +668,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
};
break;
case Policy:
authorizable = POLICIES_AUTHORIZABLE;
authorizable = getPolicies();
break;
case Resource:
authorizable = new Authorizable() {
@ -671,7 +684,6 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
};
break;
case SiteToSite:
// TODO - new site-to-site and port specific site-to-site
authorizable = new Authorizable() {
@Override
public Authorizable getParentAuthorizable() {
@ -685,21 +697,14 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
};
break;
case System:
authorizable = new Authorizable() {
@Override
public Authorizable getParentAuthorizable() {
return null;
}
@Override
public Resource getResource() {
return ResourceFactory.getSystemResource();
}
};
authorizable = getSystem();
break;
case Tenant:
authorizable = getTenant();
break;
case RestrictedComponents:
authorizable = getRestrictedComponents();
break;
}
if (authorizable == null) {
@ -720,6 +725,11 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
return group.findConnectable(id);
}
@Override
public Authorizable getRestrictedComponents() {
return RESTRICTED_COMPONENTS_AUTHORIZABLE;
}
@Override
public Authorizable getSystem() {
return SYSTEM_AUTHORIZABLE;

View File

@ -2958,6 +2958,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
entity.setControllerPermissions(dtoFactory.createPermissionsDto(authorizableLookup.getController()));
entity.setPoliciesPermissions(dtoFactory.createPermissionsDto(authorizableLookup.getPolicies()));
entity.setSystemPermissions(dtoFactory.createPermissionsDto(authorizableLookup.getSystem()));
entity.setRestrictedComponentsPermissions(dtoFactory.createPermissionsDto(authorizableLookup.getRestrictedComponents()));
return entity;
}

View File

@ -31,7 +31,7 @@ import org.apache.nifi.authorization.AuthorizationResult;
import org.apache.nifi.authorization.AuthorizationResult.Result;
import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.ControllerServiceReferencingComponentAuthorizable;
import org.apache.nifi.authorization.ConfigurableComponentAuthorizable;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.UserContextKeys;
import org.apache.nifi.authorization.resource.Authorizable;
@ -397,7 +397,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
// authorize access
serviceFacade.authorizeAccess(lookup -> {
// authorize the processor
final ControllerServiceReferencingComponentAuthorizable authorizable = lookup.getProcessor(id);
final ConfigurableComponentAuthorizable authorizable = lookup.getProcessor(id);
authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
// authorize any referenced service
@ -588,7 +588,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
// authorize access
serviceFacade.authorizeAccess(lookup -> {
// authorize the controller service
final ControllerServiceReferencingComponentAuthorizable authorizable = lookup.getControllerService(id);
final ConfigurableComponentAuthorizable authorizable = lookup.getControllerService(id);
authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
// authorize any referenced service
@ -753,7 +753,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
// authorize access
serviceFacade.authorizeAccess(lookup -> {
// authorize the reporting task
final ControllerServiceReferencingComponentAuthorizable authorizable = lookup.getReportingTask(id);
final ConfigurableComponentAuthorizable authorizable = lookup.getReportingTask(id);
authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
// authorize any referenced service

View File

@ -30,7 +30,7 @@ import org.apache.nifi.authorization.AuthorizationResult;
import org.apache.nifi.authorization.AuthorizationResult.Result;
import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.ControllerServiceReferencingComponentAuthorizable;
import org.apache.nifi.authorization.ConfigurableComponentAuthorizable;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.UserContextKeys;
import org.apache.nifi.authorization.resource.ResourceFactory;
@ -283,8 +283,12 @@ public class ControllerResource extends ApplicationResource {
lookup -> {
authorizeController(RequestAction.WRITE);
final ConfigurableComponentAuthorizable authorizable = lookup.getProcessorByType(requestReportingTask.getType());
if (authorizable.isRestricted()) {
lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
}
if (requestReportingTask.getProperties() != null) {
final ControllerServiceReferencingComponentAuthorizable authorizable = lookup.getReportingTaskByType(requestReportingTask.getType());
AuthorizeControllerServiceReference.authorizeControllerServiceReferences(requestReportingTask.getProperties(), authorizable, authorizer, lookup);
}
},
@ -376,8 +380,12 @@ public class ControllerResource extends ApplicationResource {
lookup -> {
authorizeController(RequestAction.WRITE);
final ConfigurableComponentAuthorizable authorizable = lookup.getProcessorByType(requestControllerService.getType());
if (authorizable.isRestricted()) {
lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
}
if (requestControllerService.getProperties() != null) {
final ControllerServiceReferencingComponentAuthorizable authorizable = lookup.getControllerServiceByType(requestControllerService.getType());
AuthorizeControllerServiceReference.authorizeControllerServiceReferences(requestControllerService.getProperties(), authorizable, authorizer, lookup);
}
},

View File

@ -25,7 +25,7 @@ import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.ControllerServiceReferencingComponentAuthorizable;
import org.apache.nifi.authorization.ConfigurableComponentAuthorizable;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.authorization.user.NiFiUserUtils;
@ -621,7 +621,7 @@ public class ControllerServiceResource extends ApplicationResource {
requestRevision,
lookup -> {
// authorize the service
final ControllerServiceReferencingComponentAuthorizable authorizable = lookup.getControllerService(id);
final ConfigurableComponentAuthorizable authorizable = lookup.getControllerService(id);
authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
// authorize any referenced services

View File

@ -28,7 +28,7 @@ import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.AuthorizableLookup;
import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.ControllerServiceReferencingComponentAuthorizable;
import org.apache.nifi.authorization.ConfigurableComponentAuthorizable;
import org.apache.nifi.authorization.ProcessGroupAuthorizable;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
@ -615,12 +615,18 @@ public class ProcessGroupResource extends ApplicationResource {
serviceFacade,
requestProcessorEntity,
lookup -> {
final NiFiUser user = NiFiUserUtils.getNiFiUser();
final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable();
processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
processGroup.authorize(authorizer, RequestAction.WRITE, user);
final ConfigurableComponentAuthorizable authorizable = lookup.getProcessorByType(requestProcessor.getType());
if (authorizable.isRestricted()) {
lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, user);
}
final ProcessorConfigDTO config = requestProcessor.getConfig();
if (config != null && config.getProperties() != null) {
final ControllerServiceReferencingComponentAuthorizable authorizable = lookup.getProcessorByType(requestProcessor.getType());
AuthorizeControllerServiceReference.authorizeControllerServiceReferences(config.getProperties(), authorizable, authorizer, lookup);
}
},
@ -2129,11 +2135,17 @@ public class ProcessGroupResource extends ApplicationResource {
serviceFacade,
requestControllerServiceEntity,
lookup -> {
final NiFiUser user = NiFiUserUtils.getNiFiUser();
final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable();
processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
processGroup.authorize(authorizer, RequestAction.WRITE, user);
final ConfigurableComponentAuthorizable authorizable = lookup.getProcessorByType(requestControllerService.getType());
if (authorizable.isRestricted()) {
lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, user);
}
if (requestControllerService.getProperties() != null) {
final ControllerServiceReferencingComponentAuthorizable authorizable = lookup.getControllerServiceByType(requestControllerService.getType());
AuthorizeControllerServiceReference.authorizeControllerServiceReferences(requestControllerService.getProperties(), authorizable, authorizer, lookup);
}
},

View File

@ -25,7 +25,7 @@ import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.ControllerServiceReferencingComponentAuthorizable;
import org.apache.nifi.authorization.ConfigurableComponentAuthorizable;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.authorization.user.NiFiUser;
@ -448,7 +448,7 @@ public class ProcessorResource extends ApplicationResource {
lookup -> {
final NiFiUser user = NiFiUserUtils.getNiFiUser();
final ControllerServiceReferencingComponentAuthorizable authorizable = lookup.getProcessor(id);
final ConfigurableComponentAuthorizable authorizable = lookup.getProcessor(id);
authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, user);
final ProcessorConfigDTO config = requestProcessorDTO.getConfig();

View File

@ -25,7 +25,7 @@ import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.ControllerServiceReferencingComponentAuthorizable;
import org.apache.nifi.authorization.ConfigurableComponentAuthorizable;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.authorization.user.NiFiUserUtils;
@ -425,7 +425,7 @@ public class ReportingTaskResource extends ApplicationResource {
requestRevision,
lookup -> {
// authorize reporting task
final ControllerServiceReferencingComponentAuthorizable authorizable = lookup.getReportingTask(id);
final ConfigurableComponentAuthorizable authorizable = lookup.getReportingTask(id);
authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
// authorize any referenced services

View File

@ -31,6 +31,7 @@ import org.apache.nifi.action.details.FlowChangeMoveDetails;
import org.apache.nifi.action.details.FlowChangePurgeDetails;
import org.apache.nifi.action.details.MoveDetails;
import org.apache.nifi.action.details.PurgeDetails;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.behavior.Stateful;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
@ -2011,6 +2012,11 @@ public final class DtoFactory {
return dto;
}
private String getUsageRestriction(final Class<?> cls) {
final Restricted restriction = cls.getAnnotation(Restricted.class);
return restriction == null ? null : restriction.value();
}
/**
* Gets the capability description from the specified class.
*/
@ -2050,6 +2056,7 @@ public final class DtoFactory {
final DocumentedTypeDTO type = new DocumentedTypeDTO();
type.setType(cls.getName());
type.setDescription(getCapabilityDescription(cls));
type.setUsageRestriction(getUsageRestriction(cls));
type.setTags(getTags(cls));
types.add(type);
}

View File

@ -788,6 +788,7 @@ public class ControllerFacade implements Authorizable {
final List<Resource> resources = new ArrayList<>();
resources.add(ResourceFactory.getFlowResource());
resources.add(ResourceFactory.getSystemResource());
resources.add(ResourceFactory.getRestrictedComponentsResource());
resources.add(ResourceFactory.getControllerResource());
resources.add(ResourceFactory.getCountersResource());
resources.add(ResourceFactory.getProvenanceResource());

View File

@ -43,6 +43,7 @@ public class AccessControlHelper {
private NiFiTestUser writeUser;
private NiFiTestUser readWriteUser;
private NiFiTestUser noneUser;
private NiFiTestUser restrictedUser;
private static final String CONTEXT_PATH = "/nifi-api";
@ -78,6 +79,7 @@ public class AccessControlHelper {
writeUser = new NiFiTestUser(server.getClient(), NiFiTestAuthorizer.WRITE_USER_DN);
readWriteUser = new NiFiTestUser(server.getClient(), NiFiTestAuthorizer.READ_WRITE_USER_DN);
noneUser = new NiFiTestUser(server.getClient(), NiFiTestAuthorizer.NONE_USER_DN);
restrictedUser = new NiFiTestUser(server.getClient(), NiFiTestAuthorizer.RESTRICTED_USER_DN);
// populate the initial data flow
NiFiWebApiTest.populateFlow(server.getClient(), baseUrl, readWriteUser, READ_WRITE_CLIENT_ID);
@ -99,6 +101,10 @@ public class AccessControlHelper {
return noneUser;
}
public NiFiTestUser getRestrictedUser() {
return restrictedUser;
}
public void testGenericGetUri(final String uri) throws Exception {
ClientResponse response;

View File

@ -19,6 +19,7 @@ package org.apache.nifi.integration.accesscontrol;
import com.sun.jersey.api.client.ClientResponse;
import org.apache.nifi.integration.util.NiFiTestAuthorizer;
import org.apache.nifi.integration.util.NiFiTestUser;
import org.apache.nifi.integration.util.RestrictedProcessor;
import org.apache.nifi.integration.util.SourceTestProcessor;
import org.apache.nifi.web.api.dto.ProcessorDTO;
import org.apache.nifi.web.api.dto.RevisionDTO;
@ -399,6 +400,43 @@ public class ITProcessorAccessControl {
verifyDelete(helper.getNoneUser(), NONE_CLIENT_ID, 403);
}
/**
* Tests attempt to create a restricted processor.
*
* @throws Exception
*/
@Test
public void testCreateRestrictedProcessor() throws Exception {
String url = helper.getBaseUrl() + "/process-groups/root/processors";
// create the processor
ProcessorDTO processor = new ProcessorDTO();
processor.setName("restricted");
processor.setType(RestrictedProcessor.class.getName());
// create the revision
final RevisionDTO revision = new RevisionDTO();
revision.setClientId(READ_WRITE_CLIENT_ID);
revision.setVersion(0L);
// create the entity body
ProcessorEntity entity = new ProcessorEntity();
entity.setRevision(revision);
entity.setComponent(processor);
// perform the request as a user with read/write but no restricted access
ClientResponse response = helper.getReadWriteUser().testPost(url, entity);
// ensure the request is successful
assertEquals(403, response.getStatus());
// perform the request as a user with read/write and restricted access
response = helper.getRestrictedUser().testPost(url, entity);
// ensure the request is successful
assertEquals(201, response.getStatus());
}
private ProcessorEntity getRandomProcessor(final NiFiTestUser user) throws Exception {
final String url = helper.getBaseUrl() + "/flow/process-groups/root";

View File

@ -39,6 +39,7 @@ public class NiFiTestAuthorizer implements Authorizer {
public static final String READ_USER_DN = "read@nifi";
public static final String WRITE_USER_DN = "write@nifi";
public static final String READ_WRITE_USER_DN = "readwrite@nifi";
public static final String RESTRICTED_USER_DN = "restricted@nifi";
public static final String TOKEN_USER = "user@nifi";
@ -78,15 +79,24 @@ public class NiFiTestAuthorizer implements Authorizer {
return AuthorizationResult.approved();
}
// restricted component access
if (ResourceFactory.getRestrictedComponentsResource().getIdentifier().equals(request.getResource().getIdentifier())) {
if (RESTRICTED_USER_DN.equals(request.getIdentity())) {
return AuthorizationResult.approved();
} else {
return AuthorizationResult.denied();
}
}
// read access
if (READ_USER_DN.equals(request.getIdentity()) || READ_WRITE_USER_DN.equals(request.getIdentity())) {
if (READ_USER_DN.equals(request.getIdentity()) || READ_WRITE_USER_DN.equals(request.getIdentity()) || RESTRICTED_USER_DN.equals(request.getIdentity())) {
if (RequestAction.READ.equals(request.getAction())) {
return AuthorizationResult.approved();
}
}
// write access
if (WRITE_USER_DN.equals(request.getIdentity()) || READ_WRITE_USER_DN.equals(request.getIdentity())) {
if (WRITE_USER_DN.equals(request.getIdentity()) || READ_WRITE_USER_DN.equals(request.getIdentity()) || RESTRICTED_USER_DN.equals(request.getIdentity())) {
if (RequestAction.WRITE.equals(request.getAction())) {
return AuthorizationResult.approved();
}

View File

@ -0,0 +1,51 @@
/*
* 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.integration.util;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSessionFactory;
import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import java.util.HashSet;
import java.util.Set;
@Restricted("")
public class RestrictedProcessor extends AbstractSessionFactoryProcessor {
public RestrictedProcessor() {
}
@Override
public Set<Relationship> getRelationships() {
final Set<Relationship> rels = new HashSet<>();
rels.add(new Relationship.Builder().name("success").build());
return rels;
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
}
@Override
protected void init(final ProcessorInitializationContext context) {
}
}

View File

@ -427,7 +427,13 @@ div.button {
cursor: pointer;
}
div.button:hover {
div.button.disabled-button {
color: #a8a8a8 !important;
cursor: not-allowed;
opacity: 0.7;
}
div.button:hover:not(.disabled-button) {
background-color: #004849;
}

View File

@ -89,7 +89,7 @@ div.policy-controls {
#policy-type-list {
float: left;
width: 190px;
width: 225px;
margin-right: 3px;
border-left: 1px solid transparent;
border-right: 1px solid transparent;

View File

@ -36,6 +36,7 @@
* hover: '#004849',
* text: '#ffffff'
* },
* disabled: isDisabledFunction,
* handler: {
* click: cancelHandler
* }
@ -46,6 +47,7 @@
* hover: '#C7D2D7',
* text: '#004849'
* },
* disabled: isDisabledFunction,
* handler: {
* click: applyHandler
* }
@ -101,27 +103,54 @@
var addButtons = function (dialog, buttonModel) {
if (isDefinedAndNotNull(buttonModel)) {
var buttonWrapper = $('<div class="dialog-buttons"></div>');
var button;
$.each(buttonModel, function (i, buttonConfig) {
var clazz = isDefinedAndNotNull(buttonConfig.clazz) ? buttonConfig.clazz : '';
if (buttonConfig.color) {
button = $('<div class="button ' + clazz + '" style="color:' + buttonConfig.color.text + '; background:' + buttonConfig.color.base + ';"><span>' + buttonConfig.buttonText + '</span></div>');
var isDisabled = function () {
return typeof buttonConfig.disabled === 'function' && buttonConfig.disabled.call() === true;
};
// create the button
var button = $('<div class="button"></div>').append($('<span></span>').text(buttonConfig.buttonText));
// add the class if specified
if (isDefinedAndNotNull(buttonConfig.clazz)) {
button.addClass(buttonConfig.clazz);
}
// set the color if specified
if (isDefinedAndNotNull(buttonConfig.color)) {
button.css({
'background': buttonConfig.color.base,
'color': buttonConfig.color.text
});
}
// check if the button should be disabled
if (isDisabled()) {
button.addClass('disabled-button');
} else {
// enable custom hover if specified
if (isDefinedAndNotNull(buttonConfig.color)) {
button.hover(function () {
$(this).css("background-color", buttonConfig.color.hover);
}, function () {
$(this).css("background-color", buttonConfig.color.base);
});
} else {
button = $('<div class="button ' + clazz + '"><span>' + buttonConfig.buttonText + '</span></div>');
}
button.click(function () {
var handler = $(this).data('handler');
if (isDefinedAndNotNull(handler) && typeof handler.click === 'function') {
handler.click.call(dialog);
}
}).data('handler', buttonConfig.handler).appendTo(buttonWrapper);
});
buttonWrapper.appendTo(dialog);
}
// add the button to the wrapper
button.data('handler', buttonConfig.handler).appendTo(buttonWrapper);
});
// store the button model to refresh later
dialog.append(buttonWrapper).data('buttonModel', buttonModel);
}
};
@ -258,6 +287,22 @@
});
},
/**
* Refreshes the buttons with the existing model.
*/
refreshButtons: function () {
return this.each(function () {
var dialog = $(this);
var buttons = dialog.data('buttonModel');
// remove the current buttons
dialog.children('.dialog-buttons').remove();
// add the new buttons
addButtons(dialog, buttons);
});
},
/**
* Sets the header text of the dialog.
*

View File

@ -235,6 +235,15 @@ nf.ng.ProcessorComponent = function (serviceProvider) {
}).fail(nf.Common.handleAjaxError);
};
/**
* Whether the specified item is selectable.
*
* @param item process type
*/
var isSelectable = function (item) {
return nf.Common.isBlank(item.usageRestriction) || nf.Common.canAccessRestrictedComponents();
};
function ProcessorComponent() {
this.icon = 'icon icon-processor';
@ -257,7 +266,7 @@ nf.ng.ProcessorComponent = function (serviceProvider) {
init: function () {
// initialize the processor type table
var processorTypesColumns = [
{id: 'type', name: 'Type', field: 'label', sortable: true, resizable: true},
{id: 'type', name: 'Type', field: 'label', formatter: nf.Common.typeFormatter, sortable: true, resizable: true},
{id: 'tags', name: 'Tags', field: 'tags', sortable: true, resizable: true}
];
var processorTypesOptions = {
@ -319,9 +328,15 @@ nf.ng.ProcessorComponent = function (serviceProvider) {
$('#processor-type-name').text(processorType.label).ellipsis();
$('#selected-processor-name').text(processorType.label);
$('#selected-processor-type').text(processorType.type);
// refresh the buttons based on the current selection
$('#new-processor-dialog').modal('refreshButtons');
}
}
});
processorTypesGrid.onViewportChanged.subscribe(function (e, args) {
nf.Common.cleanUpTooltips($('#processor-types-table'), 'div.view-usage-restriction');
});
// wire up the dataview to the grid
processorTypesData.onRowCountChanged.subscribe(function (e, args) {
@ -338,7 +353,31 @@ nf.ng.ProcessorComponent = function (serviceProvider) {
processorTypesData.syncGridSelection(processorTypesGrid, false);
// hold onto an instance of the grid
$('#processor-types-table').data('gridInstance', processorTypesGrid);
$('#processor-types-table').data('gridInstance', processorTypesGrid).on('mouseenter', 'div.slick-cell', function (e) {
var usageRestriction = $(this).find('div.view-usage-restriction');
if (usageRestriction.length && !usageRestriction.data('qtip')) {
var rowId = $(this).find('span.row-id').text();
// get the status item
var item = processorTypesData.getItemById(rowId);
// show the tooltip
if (nf.Common.isDefinedAndNotNull(item.usageRestriction)) {
usageRestriction.qtip($.extend({}, nf.Common.config.tooltipConfig, {
content: item.usageRestriction,
position: {
container: $('#summary'),
at: 'bottom right',
my: 'top left',
adjust: {
x: 4,
y: 4
}
}
}));
}
}
});
// load the available processor types, this select is shown in the
// new processor dialog when a processor is dragged onto the screen
@ -362,6 +401,7 @@ nf.ng.ProcessorComponent = function (serviceProvider) {
label: nf.Common.substringAfterLast(type, '.'),
type: type,
description: nf.Common.escapeHtml(documentedType.description),
usageRestriction: nf.Common.escapeHtml(documentedType.usageRestriction),
tags: documentedType.tags.join(', ')
});
@ -497,6 +537,7 @@ nf.ng.ProcessorComponent = function (serviceProvider) {
*/
promptForProcessorType: function (pt) {
var self = this;
// handles adding the selected processor at the specified point
var addProcessor = function () {
// get the type of processor currently selected
@ -525,10 +566,12 @@ nf.ng.ProcessorComponent = function (serviceProvider) {
var gridDoubleClick = function (e, args) {
var processorType = grid.getDataItem(args.row);
if (isSelectable(processorType)) {
$('#selected-processor-name').text(processorType.label);
$('#selected-processor-type').text(processorType.type);
addProcessor();
}
};
// register a handler for double click events
@ -542,6 +585,17 @@ nf.ng.ProcessorComponent = function (serviceProvider) {
hover: '#004849',
text: '#ffffff'
},
disabled: function () {
var selected = grid.getSelectedRows();
if (selected.length > 0) {
// grid configured with multi-select = false
var item = grid.getDataItem(selected[0]);
return isSelectable(item) === false;
} else {
return false;
}
},
handler: {
click: addProcessor
}
@ -576,7 +630,15 @@ nf.ng.ProcessorComponent = function (serviceProvider) {
$('#processor-type-filter').focus().off('keyup').on('keyup', function (e) {
var code = e.keyCode ? e.keyCode : e.which;
if (code === $.ui.keyCode.ENTER) {
var selected = grid.getSelectedRows();
if (selected.length > 0) {
// grid configured with multi-select = false
var item = grid.getDataItem(selected[0]);
if (isSelectable(item)) {
addProcessor();
}
}
} else {
applyFilter();
}

View File

@ -39,6 +39,15 @@ nf.ControllerServices = (function () {
rowHeight: 24
};
/**
* Whether the specified item is selectable.
*
* @param item controller service type
*/
var isSelectable = function (item) {
return nf.Common.isBlank(item.usageRestriction) || nf.Common.canAccessRestrictedComponents();
};
/**
* Get the text out of the filter field. If the filter field doesn't
* have any text it will contain the text 'filter list' so this method
@ -254,7 +263,7 @@ nf.ControllerServices = (function () {
var initNewControllerServiceDialog = function () {
// initialize the processor type table
var controllerServiceTypesColumns = [
{id: 'type', name: 'Type', field: 'label', sortable: false, resizable: true},
{id: 'type', name: 'Type', field: 'label', formatter: nf.Common.typeFormatter, sortable: false, resizable: true},
{id: 'tags', name: 'Tags', field: 'tags', sortable: false, resizable: true}
];
@ -298,9 +307,15 @@ nf.ControllerServices = (function () {
$('#controller-service-type-name').text(controllerServiceType.label).ellipsis();
$('#selected-controller-service-name').text(controllerServiceType.label);
$('#selected-controller-service-type').text(controllerServiceType.type);
// refresh the buttons based on the current selection
$('#new-controller-service-dialog').modal('refreshButtons');
}
}
});
controllerServiceTypesGrid.onViewportChanged.subscribe(function (e, args) {
nf.Common.cleanUpTooltips($('#controller-service-types-table'), 'div.view-usage-restriction');
});
// wire up the dataview to the grid
controllerServiceTypesData.onRowCountChanged.subscribe(function (e, args) {
@ -317,7 +332,31 @@ nf.ControllerServices = (function () {
controllerServiceTypesData.syncGridSelection(controllerServiceTypesGrid, true);
// hold onto an instance of the grid
$('#controller-service-types-table').data('gridInstance', controllerServiceTypesGrid);
$('#controller-service-types-table').data('gridInstance', controllerServiceTypesGrid).on('mouseenter', 'div.slick-cell', function (e) {
var usageRestriction = $(this).find('div.view-usage-restriction');
if (usageRestriction.length && !usageRestriction.data('qtip')) {
var rowId = $(this).find('span.row-id').text();
// get the status item
var item = controllerServiceTypesData.getItemById(rowId);
// show the tooltip
if (nf.Common.isDefinedAndNotNull(item.usageRestriction)) {
usageRestriction.qtip($.extend({}, nf.Common.config.tooltipConfig, {
content: item.usageRestriction,
position: {
container: $('#summary'),
at: 'bottom right',
my: 'top left',
adjust: {
x: 4,
y: 4
}
}
}));
}
}
});
// load the available controller services
$.ajax({
@ -339,6 +378,7 @@ nf.ControllerServices = (function () {
label: nf.Common.substringAfterLast(documentedType.type, '.'),
type: documentedType.type,
description: nf.Common.escapeHtml(documentedType.description),
usageRestriction: nf.Common.escapeHtml(documentedType.usageRestriction),
tags: documentedType.tags.join(', ')
});
@ -831,11 +871,22 @@ nf.ControllerServices = (function () {
* @param {jQuery} serviceTable
*/
promptNewControllerService: function (controllerServicesUri, serviceTable) {
// get the grid reference
var grid = $('#controller-service-types-table').data('gridInstance');
// update the keyhandler
$('#controller-service-type-filter').off('keyup').on('keyup', function (e) {
var code = e.keyCode ? e.keyCode : e.which;
if (code === $.ui.keyCode.ENTER) {
var selected = grid.getSelectedRows();
if (selected.length > 0) {
// grid configured with multi-select = false
var item = grid.getDataItem(selected[0]);
if (isSelectable(item)) {
addSelectedControllerService(controllerServicesUri, serviceTable);
}
}
} else {
applyControllerServiceTypeFilter();
}
@ -849,6 +900,17 @@ nf.ControllerServices = (function () {
hover: '#004849',
text: '#ffffff'
},
disabled: function () {
var selected = grid.getSelectedRows();
if (selected.length > 0) {
// grid configured with multi-select = false
var item = grid.getDataItem(selected[0]);
return isSelectable(item) === false;
} else {
return false;
}
},
handler: {
click: function () {
addSelectedControllerService(controllerServicesUri, serviceTable);
@ -878,7 +940,10 @@ nf.ControllerServices = (function () {
// update the dbl click handler and subsrcibe
dblClick = function(e, args) {
var controllerServiceType = controllerServiceTypesGrid.getDataItem(args.row);
if (isSelectable(controllerServiceType)) {
addControllerService(controllerServicesUri, serviceTable, controllerServiceType.type);
}
};
controllerServiceTypesGrid.onDblClick.subscribe(dblClick);

View File

@ -343,6 +343,7 @@ nf.PolicyManagement = (function () {
nf.Common.getPolicyTypeListing('flow'),
nf.Common.getPolicyTypeListing('controller'),
nf.Common.getPolicyTypeListing('provenance'),
nf.Common.getPolicyTypeListing('restricted-components'),
nf.Common.getPolicyTypeListing('policies'),
nf.Common.getPolicyTypeListing('tenants'),
nf.Common.getPolicyTypeListing('site-to-site'),
@ -364,7 +365,7 @@ nf.PolicyManagement = (function () {
$('#controller-policy-target').hide();
// record the action
if (option.value === 'proxy') {
if (option.value === 'proxy' || option.value === 'restricted-components') {
$('#selected-policy-action').text('write');
} else {
$('#selected-policy-action').text('read');
@ -1398,7 +1399,7 @@ nf.PolicyManagement = (function () {
if (policyType === 'controller') {
$('#selected-policy-action').text($('#controller-policy-target').combo('getSelectedOption').value);
} else if (policyType === 'proxy') {
} else if (policyType === 'proxy' || policyType === 'restricted-components') {
$('#selected-policy-action').text('write');
} else {
$('#selected-policy-action').text('read');

View File

@ -155,6 +155,15 @@ nf.Settings = (function () {
return matches;
};
/**
* Whether the specified item is selectable.
*
* @param item reporting task type
*/
var isSelectable = function (item) {
return nf.Common.isBlank(item.usageRestriction) || nf.Common.canAccessRestrictedComponents();
};
/**
* Formatter for the name column.
*
@ -406,7 +415,17 @@ nf.Settings = (function () {
$('#reporting-task-type-filter').on('keyup', function (e) {
var code = e.keyCode ? e.keyCode : e.which;
if (code === $.ui.keyCode.ENTER) {
// get the grid reference
var grid = $('#reporting-task-types-table').data('gridInstance');
var selected = grid.getSelectedRows();
if (selected.length > 0) {
// grid configured with multi-select = false
var item = grid.getDataItem(selected[0]);
if (isSelectable(item)) {
addSelectedReportingTask();
}
}
} else {
applyReportingTaskTypeFilter();
}
@ -414,7 +433,7 @@ nf.Settings = (function () {
// initialize the processor type table
var reportingTaskTypesColumns = [
{id: 'type', name: 'Type', field: 'label', sortable: false, resizable: true},
{id: 'type', name: 'Type', field: 'label', formatter: nf.Common.typeFormatter, sortable: false, resizable: true},
{id: 'tags', name: 'Tags', field: 'tags', sortable: false, resizable: true}
];
@ -458,12 +477,21 @@ nf.Settings = (function () {
$('#reporting-task-type-name').text(reportingTaskType.label).ellipsis();
$('#selected-reporting-task-name').text(reportingTaskType.label);
$('#selected-reporting-task-type').text(reportingTaskType.type);
// refresh the buttons based on the current selection
$('#new-reporting-task-dialog').modal('refreshButtons');
}
}
});
reportingTaskTypesGrid.onDblClick.subscribe(function (e, args) {
var reportingTaskType = reportingTaskTypesGrid.getDataItem(args.row);
if (isSelectable(reportingTaskType)) {
addReportingTask(reportingTaskType.type);
}
});
reportingTaskTypesGrid.onViewportChanged.subscribe(function (e, args) {
nf.Common.cleanUpTooltips($('#reporting-task-types-table'), 'div.view-usage-restriction');
});
// wire up the dataview to the grid
@ -481,7 +509,31 @@ nf.Settings = (function () {
reportingTaskTypesData.syncGridSelection(reportingTaskTypesGrid, true);
// hold onto an instance of the grid
$('#reporting-task-types-table').data('gridInstance', reportingTaskTypesGrid);
$('#reporting-task-types-table').data('gridInstance', reportingTaskTypesGrid).on('mouseenter', 'div.slick-cell', function (e) {
var usageRestriction = $(this).find('div.view-usage-restriction');
if (usageRestriction.length && !usageRestriction.data('qtip')) {
var rowId = $(this).find('span.row-id').text();
// get the status item
var item = reportingTaskTypesData.getItemById(rowId);
// show the tooltip
if (nf.Common.isDefinedAndNotNull(item.usageRestriction)) {
usageRestriction.qtip($.extend({}, nf.Common.config.tooltipConfig, {
content: item.usageRestriction,
position: {
container: $('#summary'),
at: 'bottom right',
my: 'top left',
adjust: {
x: 4,
y: 4
}
}
}));
}
}
});
// load the available reporting tasks
$.ajax({
@ -503,6 +555,7 @@ nf.Settings = (function () {
label: nf.Common.substringAfterLast(documentedType.type, '.'),
type: documentedType.type,
description: nf.Common.escapeHtml(documentedType.description),
usageRestriction: nf.Common.escapeHtml(documentedType.usageRestriction),
tags: documentedType.tags.join(', ')
});
@ -537,6 +590,17 @@ nf.Settings = (function () {
hover: '#004849',
text: '#ffffff'
},
disabled: function () {
var selected = reportingTaskTypesGrid.getSelectedRows();
if (selected.length > 0) {
// grid configured with multi-select = false
var item = reportingTaskTypesGrid.getDataItem(selected[0]);
return isSelectable(item) === false;
} else {
return false;
}
},
handler: {
click: function () {
addSelectedReportingTask();

View File

@ -90,6 +90,10 @@ nf.Common = (function () {
text: 'query provenance',
value: 'provenance',
description: 'Allows users to submit a Provenance Search and request Event Lineage'
}, {
text: 'access restricted components',
value: 'restricted-components',
description: 'Allows users to create/modify restricted components assuming otherwise sufficient permissions'
}, {
text: 'access all policies',
value: 'policies',
@ -156,6 +160,32 @@ nf.Common = (function () {
*/
currentUser: undefined,
/**
* Formats type of a component for a new instance dialog.
*
* @param row
* @param cell
* @param value
* @param columnDef
* @param dataContext
* @returns {string}
*/
typeFormatter: function (row, cell, value, columnDef, dataContext) {
var markup = '';
// restriction
if (nf.Common.isBlank(dataContext.usageRestriction) === false) {
markup += '<div class="view-usage-restriction fa fa-shield"></div><span class="hidden row-id">' + nf.Common.escapeHtml(dataContext.id) + '</span>';
} else {
markup += '<div class="fa"></div>';
}
// type
markup += value;
return markup;
},
/**
* Sets the current user.
*
@ -271,6 +301,19 @@ nf.Common = (function () {
}
},
/**
* Determines whether the current user can access restricted comopnents.
*
* @returns {boolean}
*/
canAccessRestrictedComponents: function () {
if (nf.Common.isDefinedAndNotNull(nf.Common.currentUser)) {
return nf.Common.currentUser.restrictedComponentsPermissions.canWrite === true;
} else {
return false;
}
},
/**
* Determines whether the current user can access counters.
*

View File

@ -16,20 +16,13 @@
*/
package org.apache.nifi.processors.hadoop;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
@ -41,8 +34,15 @@ import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@TriggerWhenEmpty
@InputRequirement(InputRequirement.Requirement.INPUT_ALLOWED)
@ -51,6 +51,7 @@ import com.google.common.collect.Maps;
+ "or a statically set file that is periodically removed. If this processor has an incoming connection, it"
+ "will ignore running on a periodic basis and instead rely on incoming FlowFiles to trigger a delete. "
+ "Optionally, you may specify use a wildcard character to match multiple files or directories.")
@Restricted("Provides operator the ability to delete any file that NiFi has access to in HDFS or the local filesystem.")
public class DeleteHDFS extends AbstractHadoopProcessor {
public static final Relationship REL_SUCCESS = new Relationship.Builder()
.name("success")

View File

@ -26,6 +26,7 @@ import org.apache.hadoop.io.compress.CompressionCodecFactory;
import org.apache.hadoop.security.AccessControlException;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.behavior.SupportsBatching;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
@ -59,6 +60,7 @@ import java.util.concurrent.TimeUnit;
@WritesAttribute(attribute="hdfs.failure.reason", description="When a FlowFile is routed to 'failure', this attribute is added indicating why the file could "
+ "not be fetched from HDFS")
@SeeAlso({ListHDFS.class, GetHDFS.class, PutHDFS.class})
@Restricted("Provides operator the ability to retrieve any file that NiFi has access to in HDFS or the local filesystem.")
public class FetchHDFS extends AbstractHadoopProcessor {
static final PropertyDescriptor FILENAME = new PropertyDescriptor.Builder()

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.CompressionCodecFactory;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
@ -73,6 +74,7 @@ import java.util.regex.Pattern;
+ "is set to /tmp, then files picked up from /tmp will have the path attribute set to \"./\". If the Recurse Subdirectories property is set to true and "
+ "a file is picked up from /tmp/abc/1/2/3, then the path attribute will be set to \"abc/1/2/3\".") })
@SeeAlso({PutHDFS.class, ListHDFS.class})
@Restricted("Provides operator the ability to retrieve and delete any file that NiFi has access to in HDFS or the local filesystem.")
public class GetHDFS extends AbstractHadoopProcessor {
public static final String BUFFER_SIZE_KEY = "io.file.buffer.size";

View File

@ -26,6 +26,7 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.ReadsAttribute;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
@ -75,6 +76,7 @@ import java.util.concurrent.TimeUnit;
@WritesAttribute(attribute = "absolute.hdfs.path", description = "The absolute path to the file on HDFS is stored in this attribute.")
})
@SeeAlso(GetHDFS.class)
@Restricted("Provides operator the ability to write to any file that NiFi has access to in HDFS or the local filesystem.")
public class PutHDFS extends AbstractHadoopProcessor {
public static final String REPLACE_RESOLUTION = "replace";

View File

@ -16,25 +16,14 @@
*/
package org.apache.nifi.processors.script;
import java.io.FileInputStream;
import java.io.IOException;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import javax.script.Bindings;
import javax.script.ScriptContext;
import javax.script.ScriptEngine;
import javax.script.ScriptException;
import javax.script.SimpleBindings;
import org.apache.commons.io.IOUtils;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessSessionFactory;
@ -43,6 +32,19 @@ import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.util.StringUtils;
import javax.script.Bindings;
import javax.script.ScriptContext;
import javax.script.ScriptEngine;
import javax.script.ScriptException;
import javax.script.SimpleBindings;
import java.io.FileInputStream;
import java.io.IOException;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@Tags({"script", "execute", "groovy", "python", "jython", "jruby", "ruby", "javascript", "js", "lua", "luaj"})
@CapabilityDescription("Experimental - Executes a script given the flow file and a process session. The script is responsible for "
+ "handling the incoming flow file (transfer to SUCCESS or remove, e.g.) as well as any flow files created by "
@ -54,6 +56,7 @@ import org.apache.nifi.util.StringUtils;
supportsExpressionLanguage = true,
description = "Updates a script engine property specified by the Dynamic Property's key with the value "
+ "specified by the Dynamic Property's value")
@Restricted("Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
public class ExecuteScript extends AbstractScriptProcessor {
private String scriptToRun = null;

View File

@ -16,24 +16,10 @@
*/
package org.apache.nifi.processors.script;
import java.io.File;
import java.io.FileInputStream;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import javax.script.Invocable;
import javax.script.ScriptEngine;
import javax.script.ScriptException;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
@ -53,6 +39,20 @@ import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import javax.script.Invocable;
import javax.script.ScriptEngine;
import javax.script.ScriptException;
import java.io.File;
import java.io.FileInputStream;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
@Tags({"script", "invoke", "groovy", "python", "jython", "jruby", "ruby", "javascript", "js", "lua", "luaj"})
@CapabilityDescription("Experimental - Invokes a script engine for a Processor defined in the given script. The script must define "
+ "a valid class that implements the Processor interface, and it must set a variable 'processor' to an instance of "
@ -62,6 +62,7 @@ import org.apache.nifi.processor.util.StandardValidators;
@DynamicProperty(name = "A script engine property to update", value = "The value to set it to", supportsExpressionLanguage = true,
description = "Updates a script engine property specified by the Dynamic Property's key with the value specified by the Dynamic Property's value")
@SeeAlso({ExecuteScript.class})
@Restricted("Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
public class InvokeScriptedProcessor extends AbstractScriptProcessor {
private final AtomicReference<Processor> processor = new AtomicReference<>();

View File

@ -17,6 +17,7 @@
package org.apache.nifi.reporting;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.behavior.Stateful;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
@ -58,6 +59,7 @@ import java.util.concurrent.TimeUnit;
@Tags({"provenance", "lineage", "tracking", "site", "site to site"})
@CapabilityDescription("Publishes Provenance events using the Site To Site protocol.")
@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
@Restricted("Provides operator the ability send sensitive details contained in Provenance events to any external system.")
public class SiteToSiteProvenanceReportingTask extends AbstractSiteToSiteReportingTask {
static final String TIMESTAMP_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'";

View File

@ -16,6 +16,28 @@
*/
package org.apache.nifi.processors.standard;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.Validator;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.OutputStreamCallback;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.standard.util.ArgumentUtils;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.BufferedReader;
@ -41,33 +63,13 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.Validator;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.OutputStreamCallback;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.standard.util.ArgumentUtils;
@InputRequirement(Requirement.INPUT_FORBIDDEN)
@Tags({"command", "process", "source", "external", "invoke", "script"})
@CapabilityDescription("Runs an operating system command specified by the user and writes the output of that command to a FlowFile. If the command is expected "
+ "to be long-running, the Processor can output the partial data on a specified interval. When this option is used, the output is expected to be in textual "
+ "format, as it typically does not make sense to split binary data on arbitrary time-based intervals.")
@DynamicProperty(name = "An environment variable name", value = "An environment variable value", description = "These environment variables are passed to the process spawned by this Processor")
@Restricted("Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
public class ExecuteProcess extends AbstractProcessor {
public static final PropertyDescriptor COMMAND = new PropertyDescriptor.Builder()

View File

@ -16,28 +16,13 @@
*/
package org.apache.nifi.processors.standard;
import java.io.BufferedReader;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.OutputStream;
import java.lang.ProcessBuilder.Redirect;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.EventDriven;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.behavior.SupportsBatching;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
@ -65,6 +50,22 @@ import org.apache.nifi.stream.io.BufferedInputStream;
import org.apache.nifi.stream.io.BufferedOutputStream;
import org.apache.nifi.stream.io.StreamUtils;
import java.io.BufferedReader;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.OutputStream;
import java.lang.ProcessBuilder.Redirect;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
/**
* <p>
* This processor executes an external command on the contents of a flow file, and creates a new flow file with the results of the command.
@ -132,6 +133,7 @@ import org.apache.nifi.stream.io.StreamUtils;
@WritesAttribute(attribute = "execution.command.args", description = "The semi-colon delimited list of arguments"),
@WritesAttribute(attribute = "execution.status", description = "The exit status code returned from executing the command"),
@WritesAttribute(attribute = "execution.error", description = "Any error messages returned from executing the command")})
@Restricted("Provides operator the ability to execute arbitrary code assuming all permissions that NiFi has.")
public class ExecuteStreamCommand extends AbstractProcessor {
public static final Relationship ORIGINAL_RELATIONSHIP = new Relationship.Builder()

View File

@ -17,24 +17,10 @@
package org.apache.nifi.processors.standard;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.nio.file.CopyOption;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.StandardCopyOption;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
@ -52,11 +38,27 @@ import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.util.StopWatch;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.nio.file.CopyOption;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.StandardCopyOption;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
@InputRequirement(Requirement.INPUT_REQUIRED)
@Tags({"local", "files", "filesystem", "ingest", "ingress", "get", "source", "input"})
@CapabilityDescription("Reads the contents of a file from disk and streams it into the contents of an incoming FlowFile. Once this is done, the file is optionally moved elsewhere or deleted "
+ "to help keep the file system organized.")
@SeeAlso({GetFile.class, PutFile.class, ListFile.class})
@Restricted("Provides operator the ability to read from and delete any file that NiFi has access to.")
public class FetchFile extends AbstractProcessor {
static final AllowableValue COMPLETION_NONE = new AllowableValue("None", "None", "Leave the file as-is");
static final AllowableValue COMPLETION_MOVE = new AllowableValue("Move File", "Move File", "Moves the file to the directory specified by the <Move Destination Directory> property");

View File

@ -16,6 +16,29 @@
*/
package org.apache.nifi.processors.standard;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import java.io.File;
import java.io.FileFilter;
import java.io.IOException;
@ -48,27 +71,6 @@ import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.regex.Pattern;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
@TriggerWhenEmpty
@InputRequirement(Requirement.INPUT_FORBIDDEN)
@ -91,6 +93,7 @@ import org.apache.nifi.processor.util.StandardValidators;
@WritesAttribute(attribute = "absolute.path", description = "The full/absolute path from where a file was picked up. The current 'path' "
+ "attribute is still populated, but may be a relative path")})
@SeeAlso({PutFile.class, FetchFile.class})
@Restricted("Provides operator the ability to read from and delete any file that NiFi has access to.")
public class GetFile extends AbstractProcessor {
public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder()

View File

@ -17,6 +17,27 @@
package org.apache.nifi.processors.standard;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.Stateful;
import org.apache.nifi.annotation.behavior.TriggerSerially;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.standard.util.FileInfo;
import java.io.File;
import java.io.FileFilter;
import java.io.IOException;
@ -44,27 +65,6 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.regex.Pattern;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.Stateful;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.TriggerSerially;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.standard.util.FileInfo;
@TriggerSerially
@InputRequirement(Requirement.INPUT_FORBIDDEN)
@Tags({"file", "get", "list", "ingest", "source", "filesystem"})

View File

@ -16,6 +16,27 @@
*/
package org.apache.nifi.processors.standard;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.ReadsAttribute;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.behavior.SupportsBatching;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.util.StopWatch;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
@ -34,32 +55,13 @@ import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.ReadsAttribute;
import org.apache.nifi.annotation.behavior.SupportsBatching;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.util.StopWatch;
@SupportsBatching
@InputRequirement(Requirement.INPUT_REQUIRED)
@Tags({"put", "local", "copy", "archive", "files", "filesystem"})
@CapabilityDescription("Writes the contents of a FlowFile to the local file system")
@SeeAlso({FetchFile.class, GetFile.class})
@ReadsAttribute(attribute = "filename", description = "The filename to use when writing the FlowFile to disk.")
@Restricted("Provides operator the ability to write to any file that NiFi has access to.")
public class PutFile extends AbstractProcessor {
public static final String REPLACE_RESOLUTION = "replace";

View File

@ -16,6 +16,38 @@
*/
package org.apache.nifi.processors.standard;
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.behavior.Stateful;
import org.apache.nifi.annotation.behavior.TriggerSerially;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.components.state.StateMap;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.OutputStreamCallback;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.stream.io.ByteArrayOutputStream;
import org.apache.nifi.stream.io.NullOutputStream;
import org.apache.nifi.stream.io.StreamUtils;
import java.io.BufferedOutputStream;
import java.io.File;
import java.io.FileInputStream;
@ -45,37 +77,6 @@ import java.util.zip.CRC32;
import java.util.zip.CheckedInputStream;
import java.util.zip.Checksum;
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.Stateful;
import org.apache.nifi.annotation.behavior.TriggerSerially;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.components.state.StateMap;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.OutputStreamCallback;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.stream.io.ByteArrayOutputStream;
import org.apache.nifi.stream.io.NullOutputStream;
import org.apache.nifi.stream.io.StreamUtils;
// note: it is important that this Processor is not marked as @SupportsBatching because the session commits must complete before persisting state locally; otherwise, data loss may occur
@TriggerSerially
@InputRequirement(Requirement.INPUT_FORBIDDEN)
@ -91,6 +92,7 @@ import org.apache.nifi.stream.io.StreamUtils;
@WritesAttributes({
@WritesAttribute(attribute = "tailfile.original.path", description = "Path of the original file the flow file comes from.")
})
@Restricted("Provides operator the ability to read from any file that NiFi has access to.")
public class TailFile extends AbstractProcessor {
static final String MAP_PREFIX = "file.";