NIFI-1781:

- Including access policies in the breadcrumb's trail.
- Updating toolbox according to group access policies.
- Updating actions in palette based on selection access policies.
NIFI-1554:
- Introducing authorization during two phase commit.
- Introducing snippet authorization according to the encapsulated components and the action performed.
- This closes #461
This commit is contained in:
Matt Gilman 2016-05-23 14:14:22 -04:00
parent d42ea467ca
commit 4dd50c80a4
64 changed files with 3386 additions and 2188 deletions

View File

@ -16,10 +16,10 @@
*/
package org.apache.nifi.controller;
import java.util.Map;
import org.apache.nifi.web.Revision;
import java.util.Map;
/**
* A Snippet represents a segment of the flow
*/
@ -30,14 +30,6 @@ public interface Snippet {
*/
public String getId();
/**
* @return Whether or not this snippet is linked to the data flow. If the Snippet is
* deleted and is linked, then the underlying components will also be
* deleted. If the Snippet is deleted and is NOT linked, only the Snippet is
* removed
*/
public boolean isLinked();
/**
* @return parent group id of the components in this snippet
*/

View File

@ -27,7 +27,6 @@ public class ProcessGroupDTO extends ComponentDTO {
private String name;
private String comments;
private Boolean running;
private Integer runningCount;
private Integer stoppedCount;
@ -131,20 +130,6 @@ public class ProcessGroupDTO extends ComponentDTO {
this.outputPortCount = outputPortCount;
}
/**
* @return Used in requests, indicates whether this process group should be running
*/
@ApiModelProperty(
value = "Used in requests, indicates whether the process group should be running."
)
public Boolean isRunning() {
return running;
}
public void setRunning(Boolean running) {
this.running = running;
}
/**
* @return number of running component in this process group
*/

View File

@ -32,7 +32,6 @@ public class SnippetDTO {
private String id;
private String uri;
private String parentGroupId;
private Boolean linked;
// when specified these are only considered during creation
private Map<String, RevisionDTO> processGroups = new HashMap<>();
@ -86,22 +85,6 @@ public class SnippetDTO {
this.parentGroupId = parentGroupId;
}
/**
* @return whether or not this snippet is linked to the underlying data flow
*/
@ApiModelProperty(
value = "Whether or not the snippet is linked to the underlying data flow. For instance if linked was set to true and the snippet was deleted "
+ "it would also deleted the components in the snippet. If the snippet was not linked, deleting the snippet would only remove the "
+ "snippet and leave the component intact."
)
public Boolean isLinked() {
return linked;
}
public void setLinked(Boolean linked) {
this.linked = linked;
}
/**
* @return the ids of the connections in this snippet. These ids will be populated within each response. They can be specified when creating a snippet. However, once a snippet has been created its
* contents cannot be modified (these ids are ignored during update requests)

View File

@ -28,7 +28,6 @@ public class FlowBreadcrumbDTO {
private String id;
private String name;
private FlowBreadcrumbDTO parentBreadcrumb;
/**
* The id for this group.
@ -61,20 +60,4 @@ public class FlowBreadcrumbDTO {
public void setName(final String name) {
this.name = name;
}
/**
* The parent breadcrumb for this breadcrumb.
*
* @return The parent breadcrumb for this breadcrumb
*/
@ApiModelProperty(
value = "The parent breadcrumb for this breadcrumb."
)
public FlowBreadcrumbDTO getParentBreadcrumb() {
return parentBreadcrumb;
}
public void setParentBreadcrumb(FlowBreadcrumbDTO parentBreadcrumb) {
this.parentBreadcrumb = parentBreadcrumb;
}
}

View File

@ -18,7 +18,6 @@ package org.apache.nifi.web.api.dto.flow;
import com.wordnik.swagger.annotations.ApiModelProperty;
import org.apache.nifi.web.api.entity.ConnectionEntity;
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
import org.apache.nifi.web.api.entity.FunnelEntity;
import org.apache.nifi.web.api.entity.LabelEntity;
import org.apache.nifi.web.api.entity.PortEntity;
@ -44,7 +43,6 @@ public class FlowDTO {
private Set<ConnectionEntity> connections = new LinkedHashSet<>();
private Set<LabelEntity> labels = new LinkedHashSet<>();
private Set<FunnelEntity> funnels = new LinkedHashSet<>();
private Set<ControllerServiceEntity> controllerServices = new LinkedHashSet<>();
/**
* @return connections in this flow
@ -158,17 +156,4 @@ public class FlowDTO {
this.remoteProcessGroups = remoteProcessGroups;
}
/**
* @return the Controller Services in this flow
*/
@ApiModelProperty(
value = "The controller services in this flow."
)
public Set<ControllerServiceEntity> getControllerServices() {
return controllerServices;
}
public void setControllerServices(Set<ControllerServiceEntity> controllerServices) {
this.controllerServices = controllerServices;
}
}

View File

@ -18,6 +18,7 @@ package org.apache.nifi.web.api.dto.flow;
import com.wordnik.swagger.annotations.ApiModelProperty;
import org.apache.nifi.web.api.dto.util.TimeAdapter;
import org.apache.nifi.web.api.entity.FlowBreadcrumbEntity;
import javax.xml.bind.annotation.XmlType;
import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
@ -32,7 +33,7 @@ public class ProcessGroupFlowDTO {
private String id;
private String uri;
private String parentGroupId;
private FlowBreadcrumbDTO breadcrumb;
private FlowBreadcrumbEntity breadcrumb;
private FlowDTO flow;
private Date lastRefreshed;
@ -74,11 +75,11 @@ public class ProcessGroupFlowDTO {
@ApiModelProperty(
value = "The breadcrumb of the process group."
)
public FlowBreadcrumbDTO getBreadcrumb() {
public FlowBreadcrumbEntity getBreadcrumb() {
return breadcrumb;
}
public void setBreadcrumb(FlowBreadcrumbDTO breadcrumb) {
public void setBreadcrumb(FlowBreadcrumbEntity breadcrumb) {
this.breadcrumb = breadcrumb;
}

View File

@ -0,0 +1,99 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.web.api.entity;
import com.wordnik.swagger.annotations.ApiModelProperty;
import org.apache.nifi.web.api.dto.AccessPolicyDTO;
import org.apache.nifi.web.api.dto.flow.FlowBreadcrumbDTO;
import javax.xml.bind.annotation.XmlRootElement;
/**
* A serialized representation of this class can be placed in the entity body of a request or response to or from the API. This particular entity holds a reference to a FlowBreadcrumbDTO.
*/
@XmlRootElement(name = "flowEntity")
public class FlowBreadcrumbEntity extends Entity {
private String id;
private AccessPolicyDTO accessPolicy;
private FlowBreadcrumbDTO breadcrumb;
private FlowBreadcrumbEntity parentBreadcrumb;
/**
* The id for this ancestor ProcessGroup.
*
* @return The id
*/
@ApiModelProperty(
value = "The id of this ancestor ProcessGroup."
)
public String getId() {
return id;
}
public void setId(String id) {
this.id = id;
}
/**
* The access policy for this ancestor ProcessGroup.
*
* @return The access policy
*/
@ApiModelProperty(
value = "The access policy for this ancestor ProcessGroup."
)
public AccessPolicyDTO getAccessPolicy() {
return accessPolicy;
}
public void setAccessPolicy(AccessPolicyDTO accessPolicy) {
this.accessPolicy = accessPolicy;
}
/**
* The FlowBreadcrumbDTO that is being serialized.
*
* @return The FlowBreadcrumbDTO object
*/
@ApiModelProperty(
value = "This breadcrumb."
)
public FlowBreadcrumbDTO getBreadcrumb() {
return breadcrumb;
}
public void setBreadcrumb(FlowBreadcrumbDTO breadcrumb) {
this.breadcrumb = breadcrumb;
}
/**
* The parent breadcrumb for this breadcrumb.
*
* @return The parent breadcrumb for this breadcrumb
*/
@ApiModelProperty(
value = "The parent breadcrumb for this breadcrumb."
)
public FlowBreadcrumbEntity getParentBreadcrumb() {
return parentBreadcrumb;
}
public void setParentBreadcrumb(FlowBreadcrumbEntity parentBreadcrumb) {
this.parentBreadcrumb = parentBreadcrumb;
}
}

View File

@ -16,6 +16,8 @@
*/
package org.apache.nifi.web.api.entity;
import com.wordnik.swagger.annotations.ApiModelProperty;
import org.apache.nifi.web.api.dto.AccessPolicyDTO;
import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO;
import javax.xml.bind.annotation.XmlRootElement;
@ -26,8 +28,25 @@ import javax.xml.bind.annotation.XmlRootElement;
@XmlRootElement(name = "processGroupFlowEntity")
public class ProcessGroupFlowEntity extends Entity {
private AccessPolicyDTO accessPolicy;
private ProcessGroupFlowDTO processGroupFlow;
/**
* The access policy for this component.
*
* @return The access policy
*/
@ApiModelProperty(
value = "The access policy for this process group."
)
public AccessPolicyDTO getAccessPolicy() {
return accessPolicy;
}
public void setAccessPolicy(AccessPolicyDTO accessPolicy) {
this.accessPolicy = accessPolicy;
}
/**
* The ProcessGroupFlowDTO that is being serialized.
*

View File

@ -0,0 +1,77 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.web.api.entity;
import com.wordnik.swagger.annotations.ApiModelProperty;
import org.apache.nifi.web.api.dto.RevisionDTO;
import javax.xml.bind.annotation.XmlRootElement;
import java.util.Map;
/**
* A serialized representation of this class can be placed in the entity body of a request to the API.
*/
@XmlRootElement(name = "scheduleComponentEntity")
public class ScheduleComponentsEntity extends Entity {
private String id;
private String state;
private Map<String, RevisionDTO> components;
/**
* @return The id of the ProcessGroup
*/
@ApiModelProperty(
value = "The id of the ProcessGroup"
)
public String getId() {
return id;
}
public void setId(String id) {
this.id = id;
}
/**
* @return The desired state of the descendant components. Possible states are 'RUNNING' and 'STOPPED'
*/
@ApiModelProperty(
value = "The desired state of the descendant components",
allowableValues = "RUNNING, STOPPED"
)
public String getState() {
return state;
}
public void setState(String state) {
this.state = state;
}
/**
* @return The components to schedule. If not specified, all authorized descendant components will be used.
*/
@ApiModelProperty(
value = "Optional components to schedule. If not specified, all authorized descendant components will be used."
)
public Map<String, RevisionDTO> getComponents() {
return components;
}
public void setComponents(Map<String, RevisionDTO> components) {
this.components = components;
}
}

View File

@ -16,14 +16,16 @@
*/
package org.apache.nifi.web.api.entity;
import javax.xml.bind.annotation.XmlRootElement;
import com.wordnik.swagger.annotations.ApiModelProperty;
import org.apache.nifi.web.api.dto.SnippetDTO;
import javax.xml.bind.annotation.XmlRootElement;
/**
* A serialized representation of this class can be placed in the entity body of a request or response to or from the API. This particular entity holds a reference to a SnippetDTO.
*/
@XmlRootElement(name = "snippetEntity")
public class SnippetEntity extends ComponentEntity {
public class SnippetEntity extends Entity {
private SnippetDTO snippet;
@ -32,6 +34,7 @@ public class SnippetEntity extends ComponentEntity {
*
* @return The SnippetDTO object
*/
@ApiModelProperty("The snippet.")
public SnippetDTO getSnippet() {
return snippet;
}

View File

@ -19,10 +19,11 @@ package org.apache.nifi.controller;
import java.util.Collection;
import java.util.Map;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationResult;
public interface ConfiguredComponent {
public interface ConfiguredComponent extends Authorizable {
public String getIdentifier();

View File

@ -16,16 +16,15 @@
*/
package org.apache.nifi.controller;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.reporting.ReportingContext;
import org.apache.nifi.reporting.ReportingTask;
import org.apache.nifi.scheduling.SchedulingStrategy;
public interface ReportingTaskNode extends ConfiguredComponent, Authorizable {
import java.util.Set;
import java.util.concurrent.TimeUnit;
public interface ReportingTaskNode extends ConfiguredComponent {
void setSchedulingStrategy(SchedulingStrategy schedulingStrategy);

View File

@ -16,15 +16,14 @@
*/
package org.apache.nifi.controller.service;
import java.util.Set;
import java.util.concurrent.ScheduledExecutorService;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.controller.ConfiguredComponent;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.groups.ProcessGroup;
public interface ControllerServiceNode extends ConfiguredComponent, Authorizable {
import java.util.Set;
import java.util.concurrent.ScheduledExecutorService;
public interface ControllerServiceNode extends ConfiguredComponent {
/**
* @return the Process Group that this Controller Service belongs to, or <code>null</code> if the Controller Service

View File

@ -16,10 +16,6 @@
*/
package org.apache.nifi.groups;
import java.util.Collection;
import java.util.List;
import java.util.Set;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.Connection;
@ -27,6 +23,7 @@ import org.apache.nifi.connectable.Funnel;
import org.apache.nifi.connectable.Port;
import org.apache.nifi.connectable.Position;
import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.Snippet;
import org.apache.nifi.controller.Template;
import org.apache.nifi.controller.label.Label;
@ -34,6 +31,11 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.Processor;
import java.util.Collection;
import java.util.List;
import java.util.Set;
import java.util.function.Predicate;
/**
* <p>
* ProcessGroup objects are containers for processing entities, such as
@ -45,6 +47,26 @@ import org.apache.nifi.processor.Processor;
*/
public interface ProcessGroup extends Authorizable {
/**
* Predicate for filtering schedulable Processors.
*/
Predicate<ProcessorNode> SCHEDULABLE_PROCESSORS = node -> !node.isRunning() && node.getScheduledState() != ScheduledState.DISABLED;
/**
* Predicate for filtering unschedulable Processors.
*/
Predicate<ProcessorNode> UNSCHEDULABLE_PROCESSORS = node -> node.isRunning();
/**
* Predicate for filtering schedulable Ports
*/
Predicate<Port> SCHEDULABLE_PORTS = port -> port.getScheduledState() != ScheduledState.DISABLED;
/**
* Predicate for filtering schedulable Ports
*/
Predicate<Port> UNSCHEDULABLE_PORTS = port -> port.getScheduledState() == ScheduledState.RUNNING;
/**
* @return a reference to this ProcessGroup's parent. This will be
* <tt>null</tt> if and only if this is the root group.
@ -743,8 +765,12 @@ public interface ProcessGroup extends Authorizable {
*/
void verifyCanDelete(boolean ignorePortConnections);
void verifyCanStart(Connectable connectable);
void verifyCanStart();
void verifyCanStop(Connectable connectable);
void verifyCanStop();
/**

View File

@ -33,7 +33,6 @@ public class StandardSnippet implements Snippet {
private String id;
private String parentGroupId;
private Boolean linked;
private Map<String, Revision> processGroups = new HashMap<>();
private Map<String, Revision> remoteProcessGroups = new HashMap<>();
@ -53,19 +52,6 @@ public class StandardSnippet implements Snippet {
this.id = id;
}
@Override
public boolean isLinked() {
if (linked == null) {
return false;
} else {
return linked;
}
}
public void setLinked(Boolean linked) {
this.linked = linked;
}
@Override
public String getParentGroupId() {
return parentGroupId;

View File

@ -283,38 +283,21 @@ public final class StandardProcessGroup implements ProcessGroup {
public void startProcessing() {
readLock.lock();
try {
for (final ProcessorNode node : processors.values()) {
findAllProcessors().stream().filter(SCHEDULABLE_PROCESSORS).forEach(node -> {
try {
if (!node.isRunning() && node.getScheduledState() != ScheduledState.DISABLED) {
startProcessor(node);
}
node.getProcessGroup().startProcessor(node);
} catch (final Throwable t) {
LOG.error("Unable to start {} due to {}", new Object[]{node, t});
}
}
});
for (final Port inputPort : getInputPorts()) {
if (inputPort.getScheduledState() != ScheduledState.DISABLED) {
startInputPort(inputPort);
}
}
findAllInputPorts().stream().filter(SCHEDULABLE_PORTS).forEach(port -> {
port.getProcessGroup().startInputPort(port);
});
for (final Port outputPort : getOutputPorts()) {
if (outputPort.getScheduledState() != ScheduledState.DISABLED) {
startOutputPort(outputPort);
}
}
for (final Funnel funnel : getFunnels()) {
if (funnel.getScheduledState() != ScheduledState.DISABLED) {
startFunnel(funnel);
}
}
// Recursively start child groups.
for (final ProcessGroup group : processGroups.values()) {
group.startProcessing();
}
findAllOutputPorts().stream().filter(SCHEDULABLE_PORTS).forEach(port -> {
port.getProcessGroup().startOutputPort(port);
});
} finally {
readLock.unlock();
}
@ -324,32 +307,21 @@ public final class StandardProcessGroup implements ProcessGroup {
public void stopProcessing() {
readLock.lock();
try {
for (final ProcessorNode node : processors.values()) {
findAllProcessors().stream().filter(UNSCHEDULABLE_PROCESSORS).forEach(node -> {
try {
if (node.isRunning()) {
stopProcessor(node);
}
node.getProcessGroup().stopProcessor(node);
} catch (final Throwable t) {
LOG.error("Unable to stop {} due to {}", new Object[]{node, t});
}
}
});
for (final Port inputPort : getInputPorts()) {
if (inputPort.getScheduledState() == ScheduledState.RUNNING) {
stopInputPort(inputPort);
}
}
findAllInputPorts().stream().filter(UNSCHEDULABLE_PORTS).forEach(port -> {
port.getProcessGroup().stopInputPort(port);
});
for (final Port outputPort : getOutputPorts()) {
if (outputPort.getScheduledState() == ScheduledState.RUNNING) {
stopOutputPort(outputPort);
}
}
// Recursively stop child groups.
for (final ProcessGroup group : processGroups.values()) {
group.stopProcessing();
}
findAllOutputPorts().stream().filter(UNSCHEDULABLE_PORTS).forEach(port -> {
port.getProcessGroup().stopOutputPort(port);
});
} finally {
readLock.unlock();
}
@ -2291,22 +2263,36 @@ public final class StandardProcessGroup implements ProcessGroup {
}
}
@Override
public void verifyCanStop(Connectable connectable) {
}
@Override
public void verifyCanStop() {
}
@Override
public void verifyCanStart(Connectable connectable) {
readLock.lock();
try {
if (connectable.getScheduledState() == ScheduledState.STOPPED) {
if (scheduler.getActiveThreadCount(connectable) > 0) {
throw new IllegalStateException("Cannot start " + connectable + " because it is currently stopping");
}
connectable.verifyCanStart();
}
} finally {
readLock.unlock();
}
}
@Override
public void verifyCanStart() {
readLock.lock();
try {
for (final Connectable connectable : findAllConnectables(this, false)) {
if (connectable.getScheduledState() == ScheduledState.STOPPED) {
if (scheduler.getActiveThreadCount(connectable) > 0) {
throw new IllegalStateException("Cannot start " + connectable + " because it is currently stopping");
}
connectable.verifyCanStart();
}
verifyCanStart(connectable);
}
} finally {
readLock.unlock();

View File

@ -557,4 +557,11 @@ public class MockProcessGroup implements ProcessGroup {
return null;
}
@Override
public void verifyCanStart(Connectable connectable) {
}
@Override
public void verifyCanStop(Connectable connectable) {
}
}

View File

@ -16,9 +16,6 @@
*/
package org.apache.nifi.audit;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Date;
import org.apache.nifi.action.Action;
import org.apache.nifi.action.Component;
import org.apache.nifi.action.FlowChangeAction;
@ -26,9 +23,10 @@ import org.apache.nifi.action.Operation;
import org.apache.nifi.action.details.ActionDetails;
import org.apache.nifi.action.details.FlowChangeConfigureDetails;
import org.apache.nifi.action.details.FlowChangeMoveDetails;
import org.apache.nifi.authorization.user.NiFiUserUtils;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.authorization.user.NiFiUser;
import org.apache.nifi.authorization.user.NiFiUserUtils;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.web.api.dto.ProcessGroupDTO;
import org.apache.nifi.web.dao.ProcessGroupDAO;
import org.aspectj.lang.ProceedingJoinPoint;
@ -37,6 +35,10 @@ import org.aspectj.lang.annotation.Aspect;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Date;
/**
* Audits process group creation/removal and configuration changes.
*/
@ -153,28 +155,6 @@ public class ProcessGroupAuditor extends NiFiAuditor {
}
}
// if the user was starting/stopping this process group
if (processGroupDTO.isRunning() != null) {
// create a process group action
FlowChangeAction processGroupAction = new FlowChangeAction();
processGroupAction.setUserIdentity(user.getIdentity());
processGroupAction.setUserName(user.getUserName());
processGroupAction.setSourceId(processGroup.getIdentifier());
processGroupAction.setSourceName(processGroup.getName());
processGroupAction.setSourceType(Component.ProcessGroup);
processGroupAction.setTimestamp(new Date());
// determine the running state
if (processGroupDTO.isRunning()) {
processGroupAction.setOperation(Operation.Start);
} else {
processGroupAction.setOperation(Operation.Stop);
}
// add this action
actions.add(processGroupAction);
}
// save actions if necessary
if (!actions.isEmpty()) {
saveActions(actions, logger);
@ -184,6 +164,47 @@ public class ProcessGroupAuditor extends NiFiAuditor {
return updatedProcessGroup;
}
/**
* Audits the update of process group configuration.
*
* @param proceedingJoinPoint join point
* @param groupId group id
* @param state scheduled state
* @throws Throwable ex
*/
@Around("within(org.apache.nifi.web.dao.ProcessGroupDAO+) && "
+ "execution(void scheduleComponents(java.lang.String, org.apache.nifi.controller.ScheduledState, java.util.Set)) && "
+ "args(groupId, state)")
public void scheduleComponentsAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ScheduledState state) throws Throwable {
ProcessGroupDAO processGroupDAO = getProcessGroupDAO();
ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
// perform the action
proceedingJoinPoint.proceed();
// get the current user
NiFiUser user = NiFiUserUtils.getNiFiUser();
// if the user was starting/stopping this process group
FlowChangeAction action = new FlowChangeAction();
action.setUserIdentity(user.getIdentity());
action.setUserName(user.getUserName());
action.setSourceId(processGroup.getIdentifier());
action.setSourceName(processGroup.getName());
action.setSourceType(Component.ProcessGroup);
action.setTimestamp(new Date());
// determine the running state
if (ScheduledState.RUNNING.equals(state)) {
action.setOperation(Operation.Start);
} else {
action.setOperation(Operation.Stop);
}
// add this action
saveAction(action, logger);
}
/**
* Audits the removal of a process group via deleteProcessGroup().
*

View File

@ -253,7 +253,7 @@ public class SnippetAuditor extends NiFiAuditor {
* @throws Throwable ex
*/
@Around("within(org.apache.nifi.web.dao.SnippetDAO+) && "
+ "execution(org.apache.nifi.controller.Snippet updateSnippet(org.apache.nifi.web.api.dto.SnippetDTO)) && "
+ "execution(org.apache.nifi.controller.Snippet updateSnippetComponents(org.apache.nifi.web.api.dto.SnippetDTO)) && "
+ "args(snippetDTO) && "
+ "target(snippetDAO)")
public Snippet updateSnippetAdvice(ProceedingJoinPoint proceedingJoinPoint, SnippetDTO snippetDTO, SnippetDAO snippetDAO) throws Throwable {
@ -266,7 +266,7 @@ public class SnippetAuditor extends NiFiAuditor {
// if this snippet is linked and its parent group id has changed
final String groupId = snippetDTO.getParentGroupId();
if (snippet.isLinked() && !previousGroupId.equals(groupId)) {
if (!previousGroupId.equals(groupId)) {
// create move audit records for all items in this snippet
final Collection<Action> actions = new ArrayList<>();
@ -346,114 +346,109 @@ public class SnippetAuditor extends NiFiAuditor {
* @throws Throwable ex
*/
@Around("within(org.apache.nifi.web.dao.SnippetDAO+) && "
+ "execution(void deleteSnippet(java.lang.String)) && "
+ "execution(void deleteSnippetComponents(java.lang.String)) && "
+ "args(snippetId) && "
+ "target(snippetDAO)")
public void removeSnippetAdvice(ProceedingJoinPoint proceedingJoinPoint, String snippetId, SnippetDAO snippetDAO) throws Throwable {
// get the snippet before removing it
final Snippet snippet = snippetDAO.getSnippet(snippetId);
if (snippet.isLinked()) {
// locate all the components being removed
final Set<Funnel> funnels = new HashSet<>();
for (String id : snippet.getFunnels().keySet()) {
funnels.add(funnelDAO.getFunnel(id));
// locate all the components being removed
final Set<Funnel> funnels = new HashSet<>();
for (String id : snippet.getFunnels().keySet()) {
funnels.add(funnelDAO.getFunnel(id));
}
final Set<Port> inputPorts = new HashSet<>();
for (String id : snippet.getInputPorts().keySet()) {
inputPorts.add(inputPortDAO.getPort(id));
}
final Set<Port> outputPorts = new HashSet<>();
for (String id : snippet.getOutputPorts().keySet()) {
outputPorts.add(outputPortDAO.getPort(id));
}
final Set<RemoteProcessGroup> remoteProcessGroups = new HashSet<>();
for (String id : snippet.getRemoteProcessGroups().keySet()) {
remoteProcessGroups.add(remoteProcessGroupDAO.getRemoteProcessGroup(id));
}
final Set<ProcessGroup> processGroups = new HashSet<>();
final ProcessGroupDAO processGroupDAO = getProcessGroupDAO();
for (String id : snippet.getProcessGroups().keySet()) {
processGroups.add(processGroupDAO.getProcessGroup(id));
}
final Set<ProcessorNode> processors = new HashSet<>();
for (String id : snippet.getProcessors().keySet()) {
processors.add(processorDAO.getProcessor(id));
}
final Set<Connection> connections = new HashSet<>();
for (String id : snippet.getConnections().keySet()) {
connections.add(connectionDAO.getConnection(id));
}
// remove the snippet and components
proceedingJoinPoint.proceed();
final Collection<Action> actions = new ArrayList<>();
// audit funnel removal
for (Funnel funnel : funnels) {
final Action action = funnelAuditor.generateAuditRecord(funnel, Operation.Remove);
if (action != null) {
actions.add(action);
}
}
final Set<Port> inputPorts = new HashSet<>();
for (String id : snippet.getInputPorts().keySet()) {
inputPorts.add(inputPortDAO.getPort(id));
for (Port inputPort : inputPorts) {
final Action action = portAuditor.generateAuditRecord(inputPort, Operation.Remove);
if (action != null) {
actions.add(action);
}
}
final Set<Port> outputPorts = new HashSet<>();
for (String id : snippet.getOutputPorts().keySet()) {
outputPorts.add(outputPortDAO.getPort(id));
for (Port outputPort : outputPorts) {
final Action action = portAuditor.generateAuditRecord(outputPort, Operation.Remove);
if (action != null) {
actions.add(action);
}
}
final Set<RemoteProcessGroup> remoteProcessGroups = new HashSet<>();
for (String id : snippet.getRemoteProcessGroups().keySet()) {
remoteProcessGroups.add(remoteProcessGroupDAO.getRemoteProcessGroup(id));
for (RemoteProcessGroup remoteProcessGroup : remoteProcessGroups) {
final Action action = remoteProcessGroupAuditor.generateAuditRecord(remoteProcessGroup, Operation.Remove);
if (action != null) {
actions.add(action);
}
}
final Set<ProcessGroup> processGroups = new HashSet<>();
final ProcessGroupDAO processGroupDAO = getProcessGroupDAO();
for (String id : snippet.getProcessGroups().keySet()) {
processGroups.add(processGroupDAO.getProcessGroup(id));
for (ProcessGroup processGroup : processGroups) {
final Action action = processGroupAuditor.generateAuditRecord(processGroup, Operation.Remove);
if (action != null) {
actions.add(action);
}
}
final Set<ProcessorNode> processors = new HashSet<>();
for (String id : snippet.getProcessors().keySet()) {
processors.add(processorDAO.getProcessor(id));
for (ProcessorNode processor : processors) {
final Action action = processorAuditor.generateAuditRecord(processor, Operation.Remove);
if (action != null) {
actions.add(action);
}
}
final Set<Connection> connections = new HashSet<>();
for (String id : snippet.getConnections().keySet()) {
connections.add(connectionDAO.getConnection(id));
for (Connection connection : connections) {
final ConnectDetails connectDetails = relationshipAuditor.createConnectDetails(connection, connection.getRelationships());
final Action action = relationshipAuditor.generateAuditRecordForConnection(connection, Operation.Disconnect, connectDetails);
if (action != null) {
actions.add(action);
}
}
// remove the snippet and components
proceedingJoinPoint.proceed();
final Collection<Action> actions = new ArrayList<>();
// audit funnel removal
for (Funnel funnel : funnels) {
final Action action = funnelAuditor.generateAuditRecord(funnel, Operation.Remove);
if (action != null) {
actions.add(action);
}
}
for (Port inputPort : inputPorts) {
final Action action = portAuditor.generateAuditRecord(inputPort, Operation.Remove);
if (action != null) {
actions.add(action);
}
}
for (Port outputPort : outputPorts) {
final Action action = portAuditor.generateAuditRecord(outputPort, Operation.Remove);
if (action != null) {
actions.add(action);
}
}
for (RemoteProcessGroup remoteProcessGroup : remoteProcessGroups) {
final Action action = remoteProcessGroupAuditor.generateAuditRecord(remoteProcessGroup, Operation.Remove);
if (action != null) {
actions.add(action);
}
}
for (ProcessGroup processGroup : processGroups) {
final Action action = processGroupAuditor.generateAuditRecord(processGroup, Operation.Remove);
if (action != null) {
actions.add(action);
}
}
for (ProcessorNode processor : processors) {
final Action action = processorAuditor.generateAuditRecord(processor, Operation.Remove);
if (action != null) {
actions.add(action);
}
}
for (Connection connection : connections) {
final ConnectDetails connectDetails = relationshipAuditor.createConnectDetails(connection, connection.getRelationships());
final Action action = relationshipAuditor.generateAuditRecordForConnection(connection, Operation.Disconnect, connectDetails);
if (action != null) {
actions.add(action);
}
}
// save the actions
if (CollectionUtils.isNotEmpty(actions)) {
saveActions(actions, logger);
}
} else {
// remove the snippet but not the components since this snippet isn't linked
proceedingJoinPoint.proceed();
// save the actions
if (CollectionUtils.isNotEmpty(actions)) {
saveActions(actions, logger);
}
}

View File

@ -0,0 +1,154 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.web;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.controller.Snippet;
public interface AuthorizableLookup {
/**
* Get the authorizable Processor.
*
* @param id processor id
* @return authorizable
*/
Authorizable getProcessor(String id);
/**
* Get the authorizable InputPort.
*
* @param id input port id
* @return authorizable
*/
Authorizable getInputPort(String id);
/**
* Get the authorizable OutputPort.
*
* @param id output port id
* @return authorizable
*/
Authorizable getOutputPort(String id);
/**
* Get the authorizable Connection.
*
* @param id connection id
* @return authorizable
*/
Authorizable getConnection(String id);
/**
* Get the authorizable ProcessGroup.
*
* @param id process group id
* @return authorizable
*/
Authorizable getProcessGroup(String id);
/**
* Get the authorizable RemoteProcessGroup.
*
* @param id remote process group id
* @return authorizable
*/
Authorizable getRemoteProcessGroup(String id);
/**
* Get the authorizable RemoteProcessGroup input port.
*
* @param remoteProcessGroupId remote process group id
* @param id input port id
* @return authorizable
*/
Authorizable getRemoteProcessGroupInputPort(String remoteProcessGroupId, String id);
/**
* Get the authorizable RemoteProcessGroup output port.
*
* @param remoteProcessGroupId remote process group id
* @param id output port id
* @return authorizable
*/
Authorizable getRemoteProcessGroupOutputPort(String remoteProcessGroupId, String id);
/**
* Get the authorizable Label.
*
* @param id label id
* @return authorizable
*/
Authorizable getLabel(String id);
/**
* Get the authorizable Funnel.
*
* @param id funnel id
* @return authorizable
*/
Authorizable getFunnel(String id);
/**
* Get the authorizable ControllerService.
*
* @param id controller service id
* @return authorizable
*/
Authorizable getControllerService(String id);
/**
* Get the authorizable referencing component.
*
* @param controllerSeriveId controller service id
* @param id component id
* @return authorizable
*/
Authorizable getControllerServiceReferencingComponent(String controllerSeriveId, String id);
/**
* Get the authorizable ReportingTask.
*
* @param id reporting task id
* @return authorizable
*/
Authorizable getReportingTask(String id);
/**
* Get the authorizable Template.
*
* @param id template id
* @return authorizable
*/
Authorizable getTemplate(String id);
/**
* Get the authorizable connectable.
*
* @param id connectable id
* @return authorizable
*/
Authorizable getConnectable(String id);
/**
* Get the snippet of authorizable's.
*
* @param id snippet id
* @return snippet of authorizable's
*/
Snippet getSnippet(String id);
}

View File

@ -0,0 +1,21 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.web;
public interface AuthorizeAccess {
void authorize(AuthorizableLookup lookup);
}

View File

@ -16,9 +16,11 @@
*/
package org.apache.nifi.web;
import org.apache.nifi.authorization.user.NiFiUser;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.repository.claim.ContentDirection;
import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.web.api.dto.BulletinBoardDTO;
import org.apache.nifi.web.api.dto.BulletinQueryDTO;
import org.apache.nifi.web.api.dto.ClusterDTO;
@ -51,7 +53,6 @@ import org.apache.nifi.web.api.dto.TemplateDTO;
import org.apache.nifi.web.api.dto.action.ActionDTO;
import org.apache.nifi.web.api.dto.action.HistoryDTO;
import org.apache.nifi.web.api.dto.action.HistoryQueryDTO;
import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO;
import org.apache.nifi.web.api.dto.provenance.ProvenanceDTO;
import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO;
import org.apache.nifi.web.api.dto.provenance.ProvenanceOptionsDTO;
@ -72,10 +73,12 @@ import org.apache.nifi.web.api.entity.FunnelEntity;
import org.apache.nifi.web.api.entity.LabelEntity;
import org.apache.nifi.web.api.entity.PortEntity;
import org.apache.nifi.web.api.entity.ProcessGroupEntity;
import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity;
import org.apache.nifi.web.api.entity.ProcessorEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupPortEntity;
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
import org.apache.nifi.web.api.entity.ScheduleComponentsEntity;
import org.apache.nifi.web.api.entity.SnippetEntity;
import java.util.Date;
@ -83,6 +86,7 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.function.Function;
/**
* Defines the NiFiServiceFacade interface.
@ -92,7 +96,64 @@ public interface NiFiServiceFacade {
// ----------------------------------------
// Synchronization methods
// ----------------------------------------
void claimRevision(Revision revision) throws InvalidRevisionException;
/**
* Authorizes access to the service facade.
*
* @param authorizeAccess authorize access callback
*/
void authorizeAccess(AuthorizeAccess authorizeAccess);
/**
* Claims the specified revision for the specified user.
*
* @param revision revision
* @param user user
* @throws InvalidRevisionException invalid revision
*/
void claimRevision(Revision revision, NiFiUser user) throws InvalidRevisionException;
/**
* Claims the specified revisions for the specified user.
*
* @param revisions revisions
* @param user user
* @throws InvalidRevisionException invalid revision
*/
void claimRevisions(Set<Revision> revisions, NiFiUser user) throws InvalidRevisionException;
/**
* Cancels the specified revision. Cancellation is only supported based on the current thread.
*
* @param revision revision
* @throws InvalidRevisionException invalid revision
*/
void cancelRevision(Revision revision) throws InvalidRevisionException;
/**
* Cancels the specified revisions. Cancellation is only supported based on the current thread.
*
* @param revisions revision
* @throws InvalidRevisionException invalid revision
*/
void cancelRevisions(Set<Revision> revisions) throws InvalidRevisionException;
/**
* Gets the current revisions for the components based on the specified function.
*
* @param groupId group
* @param getComponents callback
* @return component revisions
*/
Set<Revision> getRevisionsFromGroup(String groupId, Function<ProcessGroup, Set<String>> getComponents);
/**
* Gets the revisions from the specified snippet.
*
* @param snippetId snippet
* @return component revisions from the snippet
*/
Set<Revision> getRevisionsFromSnippet(String snippetId);
// ----------------------------------------
// Controller methods
@ -754,7 +815,7 @@ public interface NiFiServiceFacade {
* @param recurse recurse
* @return the flow
*/
ConfigurationSnapshot<ProcessGroupFlowDTO> getProcessGroupFlow(String groupId, boolean recurse);
ProcessGroupFlowEntity getProcessGroupFlow(String groupId, boolean recurse);
// ----------------------------------------
// ProcessGroup methods
@ -785,11 +846,23 @@ public interface NiFiServiceFacade {
Set<ProcessGroupEntity> getProcessGroups(String parentGroupId);
/**
* Verifies the specified process group can be updated.
* Verifies the contents of the specified process group can be scheduled or unscheduled.
*
* @param processGroupDTO The ProcessGroupDTO
* @param processGroupId The ProcessGroup id
* @param componentIds the components
* @param state scheduled state
*/
void verifyUpdateProcessGroup(ProcessGroupDTO processGroupDTO);
void verifyScheduleComponents(String processGroupId, ScheduledState state, Set<String> componentIds);
/**
* Schedules all applicable components under the specified ProcessGroup.
*
* @param processGroupId The ProcessGroup id
* @param state schedule state
* @param componentRevisions components and their revision
* @return snapshot
*/
ScheduleComponentsEntity scheduleComponents(String processGroupId, ScheduledState state, Map<String, Revision> componentRevisions);
/**
* Updates the specified process group.
@ -1319,45 +1392,38 @@ public interface NiFiServiceFacade {
*/
SnippetEntity createSnippet(SnippetDTO snippet);
/**
* Gets the specified snippet.
*
* @param snippetId id
* @return snippet
*/
SnippetEntity getSnippet(String snippetId);
/**
* Determines if this snippet can be updated.
*
* @param snippetDto snippet
*/
void verifyUpdateSnippet(SnippetDTO snippetDto);
void verifyUpdateSnippet(SnippetDTO snippetDto, Set<String> affectedComponentIds);
/**
* If group id is specified, moves the specified snippet to the specified group.
*
* @param revision revision
* @param revisions revisions
* @param snippetDto snippet
* @return snapshot
*/
UpdateResult<SnippetEntity> updateSnippet(Revision revision, SnippetDTO snippetDto);
SnippetEntity updateSnippet(Set<Revision> revisions, SnippetDTO snippetDto);
/**
* Determines if this snippet can be removed.
*
* @param id id
* @param affectedComponentIds affected components
*/
void verifyDeleteSnippet(String id);
void verifyDeleteSnippet(String id, Set<String> affectedComponentIds);
/**
* Removes the specified snippet.
*
* @param revision revision
* @param revisions revisions
* @param snippetId snippet
* @return snapshot
*/
SnippetEntity deleteSnippet(Revision revision, String snippetId);
SnippetEntity deleteSnippet(Set<Revision> revisions, String snippetId);
// ----------------------------------------
// Cluster methods

View File

@ -16,30 +16,6 @@
*/
package org.apache.nifi.web;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.ListIterator;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.TimeZone;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import javax.ws.rs.WebApplicationException;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.action.Action;
import org.apache.nifi.action.Component;
@ -64,7 +40,6 @@ import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.components.state.StateMap;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.Connection;
import org.apache.nifi.connectable.Funnel;
import org.apache.nifi.connectable.Port;
@ -139,7 +114,6 @@ import org.apache.nifi.web.api.dto.action.ActionDTO;
import org.apache.nifi.web.api.dto.action.HistoryDTO;
import org.apache.nifi.web.api.dto.action.HistoryQueryDTO;
import org.apache.nifi.web.api.dto.flow.FlowDTO;
import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO;
import org.apache.nifi.web.api.dto.provenance.ProvenanceDTO;
import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO;
import org.apache.nifi.web.api.dto.provenance.ProvenanceOptionsDTO;
@ -161,10 +135,12 @@ import org.apache.nifi.web.api.entity.FunnelEntity;
import org.apache.nifi.web.api.entity.LabelEntity;
import org.apache.nifi.web.api.entity.PortEntity;
import org.apache.nifi.web.api.entity.ProcessGroupEntity;
import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity;
import org.apache.nifi.web.api.entity.ProcessorEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupPortEntity;
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
import org.apache.nifi.web.api.entity.ScheduleComponentsEntity;
import org.apache.nifi.web.api.entity.SnippetEntity;
import org.apache.nifi.web.controller.ControllerFacade;
import org.apache.nifi.web.dao.ConnectionDAO;
@ -191,6 +167,29 @@ import org.apache.nifi.web.util.SnippetUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.ws.rs.WebApplicationException;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.ListIterator;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.TimeZone;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import java.util.function.Supplier;
import java.util.stream.Collectors;
/**
* Implementation of NiFiServiceFacade that performs revision checking.
*/
@ -232,12 +231,154 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
private Authorizer authorizer;
private final AuthorizableLookup authorizableLookup = new AuthorizableLookup() {
@Override
public Authorizable getProcessor(final String id) {
return processorDAO.getProcessor(id);
}
@Override
public Authorizable getInputPort(final String id) {
return inputPortDAO.getPort(id);
}
@Override
public Authorizable getOutputPort(final String id) {
return outputPortDAO.getPort(id);
}
@Override
public Authorizable getConnection(final String id) {
return connectionDAO.getConnection(id);
}
@Override
public Authorizable getProcessGroup(final String id) {
return processGroupDAO.getProcessGroup(id);
}
@Override
public Authorizable getRemoteProcessGroup(final String id) {
return remoteProcessGroupDAO.getRemoteProcessGroup(id);
}
@Override
public Authorizable getRemoteProcessGroupInputPort(final String remoteProcessGroupId, final String id) {
final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.getRemoteProcessGroup(remoteProcessGroupId);
return remoteProcessGroup.getInputPort(id);
}
@Override
public Authorizable getRemoteProcessGroupOutputPort(final String remoteProcessGroupId, final String id) {
final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.getRemoteProcessGroup(remoteProcessGroupId);
return remoteProcessGroup.getOutputPort(id);
}
@Override
public Authorizable getLabel(final String id) {
return labelDAO.getLabel(id);
}
@Override
public Authorizable getFunnel(final String id) {
return funnelDAO.getFunnel(id);
}
@Override
public Authorizable getControllerService(final String id) {
return controllerServiceDAO.getControllerService(id);
}
@Override
public Authorizable getControllerServiceReferencingComponent(String controllerSeriveId, String id) {
final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(controllerSeriveId);
final ControllerServiceReference referencingComponents = controllerService.getReferences();
ConfiguredComponent reference = null;
for (final ConfiguredComponent component : referencingComponents.getReferencingComponents()) {
if (component.getIdentifier().equals(id)) {
reference = component;
break;
}
}
if (reference == null) {
throw new ResourceNotFoundException("Unable to find referencing component with id " + id);
}
return reference;
}
@Override
public Authorizable getReportingTask(final String id) {
return reportingTaskDAO.getReportingTask(id);
}
@Override
public Snippet getSnippet(final String id) {
return snippetDAO.getSnippet(id);
}
@Override
public Authorizable getTemplate(final String id) {
return templateDAO.getTemplate(id);
}
@Override
public Authorizable getConnectable(String id) {
final ProcessGroup group = processGroupDAO.getProcessGroup(controllerFacade.getRootGroupId());
return group.findConnectable(id);
}
};
// -----------------------------------------
// Synchronization methods
// -----------------------------------------
public void authorizeAccess(AuthorizeAccess authorizeAccess) {
authorizeAccess.authorize(authorizableLookup);
}
@Override
public void claimRevision(Revision revision) {
revisionManager.requestClaim(revision, NiFiUserUtils.getNiFiUser());
public void claimRevision(Revision revision, NiFiUser user) {
revisionManager.requestClaim(revision, user);
}
@Override
public void claimRevisions(Set<Revision> revisions, NiFiUser user) {
revisionManager.requestClaim(revisions, user);
}
@Override
public void cancelRevision(Revision revision) {
revisionManager.cancelClaim(revision);
}
@Override
public void cancelRevisions(Set<Revision> revisions) {
revisionManager.cancelClaims(revisions);
}
@Override
public Set<Revision> getRevisionsFromGroup(String groupId, Function<ProcessGroup, Set<String>> getComponents) {
final ProcessGroup group = processGroupDAO.getProcessGroup(groupId);
final Set<String> componentIds = revisionManager.get(group.getIdentifier(), rev -> getComponents.apply(group));
return componentIds.stream().map(id -> revisionManager.getRevision(id)).collect(Collectors.toSet());
}
@Override
public Set<Revision> getRevisionsFromSnippet(String snippetId) {
final Snippet snippet = snippetDAO.getSnippet(snippetId);
final Set<String> componentIds = new HashSet<>();
componentIds.addAll(snippet.getProcessors().keySet());
componentIds.addAll(snippet.getFunnels().keySet());
componentIds.addAll(snippet.getLabels().keySet());
componentIds.addAll(snippet.getConnections().keySet());
componentIds.addAll(snippet.getInputPorts().keySet());
componentIds.addAll(snippet.getOutputPorts().keySet());
componentIds.addAll(snippet.getProcessGroups().keySet());
componentIds.addAll(snippet.getRemoteProcessGroups().keySet());
return componentIds.stream().map(id -> revisionManager.getRevision(id)).collect(Collectors.toSet());
}
// -----------------------------------------
@ -338,15 +479,11 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
}
@Override
public void verifyUpdateProcessGroup(ProcessGroupDTO processGroupDTO) {
public void verifyScheduleComponents(String groupId, ScheduledState state, Set<String> componentIds) {
try {
// if group does not exist, then the update request is likely creating it
// so we don't verify since it will fail
if (processGroupDAO.hasProcessGroup(processGroupDTO.getId())) {
processGroupDAO.verifyUpdate(processGroupDTO);
}
processGroupDAO.verifyScheduleComponents(groupId, state, componentIds);
} catch (final Exception e) {
revisionManager.cancelClaim(processGroupDTO.getId());
componentIds.forEach(id -> revisionManager.cancelClaim(id));
throw e;
}
}
@ -531,7 +668,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
* @param <D> the DTO Type of the updated component
* @param <C> the Component Type of the updated component
*
* @return A ConfigurationSnapshot that represents the new configuration
* @return A RevisionUpdate that represents the new configuration
*/
private <D, C> RevisionUpdate<D> updateComponent(final Revision revision, final Authorizable authorizable, final Supplier<C> daoUpdate, final Function<C, D> dtoCreation) {
final NiFiUser user = NiFiUserUtils.getNiFiUser();
@ -540,12 +677,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
final RevisionUpdate<D> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(revision), user, new UpdateRevisionTask<D>() {
@Override
public RevisionUpdate<D> update() {
// ensure write access to the flow
authorizable.authorize(authorizer, RequestAction.WRITE);
// also ensure read access to the flow as the component must be read in order to generate a response
authorizable.authorize(authorizer, RequestAction.READ);
// get the updated component
final C component = daoUpdate.get();
@ -568,164 +699,50 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
@Override
public void verifyUpdateSnippet(SnippetDTO snippetDto) {
public void verifyUpdateSnippet(SnippetDTO snippetDto, final Set<String> affectedComponentIds) {
try {
// if snippet does not exist, then the update request is likely creating it
// so we don't verify since it will fail
if (snippetDAO.hasSnippet(snippetDto.getId())) {
snippetDAO.verifyUpdate(snippetDto);
snippetDAO.verifyUpdateSnippetComponent(snippetDto);
}
} catch (final Exception e) {
revisionManager.cancelClaim(snippetDto.getId());
affectedComponentIds.forEach(id -> revisionManager.cancelClaim(snippetDto.getId()));
throw e;
}
}
private Set<Revision> getRevisionsForGroup(final String groupId) {
final Set<Revision> revisions = new HashSet<>();
revisions.add(revisionManager.getRevision(groupId));
final ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
if (processGroup == null) {
throw new IllegalArgumentException("Snippet contains a reference to Process Group with ID " + groupId + " but no Process Group exists with that ID");
}
processGroup.getConnections().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
processGroup.getFunnels().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
processGroup.getInputPorts().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
processGroup.getOutputPorts().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
processGroup.getLabels().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
processGroup.getProcessors().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
processGroup.getRemoteProcessGroups().stream().map(c -> c.getIdentifier()).map(id -> revisionManager.getRevision(id)).forEach(rev -> revisions.add(rev));
processGroup.getProcessGroups().stream().map(c -> c.getIdentifier()).forEach(id -> revisions.addAll(getRevisionsForGroup(id)));
return revisions;
}
private Set<Revision> getRevisionsForSnippet(final SnippetDTO snippetDto) {
final Set<Revision> requiredRevisions = new HashSet<>();
requiredRevisions.add(revisionManager.getRevision(snippetDto.getId()));
snippetDto.getConnections().entrySet().stream()
.map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
.forEach(rev -> requiredRevisions.add(rev));
snippetDto.getFunnels().entrySet().stream()
.map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
.forEach(rev -> requiredRevisions.add(rev));
snippetDto.getInputPorts().entrySet().stream()
.map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
.forEach(rev -> requiredRevisions.add(rev));
snippetDto.getOutputPorts().entrySet().stream()
.map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
.forEach(rev -> requiredRevisions.add(rev));
snippetDto.getLabels().entrySet().stream()
.map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
.forEach(rev -> requiredRevisions.add(rev));
snippetDto.getProcessors().entrySet().stream()
.map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
.forEach(rev -> requiredRevisions.add(rev));
snippetDto.getRemoteProcessGroups().entrySet().stream()
.map(entry -> new Revision(entry.getValue().getVersion(), entry.getValue().getClientId(), entry.getKey()))
.forEach(rev -> requiredRevisions.add(rev));
for (final String groupId : snippetDto.getProcessGroups().keySet()) {
requiredRevisions.addAll(getRevisionsForGroup(groupId));
}
return requiredRevisions;
}
private ProcessGroup getGroup(final String groupId) {
return revisionManager.get(groupId, rev -> processGroupDAO.getProcessGroup(groupId));
}
@Override
public UpdateResult<SnippetEntity> updateSnippet(final Revision revision, final SnippetDTO snippetDto) {
// if label does not exist, then create new label
if (snippetDAO.hasSnippet(snippetDto.getId()) == false) {
return new UpdateResult<>(createSnippet(snippetDto), true);
}
final Set<Revision> requiredRevisions = getRevisionsForSnippet(snippetDto);
// if the parent group is specified in the request, ensure write access to it as it could be moving the components in the snippet
final String requestProcessGroupIdentifier = snippetDto.getParentGroupId();
if (requestProcessGroupIdentifier != null) {
final ProcessGroup requestProcessGroup = processGroupDAO.getProcessGroup(requestProcessGroupIdentifier);
requestProcessGroup.authorize(authorizer, RequestAction.WRITE);
}
public SnippetEntity updateSnippet(final Set<Revision> revisions, final SnippetDTO snippetDto) {
final NiFiUser user = NiFiUserUtils.getNiFiUser();
final String modifier = user.getUserName();
final RevisionClaim revisionClaim = new StandardRevisionClaim(requiredRevisions);
final RevisionClaim revisionClaim = new StandardRevisionClaim(revisions);
RevisionUpdate<SnippetDTO> versionedSnippet;
RevisionUpdate<SnippetDTO> snapshot;
try {
versionedSnippet = revisionManager.updateRevision(revisionClaim, user, new UpdateRevisionTask<SnippetDTO>() {
snapshot = revisionManager.updateRevision(revisionClaim, user, new UpdateRevisionTask<SnippetDTO>() {
@Override
public RevisionUpdate<SnippetDTO> update() {
// get the updated component
final Snippet snippet = snippetDAO.updateSnippet(snippetDto);
final Snippet snippet = snippetDAO.updateSnippetComponents(snippetDto);
// ensure write access to the flow
final ProcessGroup processGroup = getGroup(snippet.getParentGroupId());
processGroup.authorize(authorizer, RequestAction.WRITE);
// drop the snippet
snippetDAO.dropSnippet(snippet.getId());
// save updated controller
controllerFacade.save();
final SnippetDTO snippetDto = dtoFactory.createSnippetDto(snippet);
// increment the revisions
final Set<Revision> updatedRevisions = revisions.stream().map(revision -> incrementRevision(revision)).collect(Collectors.toSet());
// Update each of the revisions that were required and
// build new SnippetDTO that contains all of the updated revisions
final SnippetDTO updatedSnippet = new SnippetDTO();
updatedSnippet.setId(snippetDto.getId());
updatedSnippet.setParentGroupId(snippetDto.getParentGroupId());
updatedSnippet.setUri(snippetDto.getUri());
updatedSnippet.setLinked(snippetDto.isLinked());
updatedSnippet.setConnections(updateRevisions(snippetDto.getConnections(), modifier));
updatedSnippet.setFunnels(updateRevisions(snippetDto.getFunnels(), modifier));
updatedSnippet.setInputPorts(updateRevisions(snippetDto.getInputPorts(), modifier));
updatedSnippet.setLabels(updateRevisions(snippetDto.getLabels(), modifier));
updatedSnippet.setOutputPorts(updateRevisions(snippetDto.getOutputPorts(), modifier));
updatedSnippet.setProcessGroups(updateRevisions(snippetDto.getProcessGroups(), modifier));
updatedSnippet.setProcessors(updateRevisions(snippetDto.getProcessors(), modifier));
updatedSnippet.setRemoteProcessGroups(updateRevisions(snippetDto.getRemoteProcessGroups(), modifier));
final Revision updatedSnippetRevision = incrementRevision(revision);
final FlowModification lastModification = new FlowModification(updatedSnippetRevision, modifier);
return new StandardRevisionUpdate<>(updatedSnippet, lastModification);
final SnippetDTO dto = dtoFactory.createSnippetDto(snippet);
return new StandardRevisionUpdate<>(dto, null, updatedRevisions);
}
});
} catch (ExpiredRevisionClaimException e) {
throw new InvalidRevisionException("Failed to update Snippet", e);
}
final ProcessGroup parentGroup = processGroupDAO.getProcessGroup(versionedSnippet.getComponent().getParentGroupId());
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(parentGroup);
return new UpdateResult<>(entityFactory.createSnippetEntity(versionedSnippet.getComponent(), dtoFactory.createRevisionDTO(versionedSnippet.getLastModification()), accessPolicy), false);
}
private Map<String, RevisionDTO> updateRevisions(final Map<String, RevisionDTO> originalDtos, final String modifier) {
final Map<String, RevisionDTO> updatedComponents = new HashMap<>(originalDtos.size());
for (final Map.Entry<String, RevisionDTO> entry : originalDtos.entrySet()) {
final String id = entry.getKey();
final RevisionDTO revisionDto = entry.getValue();
final RevisionDTO updatedDto = new RevisionDTO();
updatedDto.setClientId(revisionDto.getClientId());
updatedDto.setLastModifier(modifier);
updatedDto.setVersion(revisionDto.getVersion() + 1);
updatedComponents.put(id, updatedDto);
}
return updatedComponents;
return entityFactory.createSnippetEntity(snapshot.getComponent());
}
@Override
@ -839,6 +856,35 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
return new UpdateResult<>(entityFactory.createProcessGroupEntity(snapshot.getComponent(), updatedRevision, accessPolicy, status), false);
}
@Override
public ScheduleComponentsEntity scheduleComponents(final String processGroupId, final ScheduledState state, final Map<String, Revision> componentRevisions) {
final NiFiUser user = NiFiUserUtils.getNiFiUser();
final RevisionUpdate<ScheduleComponentsEntity> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(componentRevisions.values()), user, new
UpdateRevisionTask<ScheduleComponentsEntity>() {
@Override
public RevisionUpdate<ScheduleComponentsEntity> update() {
// schedule the components
processGroupDAO.scheduleComponents(processGroupId, state, componentRevisions.keySet());
// update the revisions
final Map<String, Revision> updatedRevisions = new HashMap<>();
for (final Revision revision : componentRevisions.values()) {
final Revision currentRevision = revisionManager.getRevision(revision.getComponentId());
final Revision updatedRevision = incrementRevision(currentRevision);
updatedRevisions.put(revision.getComponentId(), updatedRevision);
}
// gather details for response
final ScheduleComponentsEntity entity = new ScheduleComponentsEntity();
entity.setId(processGroupId);
entity.setState(state.name());
return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values()));
}
});
return updatedComponent.getComponent();
}
@Override
public ConfigurationSnapshot<ControllerConfigurationDTO> updateControllerConfiguration(final Revision revision, final ControllerConfigurationDTO controllerConfigurationDTO) {
final Supplier<ControllerConfigurationDTO> daoUpdate = () -> {
@ -1032,7 +1078,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
*
* @param revision the current revision
* @param deleteAction the action that deletes the component via the appropriate DAO object
* @return a ConfigurationSnapshot that represents the new configuration
* @return a dto that represents the new configuration
*/
private <D, C> D deleteComponent(final Revision revision, final Authorizable authorizable, final Runnable deleteAction, final D dto) {
final RevisionClaim claim = new StandardRevisionClaim(revision);
@ -1043,17 +1089,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
public D performTask() {
logger.debug("Attempting to delete component {} with claim {}", authorizable, claim);
// ensure access to the component
authorizable.authorize(authorizer, RequestAction.WRITE);
// If the component has outgoing connections, ensure that we can delete them all.
if (authorizable instanceof Connectable) {
final Connectable connectable = (Connectable) authorizable;
for (final Connection connection : connectable.getConnections()) {
connection.authorize(authorizer, RequestAction.WRITE);
}
}
deleteAction.run();
// save the flow
@ -1066,24 +1101,39 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
}
@Override
public void verifyDeleteSnippet(String id) {
snippetDAO.verifyDelete(id);
public void verifyDeleteSnippet(String snippetId, Set<String> affectedComponentIds) {
try {
snippetDAO.verifyDeleteSnippetComponents(snippetId);
} catch (final Exception e) {
affectedComponentIds.forEach(id -> revisionManager.cancelClaim(id));
throw e;
}
}
@Override
public SnippetEntity deleteSnippet(final Revision revision, final String snippetId) {
public SnippetEntity deleteSnippet(final Set<Revision> revisions, final String snippetId) {
final Snippet snippet = snippetDAO.getSnippet(snippetId);
final ProcessGroup processGroup = getGroup(snippet.getParentGroupId());
// ensure access to process group
processGroup.authorize(authorizer, RequestAction.WRITE);
final NiFiUser user = NiFiUserUtils.getNiFiUser();
final RevisionClaim claim = new StandardRevisionClaim(revisions);
final SnippetDTO dto = revisionManager.deleteRevision(claim, user, new DeleteRevisionTask<SnippetDTO>() {
@Override
public SnippetDTO performTask() {
// delete the components in the snippet
snippetDAO.deleteSnippetComponents(snippetId);
final SnippetDTO snapshot = deleteComponent(revision,
processGroup,
() -> snippetDAO.deleteSnippet(snippetId),
dtoFactory.createSnippetDto(snippet));
// drop the snippet
snippetDAO.dropSnippet(snippetId);
return entityFactory.createSnippetEntity(snapshot, null, null);
// save
controllerFacade.save();
// create the dto for the snippet that was just removed
return dtoFactory.createSnippetDto(snippet);
}
});
return entityFactory.createSnippetEntity(dto);
}
@Override
@ -1228,10 +1278,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
final String groupId = componentDto.getParentGroupId();
return revisionManager.get(groupId, rev -> {
// ensure access to process group
final ProcessGroup parent = processGroupDAO.getProcessGroup(groupId);
parent.authorize(authorizer, RequestAction.WRITE);
// add the component
final C component = daoCreation.get();
@ -1319,21 +1365,18 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
public FlowEntity copySnippet(final String groupId, final String snippetId, final Double originX, final Double originY, final String idGenerationSeed) {
final FlowDTO flowDto = revisionManager.get(groupId,
rev -> {
// ensure access to process group
final ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
processGroup.authorize(authorizer, RequestAction.WRITE);
// create the new snippet
final FlowSnippetDTO snippet = snippetDAO.copySnippet(groupId, snippetId, originX, originY, idGenerationSeed);
// TODO - READ access to all components in snippet
// validate the new snippet
validateSnippetContents(snippet);
// save the flow
controllerFacade.save();
// drop the snippet
snippetDAO.dropSnippet(snippetId);
// identify all components added
final Set<String> identifiers = new HashSet<>();
snippet.getProcessors().stream()
@ -1363,6 +1406,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
.map(remoteOutputPort -> remoteOutputPort.getId())
.forEach(id -> identifiers.add(id));
final ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
return revisionManager.get(identifiers,
() -> {
final ProcessGroupStatus groupStatus = controllerFacade.getProcessGroupStatus(groupId);
@ -1381,21 +1425,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
@Override
public SnippetEntity createSnippet(final SnippetDTO snippetDTO) {
final String modifier = NiFiUserUtils.getNiFiUserName();
// ensure id is set
if (StringUtils.isBlank(snippetDTO.getId())) {
snippetDTO.setId(UUID.randomUUID().toString());
}
final String groupId = snippetDTO.getParentGroupId();
final RevisionUpdate<SnippetDTO> snapshot = revisionManager.get(groupId, rev -> {
// ensure access to process group
final ProcessGroup parent = processGroupDAO.getProcessGroup(groupId);
parent.authorize(authorizer, RequestAction.WRITE);
// TODO - READ access to all components in snippet
// add the component
final Snippet snippet = snippetDAO.createSnippet(snippetDTO);
@ -1403,13 +1434,10 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
controllerFacade.save();
final SnippetDTO dto = dtoFactory.createSnippetDto(snippet);
final FlowModification lastMod = new FlowModification(new Revision(0L, rev.getClientId(), snippetDTO.getId()), modifier);
return new StandardRevisionUpdate<SnippetDTO>(dto, lastMod);
return new StandardRevisionUpdate<SnippetDTO>(dto, null);
});
final ProcessGroup parentGroup = processGroupDAO.getProcessGroup(snapshot.getComponent().getParentGroupId());
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(parentGroup);
return entityFactory.createSnippetEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy);
return entityFactory.createSnippetEntity(snapshot.getComponent());
}
@Override
@ -1483,6 +1511,9 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
// create the template
Template template = templateDAO.createTemplate(templateDTO, groupId);
// drop the snippet
snippetDAO.dropSnippet(snippetId);
return dtoFactory.createTemplateDTO(template);
}
@ -1505,16 +1536,10 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
@Override
public FlowEntity createTemplateInstance(final String groupId, final Double originX, final Double originY, final String templateId, final String idGenerationSeed) {
final FlowDTO flowDto = revisionManager.get(groupId, rev -> {
// ensure access to process group
final ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
processGroup.authorize(authorizer, RequestAction.WRITE);
// instantiate the template - there is no need to make another copy of the flow snippet since the actual template
// was copied and this dto is only used to instantiate it's components (which as already completed)
final FlowSnippetDTO snippet = templateDAO.instantiateTemplate(groupId, originX, originY, templateId, idGenerationSeed);
// TODO - READ access to all components in snippet
// validate the new snippet
validateSnippetContents(snippet);
@ -1673,9 +1698,9 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
final ControllerServiceReference updatedReference = controllerServiceDAO.getControllerService(controllerServiceId).getReferences();
final Map<String, Revision> updatedRevisions = new HashMap<>();
for (final Revision refRevision : referenceRevisions.values()) {
updatedRevisions.put(refRevision.getComponentId(), refRevision);
}
for (final Revision refRevision : referenceRevisions.values()) {
updatedRevisions.put(refRevision.getComponentId(), refRevision);
}
for (final ConfiguredComponent component : updated) {
final Revision currentRevision = revisionManager.getRevision(component.getIdentifier());
@ -2087,8 +2112,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
@Override
public FlowFileDTO getFlowFile(String connectionId, String flowFileUuid) {
final Connection connection = connectionDAO.getConnection(connectionId);
connection.authorize(authorizer, RequestAction.WRITE);
return dtoFactory.createFlowFileDTO(connectionDAO.getFlowFile(connectionId, flowFileUuid));
}
@ -2436,16 +2459,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
});
}
@Override
public SnippetEntity getSnippet(String snippetId) {
return revisionManager.get(snippetId, rev -> {
final Snippet snippet = snippetDAO.getSnippet(snippetId);
final ProcessGroup parentGroup = processGroupDAO.getProcessGroup(snippet.getParentGroupId());
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(parentGroup);
return entityFactory.createSnippetEntity(dtoFactory.createSnippetDto(snippet), dtoFactory.createRevisionDTO(rev), accessPolicy);
});
}
@Override
public Set<PortEntity> getInputPorts(String groupId) {
final ProcessGroup group = processGroupDAO.getProcessGroup(groupId);
@ -2584,12 +2597,12 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
}
@Override
public ConfigurationSnapshot<ProcessGroupFlowDTO> getProcessGroupFlow(String groupId, boolean recurse) {
public ProcessGroupFlowEntity getProcessGroupFlow(String groupId, boolean recurse) {
return revisionManager.get(groupId,
rev -> {
// get all identifiers for every child component
final Set<String> identifiers = new HashSet<>();
ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
final ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
processGroup.getProcessors().stream()
.map(proc -> proc.getIdentifier())
.forEach(id -> identifiers.add(id));
@ -2616,17 +2629,13 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
.flatMap(remoteGroup -> remoteGroup.getOutputPorts().stream())
.map(remoteOutputPort -> remoteOutputPort.getIdentifier())
.forEach(id -> identifiers.add(id));
processGroup.getControllerServices(false).stream()
.map(controllerService -> controllerService.getIdentifier())
.forEach(id -> identifiers.add(id));
// read lock on every component being accessed in the dto conversion
return revisionManager.get(identifiers,
() -> {
final ProcessGroupStatus groupStatus = controllerFacade.getProcessGroupStatus(groupId);
ConfigurationSnapshot<ProcessGroupFlowDTO> response = new ConfigurationSnapshot<>(rev.getVersion(),
dtoFactory.createProcessGroupFlowDto(processGroup, groupStatus, revisionManager));
return response;
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(processGroup);
return entityFactory.createProcessGroupFlowEntity(dtoFactory.createProcessGroupFlowDto(processGroup, groupStatus, revisionManager), accessPolicy);
});
});
}

View File

@ -16,16 +16,33 @@
*/
package org.apache.nifi.web.api;
import java.io.Serializable;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.charset.StandardCharsets;
import java.util.Enumeration;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.TreeMap;
import java.util.UUID;
import com.sun.jersey.api.core.HttpContext;
import com.sun.jersey.api.representation.Form;
import com.sun.jersey.core.util.MultivaluedMapImpl;
import com.sun.jersey.server.impl.model.method.dispatch.FormDispatchProvider;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.authorization.user.NiFiUser;
import org.apache.nifi.authorization.user.NiFiUserDetails;
import org.apache.nifi.authorization.user.NiFiUserUtils;
import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator;
import org.apache.nifi.controller.Snippet;
import org.apache.nifi.web.AuthorizableLookup;
import org.apache.nifi.web.AuthorizeAccess;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.api.dto.RevisionDTO;
import org.apache.nifi.web.api.dto.SnippetDTO;
import org.apache.nifi.web.api.entity.ComponentEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.security.jwt.JwtAuthenticationFilter;
import org.apache.nifi.web.util.WebUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.security.core.Authentication;
import org.springframework.security.core.context.SecurityContextHolder;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
@ -37,25 +54,19 @@ import javax.ws.rs.core.Response.ResponseBuilder;
import javax.ws.rs.core.UriBuilder;
import javax.ws.rs.core.UriBuilderException;
import javax.ws.rs.core.UriInfo;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.user.NiFiUserDetails;
import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.api.dto.RevisionDTO;
import org.apache.nifi.web.api.entity.ComponentEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.security.jwt.JwtAuthenticationFilter;
import org.apache.nifi.web.util.WebUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.security.core.Authentication;
import org.springframework.security.core.context.SecurityContextHolder;
import com.sun.jersey.api.core.HttpContext;
import com.sun.jersey.api.representation.Form;
import com.sun.jersey.core.util.MultivaluedMapImpl;
import com.sun.jersey.server.impl.model.method.dispatch.FormDispatchProvider;
import java.io.Serializable;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.charset.StandardCharsets;
import java.util.Enumeration;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.TreeMap;
import java.util.UUID;
import java.util.function.Consumer;
import java.util.function.Supplier;
/**
* Base class for controllers.
@ -369,4 +380,128 @@ public abstract class ApplicationResource {
protected Revision getRevision(ComponentEntity entity, String componentId) {
return getRevision(entity.getRevision(), componentId);
}
/**
* Authorizes the specified Snippet with the specified request action.
*
* @param authorizer authorizer
* @param lookup lookup
* @param action action
*/
protected void authorizeSnippet(final Snippet snippet, final Authorizer authorizer, final AuthorizableLookup lookup, final RequestAction action) {
final Consumer<Authorizable> authorize = authorizable -> authorizable.authorize(authorizer, action);
snippet.getProcessGroups().keySet().stream().map(id -> lookup.getProcessGroup(id)).forEach(authorize);
snippet.getRemoteProcessGroups().keySet().stream().map(id -> lookup.getRemoteProcessGroup(id)).forEach(authorize);
snippet.getProcessors().keySet().stream().map(id -> lookup.getProcessor(id)).forEach(authorize);
snippet.getInputPorts().keySet().stream().map(id -> lookup.getInputPort(id)).forEach(authorize);
snippet.getOutputPorts().keySet().stream().map(id -> lookup.getOutputPort(id)).forEach(authorize);
snippet.getConnections().keySet().stream().map(id -> lookup.getConnection(id)).forEach(authorize);
snippet.getConnections().keySet().stream().map(id -> lookup.getConnection(id)).forEach(authorize);
snippet.getFunnels().keySet().stream().map(id -> lookup.getFunnel(id)).forEach(authorize);
}
/**
* Authorizes the specified Snippet with the specified request action.
*
* @param authorizer authorizer
* @param lookup lookup
* @param action action
*/
protected void authorizeSnippet(final SnippetDTO snippet, final Authorizer authorizer, final AuthorizableLookup lookup, final RequestAction action) {
final Consumer<Authorizable> authorize = authorizable -> authorizable.authorize(authorizer, action);
snippet.getProcessGroups().keySet().stream().map(id -> lookup.getProcessGroup(id)).forEach(authorize);
snippet.getRemoteProcessGroups().keySet().stream().map(id -> lookup.getRemoteProcessGroup(id)).forEach(authorize);
snippet.getProcessors().keySet().stream().map(id -> lookup.getProcessor(id)).forEach(authorize);
snippet.getInputPorts().keySet().stream().map(id -> lookup.getInputPort(id)).forEach(authorize);
snippet.getOutputPorts().keySet().stream().map(id -> lookup.getOutputPort(id)).forEach(authorize);
snippet.getConnections().keySet().stream().map(id -> lookup.getConnection(id)).forEach(authorize);
snippet.getConnections().keySet().stream().map(id -> lookup.getConnection(id)).forEach(authorize);
snippet.getFunnels().keySet().stream().map(id -> lookup.getFunnel(id)).forEach(authorize);
}
/**
* Executes an action through the service facade using the specified revision.
*
* @param serviceFacade service facade
* @param revision revision
* @param authorizer authorizer
* @param verifier verifier
* @param action executor
* @return the response
*/
protected Response withWriteLock(
final NiFiServiceFacade serviceFacade, final Revision revision, final AuthorizeAccess authorizer, final Runnable verifier, final Supplier<Response> action) {
final NiFiUser user = NiFiUserUtils.getNiFiUser();
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
// authorize access
serviceFacade.authorizeAccess(authorizer);
serviceFacade.claimRevision(revision, user);
}
try {
if (validationPhase) {
if (verifier != null) {
verifier.run();
}
return generateContinueResponse().build();
}
} catch (final Exception e) {
serviceFacade.cancelRevision(revision);
throw e;
}
try {
// delete the specified output port
return action.get();
} finally {
serviceFacade.cancelRevision(revision);
}
}
/**
* Executes an action through the service facade using the specified revision.
*
* @param serviceFacade service facade
* @param revisions revisions
* @param authorizer authorizer
* @param verifier verifier
* @param action executor
* @return the response
*/
protected Response withWriteLock(
final NiFiServiceFacade serviceFacade, final Set<Revision> revisions, final AuthorizeAccess authorizer, final Runnable verifier, final Supplier<Response> action) {
final NiFiUser user = NiFiUserUtils.getNiFiUser();
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
// authorize access
serviceFacade.authorizeAccess(authorizer);
serviceFacade.claimRevisions(revisions, user);
}
try {
if (validationPhase) {
if (verifier != null) {
verifier.run();
}
return generateContinueResponse().build();
}
} catch (final Exception e) {
serviceFacade.cancelRevisions(revisions);
throw e;
}
try {
// delete the specified output port
return action.get();
} finally {
serviceFacade.cancelRevisions(revisions);
}
}
}

View File

@ -16,10 +16,27 @@
*/
package org.apache.nifi.web.api;
import java.net.URI;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UpdateResult;
import org.apache.nifi.web.api.dto.ConnectionDTO;
import org.apache.nifi.web.api.dto.FlowFileSummaryDTO;
import org.apache.nifi.web.api.dto.ListingRequestDTO;
import org.apache.nifi.web.api.entity.ConnectionEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
@ -35,26 +52,8 @@ import javax.ws.rs.QueryParam;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UpdateResult;
import org.apache.nifi.web.api.dto.ConnectionDTO;
import org.apache.nifi.web.api.dto.FlowFileSummaryDTO;
import org.apache.nifi.web.api.dto.ListingRequestDTO;
import org.apache.nifi.web.api.entity.ConnectionEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import java.net.URI;
import java.util.Set;
/**
* RESTful endpoint for managing a Connection.
@ -69,6 +68,7 @@ public class ConnectionResource extends ApplicationResource {
private NiFiServiceFacade serviceFacade;
private WebClusterManager clusterManager;
private NiFiProperties properties;
private Authorizer authorizer;
/**
* Populate the URIs for the specified connections.
@ -187,13 +187,19 @@ public class ConnectionResource extends ApplicationResource {
value = "The connection id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable conn = lookup.getConnection(id);
conn.authorize(authorizer, RequestAction.READ);
});
// get the specified relationship
ConnectionEntity entity = serviceFacade.getConnection(id);
populateRemainingConnectionEntityContent(entity);
@ -237,11 +243,11 @@ public class ConnectionResource extends ApplicationResource {
value = "The connection id.",
required = true
)
@PathParam("id") String id,
@PathParam("id") final String id,
@ApiParam(
value = "The connection configuration details.",
required = true
) ConnectionEntity connectionEntity) {
) final ConnectionEntity connectionEntity) {
if (connectionEntity == null || connectionEntity.getComponent() == null) {
throw new IllegalArgumentException("Connection details must be specified.");
@ -261,38 +267,32 @@ public class ConnectionResource extends ApplicationResource {
// replicate if cluster manager
if (properties.isClusterManager()) {
// change content type to JSON for serializing entity
final Map<String, String> headersToOverride = new HashMap<>();
headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
// replicate the request
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), connectionEntity, getHeaders(headersToOverride)).getResponse();
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), connectionEntity, getHeaders()).getResponse();
}
// handle expects request
final Revision revision = getRevision(connectionEntity, id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable conn = lookup.getConnection(id);
conn.authorize(authorizer, RequestAction.WRITE);
},
() -> serviceFacade.verifyUpdateConnection(connection),
() -> {
// update the relationship target
final UpdateResult<ConnectionEntity> updateResult = serviceFacade.updateConnection(revision, connection);
if (validationPhase) {
serviceFacade.verifyUpdateConnection(connection);
return generateContinueResponse().build();
}
final ConnectionEntity entity = updateResult.getResult();
populateRemainingConnectionEntityContent(entity);
// update the relationship target
final UpdateResult<ConnectionEntity> updateResult = serviceFacade.updateConnection(revision, connection);
final ConnectionEntity entity = updateResult.getResult();
populateRemainingConnectionEntityContent(entity);
// generate the response
if (updateResult.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
// generate the response
if (updateResult.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
});
}
/**
@ -331,17 +331,17 @@ public class ConnectionResource extends ApplicationResource {
value = "The revision is used to verify the client is working with the latest version of the flow.",
required = false
)
@QueryParam(VERSION) LongParameter version,
@QueryParam(VERSION) final LongParameter version,
@ApiParam(
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
required = false
)
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) final ClientIdParameter clientId,
@ApiParam(
value = "The connection id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -352,22 +352,23 @@ public class ConnectionResource extends ApplicationResource {
final Long clientVersion = version == null ? null : version.getLong();
final Revision revision = new Revision(clientVersion, clientId.getClientId(), id);
// handle expects request (usually from the cluster manager)
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
// get the current user
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable conn = lookup.getConnection(id);
conn.authorize(authorizer, RequestAction.WRITE);
},
() -> serviceFacade.verifyDeleteConnection(id),
() -> {
// delete the connection
final ConnectionEntity entity = serviceFacade.deleteConnection(revision, id);
if (validationPhase) {
serviceFacade.verifyDeleteConnection(id);
return generateContinueResponse().build();
}
// delete the connection
final ConnectionEntity entity = serviceFacade.deleteConnection(revision, id);
// generate the response
return clusterContext(generateOkResponse(entity)).build();
// generate the response
return clusterContext(generateOkResponse(entity)).build();
}
);
}
// setters
@ -382,4 +383,8 @@ public class ConnectionResource extends ApplicationResource {
public void setProperties(NiFiProperties properties) {
this.properties = properties;
}
public void setAuthorizer(Authorizer authorizer) {
this.authorizer = authorizer;
}
}

View File

@ -23,6 +23,9 @@ import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.service.ControllerServiceState;
@ -64,6 +67,7 @@ import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import java.net.URI;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -84,6 +88,7 @@ public class ControllerServiceResource extends ApplicationResource {
private NiFiServiceFacade serviceFacade;
private WebClusterManager clusterManager;
private NiFiProperties properties;
private Authorizer authorizer;
@Context
private ServletContext servletContext;
@ -184,13 +189,19 @@ public class ControllerServiceResource extends ApplicationResource {
value = "The controller service id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable controllerService = lookup.getControllerService(id);
controllerService.authorize(authorizer, RequestAction.READ);
});
// get the controller service
final ControllerServiceEntity entity = serviceFacade.getControllerService(id);
populateRemainingControllerServiceEntityContent(entity);
@ -233,12 +244,12 @@ public class ControllerServiceResource extends ApplicationResource {
value = "The controller service id.",
required = true
)
@PathParam("id") String id,
@PathParam("id") final String id,
@ApiParam(
value = "The property name to return the descriptor for.",
required = true
)
@QueryParam("propertyName") String propertyName) {
@QueryParam("propertyName") final String propertyName) {
// ensure the property name is specified
if (propertyName == null) {
@ -250,6 +261,12 @@ public class ControllerServiceResource extends ApplicationResource {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable controllerService = lookup.getControllerService(id);
controllerService.authorize(authorizer, RequestAction.READ);
});
// get the property descriptor
final PropertyDescriptorDTO descriptor = serviceFacade.getControllerServicePropertyDescriptor(id, propertyName);
@ -293,13 +310,19 @@ public class ControllerServiceResource extends ApplicationResource {
value = "The controller service id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable controllerService = lookup.getControllerService(id);
controllerService.authorize(authorizer, RequestAction.WRITE);
});
// get the component state
final ComponentStateDTO state = serviceFacade.getControllerServiceState(id);
@ -344,12 +367,12 @@ public class ControllerServiceResource extends ApplicationResource {
@ApiParam(
value = "The revision used to verify the client is working with the latest version of the flow.",
required = true
) ComponentStateEntity revisionEntity,
) final ComponentStateEntity revisionEntity,
@ApiParam(
value = "The controller service id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -359,6 +382,11 @@ public class ControllerServiceResource extends ApplicationResource {
// handle expects request (usually from the cluster manager)
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase) {
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable controllerService = lookup.getControllerService(id);
controllerService.authorize(authorizer, RequestAction.WRITE);
});
serviceFacade.verifyCanClearControllerServiceState(id);
return generateContinueResponse().build();
}
@ -407,13 +435,19 @@ public class ControllerServiceResource extends ApplicationResource {
value = "The controller service id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable controllerService = lookup.getControllerService(id);
controllerService.authorize(authorizer, RequestAction.READ);
});
// get the controller service
final ControllerServiceReferencingComponentsEntity entity = serviceFacade.getControllerServiceReferencingComponents(id);
@ -449,11 +483,16 @@ public class ControllerServiceResource extends ApplicationResource {
}
)
public Response updateControllerServiceReferences(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The controller service id.",
required = true
)
@PathParam("id") final String id,
@ApiParam(
value = "The controller service request update request.",
required = true
) UpdateControllerServiceReferenceRequestEntity updateReferenceRequest) {
) final UpdateControllerServiceReferenceRequestEntity updateReferenceRequest) {
if (updateReferenceRequest.getId() == null) {
throw new IllegalArgumentException("The controller service identifier must be specified.");
@ -467,29 +506,31 @@ public class ControllerServiceResource extends ApplicationResource {
// need to consider controller service state first as it shares a state with
// scheduled state (disabled) which is applicable for referencing services
// but not referencing schedulable components
ControllerServiceState controllerServiceState = null;
ControllerServiceState requestControllerServiceState = null;
try {
controllerServiceState = ControllerServiceState.valueOf(updateReferenceRequest.getState());
requestControllerServiceState = ControllerServiceState.valueOf(updateReferenceRequest.getState());
} catch (final IllegalArgumentException iae) {
// ignore
}
ScheduledState scheduledState = null;
ScheduledState requestScheduledState = null;
try {
scheduledState = ScheduledState.valueOf(updateReferenceRequest.getState());
requestScheduledState = ScheduledState.valueOf(updateReferenceRequest.getState());
} catch (final IllegalArgumentException iae) {
// ignore
}
// ensure an action has been specified
if (scheduledState == null && controllerServiceState == null) {
if (requestScheduledState == null && requestControllerServiceState == null) {
throw new IllegalArgumentException("Must specify the updated state. To update referencing Processors "
+ "and Reporting Tasks the state should be RUNNING or STOPPED. To update the referencing Controller Services the "
+ "state should be ENABLED or DISABLED.");
}
// ensure the controller service state is not ENABLING or DISABLING
if (controllerServiceState != null && (ControllerServiceState.ENABLING.equals(controllerServiceState) || ControllerServiceState.DISABLING.equals(controllerServiceState))) {
if (requestControllerServiceState != null
&& (ControllerServiceState.ENABLING.equals(requestControllerServiceState) || ControllerServiceState.DISABLING.equals(requestControllerServiceState))) {
throw new IllegalArgumentException("Cannot set the referencing services to ENABLING or DISABLING");
}
@ -504,24 +545,28 @@ public class ControllerServiceResource extends ApplicationResource {
final RevisionDTO rev = e.getValue();
return new Revision(rev.getVersion(), rev.getClientId(), e.getKey());
}));
final Set<Revision> revisions = new HashSet<>(referencingRevisions.values());
// handle expects request (usually from the cluster manager)
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
referencingRevisions.entrySet().stream().forEach(e -> {
serviceFacade.claimRevision(e.getValue());
});
}
if (validationPhase) {
serviceFacade.verifyUpdateControllerServiceReferencingComponents(updateReferenceRequest.getId(), scheduledState, controllerServiceState);
return generateContinueResponse().build();
}
final ScheduledState scheduledState = requestScheduledState;
final ControllerServiceState controllerServiceState = requestControllerServiceState;
return withWriteLock(
serviceFacade,
revisions,
lookup -> {
referencingRevisions.entrySet().stream().forEach(e -> {
final Authorizable controllerService = lookup.getControllerServiceReferencingComponent(id, e.getKey());
controllerService.authorize(authorizer, RequestAction.WRITE);
});
},
() -> serviceFacade.verifyUpdateControllerServiceReferencingComponents(updateReferenceRequest.getId(), scheduledState, controllerServiceState),
() -> {
// update the controller service references
final ControllerServiceReferencingComponentsEntity entity = serviceFacade.updateControllerServiceReferencingComponents(
referencingRevisions, updateReferenceRequest.getId(), scheduledState, controllerServiceState);
// update the controller service references
final ControllerServiceReferencingComponentsEntity entity = serviceFacade.updateControllerServiceReferencingComponents(
referencingRevisions, updateReferenceRequest.getId(), scheduledState, controllerServiceState);
return clusterContext(generateOkResponse(entity)).build();
return clusterContext(generateOkResponse(entity)).build();
}
);
}
/**
@ -559,11 +604,11 @@ public class ControllerServiceResource extends ApplicationResource {
value = "The controller service id.",
required = true
)
@PathParam("id") String id,
@PathParam("id") final String id,
@ApiParam(
value = "The controller service configuration details.",
required = true
) ControllerServiceEntity controllerServiceEntity) {
) final ControllerServiceEntity controllerServiceEntity) {
if (controllerServiceEntity == null || controllerServiceEntity.getComponent() == null) {
throw new IllegalArgumentException("Controller service details must be specified.");
@ -587,27 +632,29 @@ public class ControllerServiceResource extends ApplicationResource {
// handle expects request (usually from the cluster manager)
final Revision revision = getRevision(controllerServiceEntity, id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
serviceFacade.verifyUpdateControllerService(requestControllerServiceDTO);
return generateContinueResponse().build();
}
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable controllerService = lookup.getControllerService(id);
controllerService.authorize(authorizer, RequestAction.WRITE);
},
() -> serviceFacade.verifyUpdateControllerService(requestControllerServiceDTO),
() -> {
// update the controller service
final UpdateResult<ControllerServiceEntity> updateResult = serviceFacade.updateControllerService(revision, requestControllerServiceDTO);
// update the controller service
final UpdateResult<ControllerServiceEntity> updateResult = serviceFacade.updateControllerService(revision, requestControllerServiceDTO);
// build the response entity
final ControllerServiceEntity entity = updateResult.getResult();
populateRemainingControllerServiceContent(entity.getComponent());
// build the response entity
final ControllerServiceEntity entity = updateResult.getResult();
populateRemainingControllerServiceContent(entity.getComponent());
if (updateResult.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
if (updateResult.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
}
);
}
/**
@ -649,17 +696,17 @@ public class ControllerServiceResource extends ApplicationResource {
value = "The revision is used to verify the client is working with the latest version of the flow.",
required = false
)
@QueryParam(VERSION) LongParameter version,
@QueryParam(VERSION) final LongParameter version,
@ApiParam(
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
required = false
)
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) final ClientIdParameter clientId,
@ApiParam(
value = "The controller service id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -668,18 +715,20 @@ public class ControllerServiceResource extends ApplicationResource {
// handle expects request (usually from the cluster manager)
final Revision revision = new Revision(version == null ? null : version.getLong(), clientId.getClientId(), id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
serviceFacade.verifyDeleteControllerService(id);
return generateContinueResponse().build();
}
// delete the specified controller service
final ControllerServiceEntity entity = serviceFacade.deleteControllerService(revision, id);
return clusterContext(generateOkResponse(entity)).build();
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable controllerService = lookup.getControllerService(id);
controllerService.authorize(authorizer, RequestAction.WRITE);
},
() -> serviceFacade.verifyDeleteControllerService(id),
() -> {
// delete the specified controller service
final ControllerServiceEntity entity = serviceFacade.deleteControllerService(revision, id);
return clusterContext(generateOkResponse(entity)).build();
}
);
}
// setters
@ -694,4 +743,8 @@ public class ControllerServiceResource extends ApplicationResource {
public void setProperties(NiFiProperties properties) {
this.properties = properties;
}
public void setAuthorizer(Authorizer authorizer) {
this.authorizer = authorizer;
}
}

View File

@ -16,32 +16,16 @@
*/
package org.apache.nifi.web.api;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.URI;
import java.util.HashSet;
import java.util.Set;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
import javax.ws.rs.DELETE;
import javax.ws.rs.DefaultValue;
import javax.ws.rs.GET;
import javax.ws.rs.HttpMethod;
import javax.ws.rs.POST;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.WebApplicationException;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.Response.Status;
import javax.ws.rs.core.StreamingOutput;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator;
import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
@ -60,12 +44,29 @@ import org.apache.nifi.web.api.entity.FlowFileEntity;
import org.apache.nifi.web.api.entity.ListingRequestEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
import javax.ws.rs.DELETE;
import javax.ws.rs.DefaultValue;
import javax.ws.rs.GET;
import javax.ws.rs.HttpMethod;
import javax.ws.rs.POST;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.WebApplicationException;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.Response.Status;
import javax.ws.rs.core.StreamingOutput;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.URI;
import java.util.HashSet;
import java.util.Set;
/**
* RESTful endpoint for managing a flowfile queue.
@ -75,12 +76,12 @@ import com.wordnik.swagger.annotations.Authorization;
value = "/flowfile-queues",
description = "Endpoint for managing a FlowFile Queue."
)
// TODO: Need revisions of the Connections for these endpoints!
public class FlowFileQueueResource extends ApplicationResource {
private NiFiServiceFacade serviceFacade;
private WebClusterManager clusterManager;
private NiFiProperties properties;
private Authorizer authorizer;
/**
* Populate the URIs for the specified flowfile listing.
@ -147,17 +148,17 @@ public class FlowFileQueueResource extends ApplicationResource {
value = "The connection id.",
required = true
)
@PathParam("connection-id") String connectionId,
@PathParam("connection-id") final String connectionId,
@ApiParam(
value = "The flowfile uuid.",
required = true
)
@PathParam("flowfile-uuid") String flowFileUuid,
@PathParam("flowfile-uuid") final String flowFileUuid,
@ApiParam(
value = "The id of the node where the content exists if clustered.",
required = false
)
@QueryParam("clusterNodeId") String clusterNodeId) {
@QueryParam("clusterNodeId") final String clusterNodeId) {
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -179,6 +180,12 @@ public class FlowFileQueueResource extends ApplicationResource {
}
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable connection = lookup.getConnection(connectionId);
connection.authorize(authorizer, RequestAction.WRITE);
});
// get the flowfile
final FlowFileDTO flowfileDto = serviceFacade.getFlowFile(connectionId, flowFileUuid);
populateRemainingFlowFileContent(connectionId, flowfileDto);
@ -224,22 +231,22 @@ public class FlowFileQueueResource extends ApplicationResource {
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
required = false
)
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) final ClientIdParameter clientId,
@ApiParam(
value = "The connection id.",
required = true
)
@PathParam("connection-id") String connectionId,
@PathParam("connection-id") final String connectionId,
@ApiParam(
value = "The flowfile uuid.",
required = true
)
@PathParam("flowfile-uuid") String flowFileUuid,
@PathParam("flowfile-uuid") final String flowFileUuid,
@ApiParam(
value = "The id of the node where the content exists if clustered.",
required = false
)
@QueryParam("clusterNodeId") String clusterNodeId) {
@QueryParam("clusterNodeId") final String clusterNodeId) {
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -261,6 +268,12 @@ public class FlowFileQueueResource extends ApplicationResource {
}
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable connection = lookup.getConnection(connectionId);
connection.authorize(authorizer, RequestAction.WRITE);
});
// get the uri of the request
final String uri = generateResourceUri("flowfile-queues", connectionId, "flowfiles", flowFileUuid, "content");
@ -320,18 +333,24 @@ public class FlowFileQueueResource extends ApplicationResource {
}
)
public Response createFlowFileListing(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The connection id.",
required = true
)
@PathParam("connection-id") String id) {
@PathParam("connection-id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable connection = lookup.getConnection(id);
connection.authorize(authorizer, RequestAction.WRITE);
});
// handle expects request (usually from the cluster manager)
final String expects = httpServletRequest.getHeader(RequestReplicator.REQUEST_VALIDATION_HTTP_HEADER);
if (expects != null) {
@ -388,18 +407,24 @@ public class FlowFileQueueResource extends ApplicationResource {
value = "The connection id.",
required = true
)
@PathParam("connection-id") String connectionId,
@PathParam("connection-id") final String connectionId,
@ApiParam(
value = "The listing request id.",
required = true
)
@PathParam("listing-request-id") String listingRequestId) {
@PathParam("listing-request-id") final String listingRequestId) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable connection = lookup.getConnection(connectionId);
connection.authorize(authorizer, RequestAction.WRITE);
});
// get the listing request
final ListingRequestDTO listingRequest = serviceFacade.getFlowFileListingRequest(connectionId, listingRequestId);
populateRemainingFlowFileListingContent(connectionId, listingRequest);
@ -441,17 +466,17 @@ public class FlowFileQueueResource extends ApplicationResource {
}
)
public Response deleteListingRequest(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The connection id.",
required = true
)
@PathParam("connection-id") String connectionId,
@PathParam("connection-id") final String connectionId,
@ApiParam(
value = "The listing request id.",
required = true
)
@PathParam("listing-request-id") String listingRequestId) {
@PathParam("listing-request-id") final String listingRequestId) {
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -464,6 +489,12 @@ public class FlowFileQueueResource extends ApplicationResource {
return generateContinueResponse().build();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable connection = lookup.getConnection(connectionId);
connection.authorize(authorizer, RequestAction.WRITE);
});
// delete the listing request
final ListingRequestDTO listingRequest = serviceFacade.deleteFlowFileListingRequest(connectionId, listingRequestId);
@ -510,18 +541,24 @@ public class FlowFileQueueResource extends ApplicationResource {
}
)
public Response createDropRequest(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The connection id.",
required = true
)
@PathParam("connection-id") String id) {
@PathParam("connection-id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable connection = lookup.getConnection(id);
connection.authorize(authorizer, RequestAction.WRITE);
});
// handle expects request (usually from the cluster manager)
final String expects = httpServletRequest.getHeader(RequestReplicator.REQUEST_VALIDATION_HTTP_HEADER);
if (expects != null) {
@ -577,18 +614,24 @@ public class FlowFileQueueResource extends ApplicationResource {
value = "The connection id.",
required = true
)
@PathParam("connection-id") String connectionId,
@PathParam("connection-id") final String connectionId,
@ApiParam(
value = "The drop request id.",
required = true
)
@PathParam("drop-request-id") String dropRequestId) {
@PathParam("drop-request-id") final String dropRequestId) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable connection = lookup.getConnection(connectionId);
connection.authorize(authorizer, RequestAction.WRITE);
});
// get the drop request
final DropRequestDTO dropRequest = serviceFacade.getFlowFileDropRequest(connectionId, dropRequestId);
dropRequest.setUri(generateResourceUri("flowfile-queues", connectionId, "drop-requests", dropRequestId));
@ -630,23 +673,29 @@ public class FlowFileQueueResource extends ApplicationResource {
}
)
public Response removeDropRequest(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The connection id.",
required = true
)
@PathParam("connection-id") String connectionId,
@PathParam("connection-id") final String connectionId,
@ApiParam(
value = "The drop request id.",
required = true
)
@PathParam("drop-request-id") String dropRequestId) {
@PathParam("drop-request-id") final String dropRequestId) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.DELETE, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable connection = lookup.getConnection(connectionId);
connection.authorize(authorizer, RequestAction.WRITE);
});
// handle expects request (usually from the cluster manager)
final String expects = httpServletRequest.getHeader(RequestReplicator.REQUEST_VALIDATION_HTTP_HEADER);
if (expects != null) {
@ -676,4 +725,8 @@ public class FlowFileQueueResource extends ApplicationResource {
public void setProperties(NiFiProperties properties) {
this.properties = properties;
}
public void setAuthorizer(Authorizer authorizer) {
this.authorizer = authorizer;
}
}

View File

@ -16,22 +16,13 @@
*/
package org.apache.nifi.web.api;
import java.util.HashSet;
import java.util.Set;
import javax.ws.rs.Consumes;
import javax.ws.rs.DefaultValue;
import javax.ws.rs.GET;
import javax.ws.rs.HttpMethod;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.WebApplicationException;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import com.sun.jersey.api.core.ResourceContext;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.AccessDeniedException;
import org.apache.nifi.authorization.AuthorizationRequest;
@ -39,6 +30,7 @@ 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.Authorizable;
import org.apache.nifi.authorization.resource.ResourceFactory;
import org.apache.nifi.authorization.user.NiFiUser;
import org.apache.nifi.authorization.user.NiFiUserUtils;
@ -47,9 +39,11 @@ import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.cluster.node.Node;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.ConfigurationSnapshot;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.api.dto.AboutDTO;
import org.apache.nifi.web.api.dto.BannerDTO;
import org.apache.nifi.web.api.dto.BulletinBoardDTO;
@ -88,6 +82,7 @@ import org.apache.nifi.web.api.entity.ProcessorStatusEntity;
import org.apache.nifi.web.api.entity.ProcessorTypesEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupStatusEntity;
import org.apache.nifi.web.api.entity.ReportingTaskTypesEntity;
import org.apache.nifi.web.api.entity.ScheduleComponentsEntity;
import org.apache.nifi.web.api.entity.SearchResultsEntity;
import org.apache.nifi.web.api.entity.StatusHistoryEntity;
import org.apache.nifi.web.api.request.BulletinBoardPatternParameter;
@ -95,13 +90,26 @@ import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.IntegerParameter;
import org.apache.nifi.web.api.request.LongParameter;
import com.sun.jersey.api.core.ResourceContext;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
import javax.ws.rs.DefaultValue;
import javax.ws.rs.GET;
import javax.ws.rs.HttpMethod;
import javax.ws.rs.PUT;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.WebApplicationException;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
/**
* RESTful endpoint for managing a Flow.
@ -254,20 +262,11 @@ public class FlowResource extends ApplicationResource {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// get this process group contents
final ConfigurationSnapshot<ProcessGroupFlowDTO> controllerResponse = serviceFacade.getProcessGroupFlow(groupId, recursive);
final ProcessGroupFlowDTO flow = controllerResponse.getConfiguration();
// get this process group flow
final ProcessGroupFlowEntity entity = serviceFacade.getProcessGroupFlow(groupId, recursive);
populateRemainingFlowContent(entity.getProcessGroupFlow());
// create the revision
final RevisionDTO revision = new RevisionDTO();
revision.setClientId(clientId.getClientId());
revision.setVersion(controllerResponse.getVersion());
// create the response entity
final ProcessGroupFlowEntity processGroupEntity = new ProcessGroupFlowEntity();
processGroupEntity.setProcessGroupFlow(populateRemainingFlowContent(flow));
return clusterContext(generateOkResponse(processGroupEntity)).build();
return clusterContext(generateOkResponse(entity)).build();
}
/**
@ -321,6 +320,142 @@ public class FlowResource extends ApplicationResource {
return clusterContext(generateOkResponse(entity)).build();
}
/**
* Updates the specified process group.
*
* @param httpServletRequest request
* @param id The id of the process group.
* @param scheduleComponentsEntity A scheduleComponentsEntity.
* @return A processGroupEntity.
*/
@PUT
@Consumes(MediaType.APPLICATION_JSON)
@Produces(MediaType.APPLICATION_JSON)
@Path("process-groups/{id}")
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
@ApiOperation(
value = "Updates a process group",
response = ScheduleComponentsEntity.class,
authorizations = {
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
}
)
@ApiResponses(
value = {
@ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
@ApiResponse(code = 401, message = "Client could not be authenticated."),
@ApiResponse(code = 403, message = "Client is not authorized to make this request."),
@ApiResponse(code = 404, message = "The specified resource could not be found."),
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
}
)
public Response scheduleComponents(
@Context HttpServletRequest httpServletRequest,
@ApiParam(
value = "The process group id.",
required = true
)
@PathParam("id") String id,
ScheduleComponentsEntity scheduleComponentsEntity) {
authorizeFlow();
// ensure the same id is being used
if (!id.equals(scheduleComponentsEntity.getId())) {
throw new IllegalArgumentException(String.format("The process group id (%s) in the request body does "
+ "not equal the process group id of the requested resource (%s).", scheduleComponentsEntity.getId(), id));
}
final ScheduledState state;
if (scheduleComponentsEntity.getState() == null) {
throw new IllegalArgumentException("The scheduled state must be specified.");
} else {
try {
state = ScheduledState.valueOf(scheduleComponentsEntity.getState());
} catch (final IllegalArgumentException iae) {
throw new IllegalArgumentException(String.format("The scheduled must be one of [%s].", StringUtils.join(EnumSet.of(ScheduledState.RUNNING, ScheduledState.STOPPED), ", ")));
}
}
// ensure its a supported scheduled state
if (ScheduledState.DISABLED.equals(state) || ScheduledState.STARTING.equals(state) || ScheduledState.STOPPING.equals(state)) {
throw new IllegalArgumentException(String.format("The scheduled must be one of [%s].", StringUtils.join(EnumSet.of(ScheduledState.RUNNING, ScheduledState.STOPPED), ", ")));
}
// if the components are not specified, gather all components and their current revision
if (scheduleComponentsEntity.getComponents() == null) {
// TODO - this will break while clustered until nodes are able to process/replicate requests
// get the current revisions for the components being updated
final Set<Revision> revisions = serviceFacade.getRevisionsFromGroup(id, group -> {
final Set<String> componentIds = new HashSet<>();
// ensure authorized for each processor we will attempt to schedule
group.findAllProcessors().stream()
.filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PROCESSORS : ProcessGroup.UNSCHEDULABLE_PROCESSORS)
.filter(processor -> processor.isAuthorized(authorizer, RequestAction.WRITE))
.forEach(processor -> {
componentIds.add(processor.getIdentifier());
});
// ensure authorized for each input port we will attempt to schedule
group.findAllInputPorts().stream()
.filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PORTS : ProcessGroup.UNSCHEDULABLE_PORTS)
.filter(inputPort -> inputPort.isAuthorized(authorizer, RequestAction.WRITE))
.forEach(inputPort -> {
componentIds.add(inputPort.getIdentifier());
});
// ensure authorized for each output port we will attempt to schedule
group.findAllOutputPorts().stream()
.filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PORTS : ProcessGroup.UNSCHEDULABLE_PORTS)
.filter(outputPort -> outputPort.isAuthorized(authorizer, RequestAction.WRITE))
.forEach(outputPort -> {
componentIds.add(outputPort.getIdentifier());
});
return componentIds;
});
// build the component mapping
final Map<String, RevisionDTO> componentsToSchedule = new HashMap<>();
revisions.forEach(revision -> {
final RevisionDTO dto = new RevisionDTO();
dto.setClientId(revision.getClientId());
dto.setVersion(revision.getVersion());
componentsToSchedule.put(revision.getComponentId(), dto);
});
// set the components and their current revision
scheduleComponentsEntity.setComponents(componentsToSchedule);
}
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), scheduleComponentsEntity, getHeaders()).getResponse();
}
final Map<String, RevisionDTO> componentsToSchedule = scheduleComponentsEntity.getComponents();
final Map<String, Revision> componentRevisions = componentsToSchedule.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> getRevision(e.getValue(), e.getKey())));
final Set<Revision> revisions = new HashSet<>(componentRevisions.values());
return withWriteLock(
serviceFacade,
revisions,
lookup -> {
// ensure access to every component being scheduled
componentsToSchedule.keySet().forEach(componentId -> {
final Authorizable connectable = lookup.getConnectable(componentId);
connectable.authorize(authorizer, RequestAction.WRITE);
});
},
() -> serviceFacade.verifyScheduleComponents(id, state, componentRevisions.keySet()),
() -> {
// update the process group
final ScheduleComponentsEntity entity = serviceFacade.scheduleComponents(id, state, componentRevisions);
return clusterContext(generateOkResponse(entity)).build();
}
);
}
@GET
@Consumes(MediaType.WILDCARD)
@Produces(MediaType.TEXT_PLAIN)

View File

@ -16,10 +16,27 @@
*/
package org.apache.nifi.web.api;
import java.net.URI;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UpdateResult;
import org.apache.nifi.web.api.dto.FunnelDTO;
import org.apache.nifi.web.api.entity.FunnelEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
@ -35,26 +52,8 @@ import javax.ws.rs.QueryParam;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UpdateResult;
import org.apache.nifi.web.api.dto.FunnelDTO;
import org.apache.nifi.web.api.entity.FunnelEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import java.net.URI;
import java.util.Set;
/**
* RESTful endpoint for managing a Funnel.
@ -71,6 +70,7 @@ public class FunnelResource extends ApplicationResource {
private NiFiServiceFacade serviceFacade;
private WebClusterManager clusterManager;
private NiFiProperties properties;
private Authorizer authorizer;
/**
* Populates the uri for the specified funnels.
@ -154,13 +154,19 @@ public class FunnelResource extends ApplicationResource {
value = "The funnel id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable funnel = lookup.getFunnel(id);
funnel.authorize(authorizer, RequestAction.READ);
});
// get the funnel
final FunnelEntity entity = serviceFacade.getFunnel(id);
populateRemainingFunnelEntityContent(entity);
@ -198,16 +204,16 @@ public class FunnelResource extends ApplicationResource {
}
)
public Response updateFunnel(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The funnel id.",
required = true
)
@PathParam("id") String id,
@PathParam("id") final String id,
@ApiParam(
value = "The funnel configuration details.",
required = true
) FunnelEntity funnelEntity) {
) final FunnelEntity funnelEntity) {
if (funnelEntity == null || funnelEntity.getComponent() == null) {
throw new IllegalArgumentException("Funnel details must be specified.");
@ -226,39 +232,34 @@ public class FunnelResource extends ApplicationResource {
// replicate if cluster manager
if (properties.isClusterManager()) {
// change content type to JSON for serializing entity
final Map<String, String> headersToOverride = new HashMap<>();
headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
// replicate the request
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), funnelEntity, getHeaders(headersToOverride)).getResponse();
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), funnelEntity, getHeaders()).getResponse();
}
// Extract the revision
final Revision revision = getRevision(funnelEntity, id);
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable funnel = lookup.getFunnel(id);
funnel.authorize(authorizer, RequestAction.WRITE);
},
null,
() -> {
// update the funnel
final UpdateResult<FunnelEntity> updateResult = serviceFacade.updateFunnel(revision, requestFunnelDTO);
// handle expects request (usually from the cluster manager)
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
serviceFacade.claimRevision(revision);
return generateContinueResponse().build();
}
// get the results
final FunnelEntity entity = updateResult.getResult();
populateRemainingFunnelEntityContent(entity);
// update the funnel
final UpdateResult<FunnelEntity> updateResult = serviceFacade.updateFunnel(revision, requestFunnelDTO);
// get the results
final FunnelEntity entity = updateResult.getResult();
populateRemainingFunnelEntityContent(entity);
if (updateResult.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
if (updateResult.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
}
);
}
/**
@ -295,22 +296,22 @@ public class FunnelResource extends ApplicationResource {
}
)
public Response removeFunnel(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The revision is used to verify the client is working with the latest version of the flow.",
required = false
)
@QueryParam(VERSION) LongParameter version,
@QueryParam(VERSION) final LongParameter version,
@ApiParam(
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
required = false
)
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) final ClientIdParameter clientId,
@ApiParam(
value = "The funnel id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -319,18 +320,20 @@ public class FunnelResource extends ApplicationResource {
// handle expects request (usually from the cluster manager)
final Revision revision = new Revision(version == null ? null : version.getLong(), clientId.getClientId(), id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
serviceFacade.verifyDeleteFunnel(id);
return generateContinueResponse().build();
}
// delete the specified funnel
final FunnelEntity entity = serviceFacade.deleteFunnel(revision, id);
return clusterContext(generateOkResponse(entity)).build();
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable funnel = lookup.getFunnel(id);
funnel.authorize(authorizer, RequestAction.READ);
},
() -> serviceFacade.verifyDeleteFunnel(id),
() -> {
// delete the specified funnel
final FunnelEntity entity = serviceFacade.deleteFunnel(revision, id);
return clusterContext(generateOkResponse(entity)).build();
}
);
}
// setters
@ -345,4 +348,8 @@ public class FunnelResource extends ApplicationResource {
public void setProperties(NiFiProperties properties) {
this.properties = properties;
}
public void setAuthorizer(Authorizer authorizer) {
this.authorizer = authorizer;
}
}

View File

@ -16,10 +16,25 @@
*/
package org.apache.nifi.web.api;
import java.net.URI;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UpdateResult;
import org.apache.nifi.web.api.dto.PortDTO;
import org.apache.nifi.web.api.entity.PortEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
@ -35,24 +50,8 @@ import javax.ws.rs.QueryParam;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UpdateResult;
import org.apache.nifi.web.api.dto.PortDTO;
import org.apache.nifi.web.api.entity.PortEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import java.net.URI;
import java.util.Set;
/**
* RESTful endpoint for managing an Input Port.
@ -67,6 +66,7 @@ public class InputPortResource extends ApplicationResource {
private NiFiServiceFacade serviceFacade;
private WebClusterManager clusterManager;
private NiFiProperties properties;
private Authorizer authorizer;
/**
* Populates the uri for the specified input ports.
@ -150,13 +150,19 @@ public class InputPortResource extends ApplicationResource {
value = "The input port id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable inputPort = lookup.getInputPort(id);
inputPort.authorize(authorizer, RequestAction.READ);
});
// get the port
final PortEntity entity = serviceFacade.getInputPort(id);
populateRemainingInputPortEntityContent(entity);
@ -199,11 +205,11 @@ public class InputPortResource extends ApplicationResource {
value = "The input port id.",
required = true
)
@PathParam("id") String id,
@PathParam("id") final String id,
@ApiParam(
value = "The input port configuration details.",
required = true
) PortEntity portEntity) {
) final PortEntity portEntity) {
if (portEntity == null || portEntity.getComponent() == null) {
throw new IllegalArgumentException("Input port details must be specified.");
@ -222,37 +228,34 @@ public class InputPortResource extends ApplicationResource {
// replicate if cluster manager
if (properties.isClusterManager()) {
// change content type to JSON for serializing entity
final Map<String, String> headersToOverride = new HashMap<>();
headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
// replicate the request
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), portEntity, getHeaders(headersToOverride)).getResponse();
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), portEntity, getHeaders()).getResponse();
}
// handle expects request (usually from the cluster manager)
final Revision revision = getRevision(portEntity, id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
serviceFacade.verifyUpdateInputPort(requestPortDTO);
return generateContinueResponse().build();
}
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable inputPort = lookup.getInputPort(id);
inputPort.authorize(authorizer, RequestAction.WRITE);
},
() -> serviceFacade.verifyUpdateInputPort(requestPortDTO),
() -> {
// update the input port
final UpdateResult<PortEntity> updateResult = serviceFacade.updateInputPort(revision, requestPortDTO);
// update the input port
final UpdateResult<PortEntity> updateResult = serviceFacade.updateInputPort(revision, requestPortDTO);
// build the response entity
final PortEntity entity = updateResult.getResult();
populateRemainingInputPortEntityContent(entity);
// build the response entity
final PortEntity entity = updateResult.getResult();
populateRemainingInputPortEntityContent(entity);
if (updateResult.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
if (updateResult.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
}
);
}
/**
@ -291,17 +294,17 @@ public class InputPortResource extends ApplicationResource {
value = "The revision is used to verify the client is working with the latest version of the flow.",
required = false
)
@QueryParam(VERSION) LongParameter version,
@QueryParam(VERSION) final LongParameter version,
@ApiParam(
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
required = false
)
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) final ClientIdParameter clientId,
@ApiParam(
value = "The input port id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -310,18 +313,20 @@ public class InputPortResource extends ApplicationResource {
// handle expects request (usually from the cluster manager)
final Revision revision = new Revision(version == null ? null : version.getLong(), clientId.getClientId(), id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
serviceFacade.verifyDeleteInputPort(id);
return generateContinueResponse().build();
}
// delete the specified input port
final PortEntity entity = serviceFacade.deleteInputPort(revision, id);
return clusterContext(generateOkResponse(entity)).build();
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable inputPort = lookup.getInputPort(id);
inputPort.authorize(authorizer, RequestAction.WRITE);
},
() -> serviceFacade.verifyDeleteInputPort(id),
() -> {
// delete the specified input port
final PortEntity entity = serviceFacade.deleteInputPort(revision, id);
return clusterContext(generateOkResponse(entity)).build();
}
);
}
// setters
@ -336,4 +341,8 @@ public class InputPortResource extends ApplicationResource {
public void setProperties(NiFiProperties properties) {
this.properties = properties;
}
public void setAuthorizer(Authorizer authorizer) {
this.authorizer = authorizer;
}
}

View File

@ -16,10 +16,27 @@
*/
package org.apache.nifi.web.api;
import java.net.URI;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UpdateResult;
import org.apache.nifi.web.api.dto.LabelDTO;
import org.apache.nifi.web.api.entity.LabelEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
@ -35,26 +52,8 @@ import javax.ws.rs.QueryParam;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UpdateResult;
import org.apache.nifi.web.api.dto.LabelDTO;
import org.apache.nifi.web.api.entity.LabelEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import java.net.URI;
import java.util.Set;
/**
* RESTful endpoint for managing a Label.
@ -71,6 +70,7 @@ public class LabelResource extends ApplicationResource {
private NiFiServiceFacade serviceFacade;
private WebClusterManager clusterManager;
private NiFiProperties properties;
private Authorizer authorizer;
/**
* Populates the uri for the specified labels.
@ -154,13 +154,19 @@ public class LabelResource extends ApplicationResource {
value = "The label id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable label = lookup.getLabel(id);
label.authorize(authorizer, RequestAction.READ);
});
// get the label
final LabelEntity entity = serviceFacade.getLabel(id);
populateRemainingLabelEntityContent(entity);
@ -198,16 +204,16 @@ public class LabelResource extends ApplicationResource {
}
)
public Response updateLabel(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The label id.",
required = true
)
@PathParam("id") String id,
@PathParam("id") final String id,
@ApiParam(
value = "The label configuraiton details.",
required = true
) LabelEntity labelEntity) {
) final LabelEntity labelEntity) {
if (labelEntity == null || labelEntity.getComponent() == null) {
throw new IllegalArgumentException("Label details must be specified.");
@ -226,34 +232,32 @@ public class LabelResource extends ApplicationResource {
// replicate if cluster manager
if (properties.isClusterManager()) {
// change content type to JSON for serializing entity
final Map<String, String> headersToOverride = new HashMap<>();
headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
// replicate the request
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), labelEntity, getHeaders(headersToOverride)).getResponse();
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), labelEntity, getHeaders()).getResponse();
}
// handle expects request (usually from the cluster manager)
final Revision revision = getRevision(labelEntity, id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
return generateContinueResponse().build();
}
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable label = lookup.getLabel(id);
label.authorize(authorizer, RequestAction.WRITE);
},
null,
() -> {
// update the label
final UpdateResult<LabelEntity> result = serviceFacade.updateLabel(revision, requestLabelDTO);
final LabelEntity entity = result.getResult();
populateRemainingLabelEntityContent(entity);
// update the label
final UpdateResult<LabelEntity> result = serviceFacade.updateLabel(revision, requestLabelDTO);
final LabelEntity entity = result.getResult();
populateRemainingLabelEntityContent(entity);
if (result.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
if (result.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
}
);
}
/**
@ -287,22 +291,22 @@ public class LabelResource extends ApplicationResource {
}
)
public Response removeLabel(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The revision is used to verify the client is working with the latest version of the flow.",
required = false
)
@QueryParam(VERSION) LongParameter version,
@QueryParam(VERSION) final LongParameter version,
@ApiParam(
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
required = false
)
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) final ClientIdParameter clientId,
@ApiParam(
value = "The label id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -311,17 +315,20 @@ public class LabelResource extends ApplicationResource {
// handle expects request (usually from the cluster manager)
final Revision revision = new Revision(version == null ? null : version.getLong(), clientId.getClientId(), id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
return generateContinueResponse().build();
}
// delete the specified label
final LabelEntity entity = serviceFacade.deleteLabel(revision, id);
return clusterContext(generateOkResponse(entity)).build();
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable label = lookup.getLabel(id);
label.authorize(authorizer, RequestAction.WRITE);
},
null,
() -> {
// delete the specified label
final LabelEntity entity = serviceFacade.deleteLabel(revision, id);
return clusterContext(generateOkResponse(entity)).build();
}
);
}
// setters
@ -336,4 +343,8 @@ public class LabelResource extends ApplicationResource {
public void setProperties(NiFiProperties properties) {
this.properties = properties;
}
public void setAuthorizer(Authorizer authorizer) {
this.authorizer = authorizer;
}
}

View File

@ -16,10 +16,27 @@
*/
package org.apache.nifi.web.api;
import java.net.URI;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UpdateResult;
import org.apache.nifi.web.api.dto.PortDTO;
import org.apache.nifi.web.api.entity.PortEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
@ -35,26 +52,8 @@ import javax.ws.rs.QueryParam;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UpdateResult;
import org.apache.nifi.web.api.dto.PortDTO;
import org.apache.nifi.web.api.entity.PortEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import java.net.URI;
import java.util.Set;
/**
* RESTful endpoint for managing an Output Port.
@ -71,6 +70,7 @@ public class OutputPortResource extends ApplicationResource {
private NiFiServiceFacade serviceFacade;
private WebClusterManager clusterManager;
private NiFiProperties properties;
private Authorizer authorizer;
/**
* Populates the uri for the specified output ports.
@ -154,13 +154,19 @@ public class OutputPortResource extends ApplicationResource {
value = "The output port id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable outputPort = lookup.getOutputPort(id);
outputPort.authorize(authorizer, RequestAction.READ);
});
// get the port
final PortEntity entity = serviceFacade.getOutputPort(id);
populateRemainingOutputPortEntityContent(entity);
@ -198,16 +204,16 @@ public class OutputPortResource extends ApplicationResource {
}
)
public Response updateOutputPort(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The output port id.",
required = true
)
@PathParam("id") String id,
@PathParam("id") final String id,
@ApiParam(
value = "The output port configuration details.",
required = true
) PortEntity portEntity) {
) final PortEntity portEntity) {
if (portEntity == null || portEntity.getComponent() == null) {
throw new IllegalArgumentException("Output port details must be specified.");
@ -226,37 +232,34 @@ public class OutputPortResource extends ApplicationResource {
// replicate if cluster manager
if (properties.isClusterManager()) {
// change content type to JSON for serializing entity
final Map<String, String> headersToOverride = new HashMap<>();
headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
// replicate the request
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), portEntity, getHeaders(headersToOverride)).getResponse();
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), portEntity, getHeaders()).getResponse();
}
// handle expects request (usually from the cluster manager)
final Revision revision = getRevision(portEntity, id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
serviceFacade.verifyUpdateOutputPort(requestPortDTO);
return generateContinueResponse().build();
}
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable outputPort = lookup.getOutputPort(id);
outputPort.authorize(authorizer, RequestAction.WRITE);
},
() -> serviceFacade.verifyUpdateOutputPort(requestPortDTO),
() -> {
// update the output port
final UpdateResult<PortEntity> updateResult = serviceFacade.updateOutputPort(revision, requestPortDTO);
// update the output port
final UpdateResult<PortEntity> updateResult = serviceFacade.updateOutputPort(revision, requestPortDTO);
// get the results
final PortEntity entity = updateResult.getResult();
populateRemainingOutputPortEntityContent(entity);
// get the results
final PortEntity entity = updateResult.getResult();
populateRemainingOutputPortEntityContent(entity);
if (updateResult.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
if (updateResult.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
}
);
}
/**
@ -290,22 +293,22 @@ public class OutputPortResource extends ApplicationResource {
}
)
public Response removeOutputPort(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The revision is used to verify the client is working with the latest version of the flow.",
required = false
)
@QueryParam(VERSION) LongParameter version,
@QueryParam(VERSION) final LongParameter version,
@ApiParam(
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
required = false
)
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) final ClientIdParameter clientId,
@ApiParam(
value = "The output port id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -314,18 +317,20 @@ public class OutputPortResource extends ApplicationResource {
// handle expects request (usually from the cluster manager)
final Revision revision = new Revision(version == null ? null : version.getLong(), clientId.getClientId(), id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
serviceFacade.verifyDeleteOutputPort(id);
return generateContinueResponse().build();
}
// delete the specified output port
final PortEntity entity = serviceFacade.deleteOutputPort(revision, id);
return clusterContext(generateOkResponse(entity)).build();
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable outputPort = lookup.getOutputPort(id);
outputPort.authorize(authorizer, RequestAction.WRITE);
},
() -> serviceFacade.verifyDeleteOutputPort(id),
() -> {
// delete the specified output port
final PortEntity entity = serviceFacade.deleteOutputPort(revision, id);
return clusterContext(generateOkResponse(entity)).build();
}
);
}
// setters
@ -340,4 +345,8 @@ public class OutputPortResource extends ApplicationResource {
public void setProperties(NiFiProperties properties) {
this.properties = properties;
}
public void setAuthorizer(Authorizer authorizer) {
this.authorizer = authorizer;
}
}

View File

@ -23,6 +23,9 @@ import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.cluster.manager.exception.IllegalClusterStateException;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.cluster.node.Node;
@ -83,6 +86,7 @@ public class ProcessorResource extends ApplicationResource {
private NiFiServiceFacade serviceFacade;
private WebClusterManager clusterManager;
private NiFiProperties properties;
private Authorizer authorizer;
@Context
private ServletContext servletContext;
@ -193,13 +197,19 @@ public class ProcessorResource extends ApplicationResource {
value = "The processor id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable processor = lookup.getProcessor(id);
processor.authorize(authorizer, RequestAction.READ);
});
// get the specified processor
final ProcessorEntity entity = serviceFacade.getProcessor(id);
populateRemainingProcessorEntityContent(entity);
@ -243,17 +253,17 @@ public class ProcessorResource extends ApplicationResource {
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
required = false
)
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) final ClientIdParameter clientId,
@ApiParam(
value = "The processor id.",
required = true
)
@PathParam("id") String id,
@PathParam("id") final String id,
@ApiParam(
value = "The property name.",
required = true
)
@QueryParam("propertyName") String propertyName) {
@QueryParam("propertyName") final String propertyName) {
// ensure the property name is specified
if (propertyName == null) {
@ -265,6 +275,12 @@ public class ProcessorResource extends ApplicationResource {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable processor = lookup.getProcessor(id);
processor.authorize(authorizer, RequestAction.READ);
});
// get the property descriptor
final PropertyDescriptorDTO descriptor = serviceFacade.getProcessorPropertyDescriptor(id, propertyName);
@ -308,13 +324,19 @@ public class ProcessorResource extends ApplicationResource {
value = "The processor id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable processor = lookup.getProcessor(id);
processor.authorize(authorizer, RequestAction.WRITE);
});
// get the component state
final ComponentStateDTO state = serviceFacade.getProcessorState(id);
@ -356,17 +378,16 @@ public class ProcessorResource extends ApplicationResource {
}
)
public Response clearState(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The revision used to verify the client is working with the latest version of the flow.",
required = true
)
ComponentStateEntity revisionEntity,
) final ComponentStateEntity revisionEntity,
@ApiParam(
value = "The processor id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// ensure the revision was specified
if (revisionEntity == null) {
@ -380,6 +401,11 @@ public class ProcessorResource extends ApplicationResource {
// handle expects request (usually from the cluster manager)
if (isValidationPhase(httpServletRequest)) {
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable processor = lookup.getProcessor(id);
processor.authorize(authorizer, RequestAction.WRITE);
});
serviceFacade.verifyCanClearProcessorState(id);
return generateContinueResponse().build();
}
@ -424,17 +450,16 @@ public class ProcessorResource extends ApplicationResource {
}
)
public Response updateProcessor(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The processor id.",
required = true
)
@PathParam("id") String id,
@PathParam("id") final String id,
@ApiParam(
value = "The processor configuration details.",
required = true
)
ProcessorEntity processorEntity) {
) final ProcessorEntity processorEntity) {
if (processorEntity == null || processorEntity.getComponent() == null) {
throw new IllegalArgumentException("Processor details must be specified.");
@ -468,41 +493,27 @@ public class ProcessorResource extends ApplicationResource {
// handle expects request (usually from the cluster manager)
final Revision revision = getRevision(processorEntity, id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
final boolean twoPhaseRequest = isTwoPhaseRequest(httpServletRequest);
final String requestId = getHeaders().get("X-RequestTransactionId");
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable processor = lookup.getProcessor(id);
processor.authorize(authorizer, RequestAction.WRITE);
},
() -> serviceFacade.verifyUpdateProcessor(requestProcessorDTO),
() -> {
// update the processor
final UpdateResult<ProcessorEntity> result = serviceFacade.updateProcessor(revision, requestProcessorDTO);
final ProcessorEntity entity = result.getResult();
populateRemainingProcessorEntityContent(entity);
logger.debug("For Update Processor, Validation Phase = {}, Two-phase request = {}, Request ID = {}", validationPhase, twoPhaseRequest, requestId);
if (validationPhase || !twoPhaseRequest) {
serviceFacade.claimRevision(revision);
logger.debug("Claimed Revision {}", revision);
}
if (validationPhase) {
serviceFacade.verifyUpdateProcessor(requestProcessorDTO);
logger.debug("Verified Update of Processor");
return generateContinueResponse().build();
}
// update the processor
final UpdateResult<ProcessorEntity> result;
try {
logger.debug("Updating Processor with Revision {}", revision);
result = serviceFacade.updateProcessor(revision, requestProcessorDTO);
logger.debug("Updated Processor with Revision {}", revision);
} catch (final Exception e) {
final boolean tpr = isTwoPhaseRequest(httpServletRequest);
logger.error("Got Exception trying to update processor. two-phase request = {}, validation phase = {}, revision = {}", tpr, validationPhase, revision);
logger.error("", e);
throw e;
}
final ProcessorEntity entity = result.getResult();
populateRemainingProcessorEntityContent(entity);
if (result.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
if (result.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
}
);
}
/**
@ -536,22 +547,22 @@ public class ProcessorResource extends ApplicationResource {
}
)
public Response deleteProcessor(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The revision is used to verify the client is working with the latest version of the flow.",
required = false
)
@QueryParam(VERSION) LongParameter version,
@QueryParam(VERSION) final LongParameter version,
@ApiParam(
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
required = false
)
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) final ClientIdParameter clientId,
@ApiParam(
value = "The processor id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -559,25 +570,22 @@ public class ProcessorResource extends ApplicationResource {
}
final Revision revision = new Revision(version == null ? null : version.getLong(), clientId.getClientId(), id);
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable processor = lookup.getProcessor(id);
processor.authorize(authorizer, RequestAction.WRITE);
},
() -> serviceFacade.verifyDeleteProcessor(id),
() -> {
// delete the processor
final ProcessorEntity entity = serviceFacade.deleteProcessor(revision, id);
// handle expects request (usually from the cluster manager)
final boolean validationPhase = isValidationPhase(httpServletRequest);
// We need to claim the revision for the Processor if either this is the first phase of a two-phase
// request, or if this is not a two-phase request.
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
serviceFacade.verifyDeleteProcessor(id);
return generateContinueResponse().build();
}
// delete the processor
final ProcessorEntity entity = serviceFacade.deleteProcessor(revision, id);
// generate the response
return clusterContext(generateOkResponse(entity)).build();
// generate the response
return clusterContext(generateOkResponse(entity)).build();
}
);
}
// setters
@ -592,4 +600,8 @@ public class ProcessorResource extends ApplicationResource {
public void setProperties(NiFiProperties properties) {
this.properties = properties;
}
public void setAuthorizer(Authorizer authorizer) {
this.authorizer = authorizer;
}
}

View File

@ -16,31 +16,22 @@
*/
package org.apache.nifi.web.api;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.URI;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
import javax.ws.rs.DELETE;
import javax.ws.rs.GET;
import javax.ws.rs.HttpMethod;
import javax.ws.rs.POST;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.WebApplicationException;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.StreamingOutput;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.AccessDeniedException;
import org.apache.nifi.authorization.AuthorizationRequest;
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.ResourceFactory;
import org.apache.nifi.authorization.user.NiFiUser;
import org.apache.nifi.authorization.user.NiFiUserUtils;
import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator;
import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
@ -65,12 +56,29 @@ import org.apache.nifi.web.api.request.LongParameter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
import javax.ws.rs.DELETE;
import javax.ws.rs.GET;
import javax.ws.rs.HttpMethod;
import javax.ws.rs.POST;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.WebApplicationException;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.StreamingOutput;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.URI;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
/**
@ -89,6 +97,7 @@ public class ProvenanceResource extends ApplicationResource {
private NiFiProperties properties;
private NiFiServiceFacade serviceFacade;
private WebClusterManager clusterManager;
private Authorizer authorizer;
/**
* Populates the uri for the specified provenance.
@ -106,6 +115,24 @@ public class ProvenanceResource extends ApplicationResource {
return lineage;
}
private void authorizeProvenanceRequest() {
final NiFiUser user = NiFiUserUtils.getNiFiUser();
final AuthorizationRequest request = new AuthorizationRequest.Builder()
.resource(ResourceFactory.getProvenanceResource())
.identity(user.getIdentity())
.anonymous(user.isAnonymous())
.accessAttempt(true)
.action(RequestAction.READ)
.build();
final AuthorizationResult result = authorizer.authorize(request);
if (!Result.Approved.equals(result.getResult())) {
final String message = StringUtils.isNotBlank(result.getExplanation()) ? result.getExplanation() : "Access is denied";
throw new AccessDeniedException(message);
}
}
/**
* Gets the provenance search options for this NiFi.
*
@ -133,6 +160,8 @@ public class ProvenanceResource extends ApplicationResource {
)
public Response getSearchOptions() {
authorizeProvenanceRequest();
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
@ -178,11 +207,13 @@ public class ProvenanceResource extends ApplicationResource {
}
)
public Response submitReplay(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The replay request.",
required = true
) SubmitReplayRequestEntity replayRequestEntity) {
) final SubmitReplayRequestEntity replayRequestEntity) {
authorizeProvenanceRequest();
// ensure the event id is specified
if (replayRequestEntity == null || replayRequestEntity.getEventId() == null) {
@ -259,12 +290,14 @@ public class ProvenanceResource extends ApplicationResource {
value = "The id of the node where the content exists if clustered.",
required = false
)
@QueryParam("clusterNodeId") String clusterNodeId,
@QueryParam("clusterNodeId") final String clusterNodeId,
@ApiParam(
value = "The provenance event id.",
required = true
)
@PathParam("id") LongParameter id) {
@PathParam("id") final LongParameter id) {
authorizeProvenanceRequest();
// ensure proper input
if (id == null) {
@ -352,12 +385,14 @@ public class ProvenanceResource extends ApplicationResource {
value = "The id of the node where the content exists if clustered.",
required = false
)
@QueryParam("clusterNodeId") String clusterNodeId,
@QueryParam("clusterNodeId") final String clusterNodeId,
@ApiParam(
value = "The provenance event id.",
required = true
)
@PathParam("id") LongParameter id) {
@PathParam("id") final LongParameter id) {
authorizeProvenanceRequest();
// ensure proper input
if (id == null) {
@ -445,12 +480,14 @@ public class ProvenanceResource extends ApplicationResource {
}
)
public Response submitProvenanceRequest(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The provenance query details.",
required = true
) ProvenanceEntity provenanceEntity) {
authorizeProvenanceRequest();
// check the request
if (provenanceEntity == null) {
provenanceEntity = new ProvenanceEntity();
@ -546,12 +583,14 @@ public class ProvenanceResource extends ApplicationResource {
value = "The id of the node where this query exists if clustered.",
required = false
)
@QueryParam("clusterNodeId") String clusterNodeId,
@QueryParam("clusterNodeId") final String clusterNodeId,
@ApiParam(
value = "The id of the provenance query.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
authorizeProvenanceRequest();
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -617,17 +656,19 @@ public class ProvenanceResource extends ApplicationResource {
}
)
public Response deleteProvenance(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The id of the node where this query exists if clustered.",
required = false
)
@QueryParam("clusterNodeId") String clusterNodeId,
@QueryParam("clusterNodeId") final String clusterNodeId,
@ApiParam(
value = "The id of the provenance query.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
authorizeProvenanceRequest();
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -699,12 +740,14 @@ public class ProvenanceResource extends ApplicationResource {
value = "The id of the node where this event exists if clustered.",
required = false
)
@QueryParam("clusterNodeId") String clusterNodeId,
@QueryParam("clusterNodeId") final String clusterNodeId,
@ApiParam(
value = "The provenence event id.",
required = true
)
@PathParam("id") LongParameter id) {
@PathParam("id") final LongParameter id) {
authorizeProvenanceRequest();
// ensure the id is specified
if (id == null) {
@ -780,12 +823,13 @@ public class ProvenanceResource extends ApplicationResource {
}
)
public Response submitLineageRequest(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The lineage query details.",
required = true
)
final LineageEntity lineageEntity) {
) final LineageEntity lineageEntity) {
authorizeProvenanceRequest();
if (lineageEntity == null || lineageEntity.getLineage() == null || lineageEntity.getLineage().getRequest() == null) {
throw new IllegalArgumentException("Lineage request must be specified.");
@ -892,12 +936,14 @@ public class ProvenanceResource extends ApplicationResource {
value = "The id of the node where this query exists if clustered.",
required = false
)
@QueryParam("clusterNodeId") String clusterNodeId,
@QueryParam("clusterNodeId") final String clusterNodeId,
@ApiParam(
value = "The id of the lineage query.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
authorizeProvenanceRequest();
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -961,17 +1007,19 @@ public class ProvenanceResource extends ApplicationResource {
}
)
public Response deleteLineage(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The id of the node where this query exists if clustered.",
required = false
)
@QueryParam("clusterNodeId") String clusterNodeId,
@QueryParam("clusterNodeId") final String clusterNodeId,
@ApiParam(
value = "The id of the lineage query.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
authorizeProvenanceRequest();
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -1020,4 +1068,8 @@ public class ProvenanceResource extends ApplicationResource {
public void setServiceFacade(NiFiServiceFacade serviceFacade) {
this.serviceFacade = serviceFacade;
}
public void setAuthorizer(Authorizer authorizer) {
this.authorizer = authorizer;
}
}

View File

@ -16,10 +16,28 @@
*/
package org.apache.nifi.web.api;
import java.net.URI;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UpdateResult;
import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
import org.apache.nifi.web.api.dto.RevisionDTO;
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupPortEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
@ -35,27 +53,8 @@ import javax.ws.rs.QueryParam;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UpdateResult;
import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
import org.apache.nifi.web.api.dto.RevisionDTO;
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupPortEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import java.net.URI;
import java.util.Set;
/**
* RESTful endpoint for managing a Remote group.
@ -72,6 +71,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
private NiFiServiceFacade serviceFacade;
private WebClusterManager clusterManager;
private NiFiProperties properties;
private Authorizer authorizer;
/**
* Populates the remaining content for each remote process group. The uri must be generated and the remote process groups name must be retrieved.
@ -160,18 +160,24 @@ public class RemoteProcessGroupResource extends ApplicationResource {
value = "Whether to include any encapulated ports or just details about the remote process group.",
required = false
)
@QueryParam("verbose") @DefaultValue(VERBOSE_DEFAULT_VALUE) Boolean verbose,
@QueryParam("verbose") @DefaultValue(VERBOSE_DEFAULT_VALUE) final Boolean verbose,
@ApiParam(
value = "The remote process group id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable remoteProcessGroup = lookup.getRemoteProcessGroup(id);
remoteProcessGroup.authorize(authorizer, RequestAction.READ);
});
// get the remote process group
final RemoteProcessGroupEntity entity = serviceFacade.getRemoteProcessGroup(id);
populateRemainingRemoteProcessGroupEntityContent(entity);
@ -217,22 +223,22 @@ public class RemoteProcessGroupResource extends ApplicationResource {
}
)
public Response removeRemoteProcessGroup(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The revision is used to verify the client is working with the latest version of the flow.",
required = false
)
@QueryParam(VERSION) LongParameter version,
@QueryParam(VERSION) final LongParameter version,
@ApiParam(
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
required = false
)
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) final ClientIdParameter clientId,
@ApiParam(
value = "The remote process group id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -241,17 +247,19 @@ public class RemoteProcessGroupResource extends ApplicationResource {
// handle expects request (usually from the cluster manager)
final Revision revision = new Revision(version == null ? null : version.getLong(), clientId.getClientId(), id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
serviceFacade.verifyDeleteRemoteProcessGroup(id);
return generateContinueResponse().build();
}
final RemoteProcessGroupEntity entity = serviceFacade.deleteRemoteProcessGroup(revision, id);
return clusterContext(generateOkResponse(entity)).build();
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable remoteProcessGroup = lookup.getRemoteProcessGroup(id);
remoteProcessGroup.authorize(authorizer, RequestAction.WRITE);
},
() -> serviceFacade.verifyDeleteRemoteProcessGroup(id),
() -> {
final RemoteProcessGroupEntity entity = serviceFacade.deleteRemoteProcessGroup(revision, id);
return clusterContext(generateOkResponse(entity)).build();
}
);
}
/**
@ -286,10 +294,10 @@ public class RemoteProcessGroupResource extends ApplicationResource {
}
)
public Response updateRemoteProcessGroupInputPort(
@Context HttpServletRequest httpServletRequest,
@PathParam("id") String id,
@PathParam("port-id") String portId,
RemoteProcessGroupPortEntity remoteProcessGroupPortEntity) {
@Context final HttpServletRequest httpServletRequest,
@PathParam("id") final String id,
@PathParam("port-id") final String portId,
final RemoteProcessGroupPortEntity remoteProcessGroupPortEntity) {
if (remoteProcessGroupPortEntity == null || remoteProcessGroupPortEntity.getRemoteProcessGroupPort() == null) {
throw new IllegalArgumentException("Remote process group port details must be specified.");
@ -308,38 +316,34 @@ public class RemoteProcessGroupResource extends ApplicationResource {
// replicate if cluster manager
if (properties.isClusterManager()) {
// change content type to JSON for serializing entity
final Map<String, String> headersToOverride = new HashMap<>();
headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
// replicate the request
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), remoteProcessGroupPortEntity, getHeaders(headersToOverride)).getResponse();
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), remoteProcessGroupPortEntity, getHeaders()).getResponse();
}
// handle expects request (usually from the cluster manager)
final Revision revision = getRevision(remoteProcessGroupPortEntity, id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
// verify the update at this time
serviceFacade.verifyUpdateRemoteProcessGroupInputPort(id, requestRemoteProcessGroupPort);
return generateContinueResponse().build();
}
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable remoteProcessGroupInputPort = lookup.getRemoteProcessGroupInputPort(id, portId);
remoteProcessGroupInputPort.authorize(authorizer, RequestAction.WRITE);
},
() -> serviceFacade.verifyUpdateRemoteProcessGroupInputPort(id, requestRemoteProcessGroupPort),
() -> {
// update the specified remote process group
final RemoteProcessGroupPortEntity controllerResponse = serviceFacade.updateRemoteProcessGroupInputPort(revision, id, requestRemoteProcessGroupPort);
// update the specified remote process group
final RemoteProcessGroupPortEntity controllerResponse = serviceFacade.updateRemoteProcessGroupInputPort(revision, id, requestRemoteProcessGroupPort);
// get the updated revision
final RevisionDTO updatedRevision = controllerResponse.getRevision();
// get the updated revision
final RevisionDTO updatedRevision = controllerResponse.getRevision();
// build the response entity
final RemoteProcessGroupPortEntity entity = new RemoteProcessGroupPortEntity();
entity.setRevision(updatedRevision);
entity.setRemoteProcessGroupPort(controllerResponse.getRemoteProcessGroupPort());
// build the response entity
final RemoteProcessGroupPortEntity entity = new RemoteProcessGroupPortEntity();
entity.setRevision(updatedRevision);
entity.setRemoteProcessGroupPort(controllerResponse.getRemoteProcessGroupPort());
return clusterContext(generateOkResponse(entity)).build();
return clusterContext(generateOkResponse(entity)).build();
}
);
}
/**
@ -396,38 +400,34 @@ public class RemoteProcessGroupResource extends ApplicationResource {
// replicate if cluster manager
if (properties.isClusterManager()) {
// change content type to JSON for serializing entity
final Map<String, String> headersToOverride = new HashMap<>();
headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
// replicate the request
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), remoteProcessGroupPortEntity, getHeaders(headersToOverride)).getResponse();
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), remoteProcessGroupPortEntity, getHeaders()).getResponse();
}
// handle expects request (usually from the cluster manager)
final Revision revision = getRevision(remoteProcessGroupPortEntity, portId);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
// verify the update at this time
serviceFacade.verifyUpdateRemoteProcessGroupOutputPort(id, requestRemoteProcessGroupPort);
return generateContinueResponse().build();
}
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable remoteProcessGroupOutputPort = lookup.getRemoteProcessGroupOutputPort(id, portId);
remoteProcessGroupOutputPort.authorize(authorizer, RequestAction.WRITE);
},
() -> serviceFacade.verifyUpdateRemoteProcessGroupOutputPort(id, requestRemoteProcessGroupPort),
() -> {
// update the specified remote process group
final RemoteProcessGroupPortEntity controllerResponse = serviceFacade.updateRemoteProcessGroupOutputPort(revision, id, requestRemoteProcessGroupPort);
// update the specified remote process group
final RemoteProcessGroupPortEntity controllerResponse = serviceFacade.updateRemoteProcessGroupOutputPort(revision, id, requestRemoteProcessGroupPort);
// get the updated revision
final RevisionDTO updatedRevision = controllerResponse.getRevision();
// get the updated revision
final RevisionDTO updatedRevision = controllerResponse.getRevision();
// build the response entity
RemoteProcessGroupPortEntity entity = new RemoteProcessGroupPortEntity();
entity.setRevision(updatedRevision);
entity.setRemoteProcessGroupPort(controllerResponse.getRemoteProcessGroupPort());
// build the response entity
RemoteProcessGroupPortEntity entity = new RemoteProcessGroupPortEntity();
entity.setRevision(updatedRevision);
entity.setRemoteProcessGroupPort(controllerResponse.getRemoteProcessGroupPort());
return clusterContext(generateOkResponse(entity)).build();
return clusterContext(generateOkResponse(entity)).build();
}
);
}
/**
@ -481,69 +481,65 @@ public class RemoteProcessGroupResource extends ApplicationResource {
// replicate if cluster manager
if (properties.isClusterManager()) {
// change content type to JSON for serializing entity
final Map<String, String> headersToOverride = new HashMap<>();
headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
// replicate the request
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), remoteProcessGroupEntity, getHeaders(headersToOverride)).getResponse();
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), remoteProcessGroupEntity, getHeaders()).getResponse();
}
// handle expects request (usually from the cluster manager)
final Revision revision = getRevision(remoteProcessGroupEntity, id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
// verify the update at this time
serviceFacade.verifyUpdateRemoteProcessGroup(requestRemoteProcessGroup);
return generateContinueResponse().build();
}
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable remoteProcessGroup = lookup.getRemoteProcessGroup(id);
remoteProcessGroup.authorize(authorizer, RequestAction.WRITE);
},
() -> serviceFacade.verifyUpdateRemoteProcessGroup(requestRemoteProcessGroup),
() -> {
// if the target uri is set we have to verify it here - we don't support updating the target uri on
// an existing remote process group, however if the remote process group is being created with an id
// as is the case in clustered mode we need to verify the remote process group. treat this request as
// though its a new remote process group.
if (requestRemoteProcessGroup.getTargetUri() != null) {
// parse the uri
final URI uri;
try {
uri = URI.create(requestRemoteProcessGroup.getTargetUri());
} catch (final IllegalArgumentException e) {
throw new IllegalArgumentException("The specified remote process group URL is malformed: " + requestRemoteProcessGroup.getTargetUri());
}
// if the target uri is set we have to verify it here - we don't support updating the target uri on
// an existing remote process group, however if the remote process group is being created with an id
// as is the case in clustered mode we need to verify the remote process group. treat this request as
// though its a new remote process group.
if (requestRemoteProcessGroup.getTargetUri() != null) {
// parse the uri
final URI uri;
try {
uri = URI.create(requestRemoteProcessGroup.getTargetUri());
} catch (final IllegalArgumentException e) {
throw new IllegalArgumentException("The specified remote process group URL is malformed: " + requestRemoteProcessGroup.getTargetUri());
// validate each part of the uri
if (uri.getScheme() == null || uri.getHost() == null) {
throw new IllegalArgumentException("The specified remote process group URL is malformed: " + requestRemoteProcessGroup.getTargetUri());
}
if (!(uri.getScheme().equalsIgnoreCase("http") || uri.getScheme().equalsIgnoreCase("https"))) {
throw new IllegalArgumentException("The specified remote process group URL is invalid because it is not http or https: " + requestRemoteProcessGroup.getTargetUri());
}
// normalize the uri to the other controller
String controllerUri = uri.toString();
if (controllerUri.endsWith("/")) {
controllerUri = StringUtils.substringBeforeLast(controllerUri, "/");
}
// update with the normalized uri
requestRemoteProcessGroup.setTargetUri(controllerUri);
}
// update the specified remote process group
final UpdateResult<RemoteProcessGroupEntity> updateResult = serviceFacade.updateRemoteProcessGroup(revision, requestRemoteProcessGroup);
final RemoteProcessGroupEntity entity = updateResult.getResult();
populateRemainingRemoteProcessGroupEntityContent(entity);
if (updateResult.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
}
// validate each part of the uri
if (uri.getScheme() == null || uri.getHost() == null) {
throw new IllegalArgumentException("The specified remote process group URL is malformed: " + requestRemoteProcessGroup.getTargetUri());
}
if (!(uri.getScheme().equalsIgnoreCase("http") || uri.getScheme().equalsIgnoreCase("https"))) {
throw new IllegalArgumentException("The specified remote process group URL is invalid because it is not http or https: " + requestRemoteProcessGroup.getTargetUri());
}
// normalize the uri to the other controller
String controllerUri = uri.toString();
if (controllerUri.endsWith("/")) {
controllerUri = StringUtils.substringBeforeLast(controllerUri, "/");
}
// update with the normalized uri
requestRemoteProcessGroup.setTargetUri(controllerUri);
}
// update the specified remote process group
final UpdateResult<RemoteProcessGroupEntity> updateResult = serviceFacade.updateRemoteProcessGroup(revision, requestRemoteProcessGroup);
final RemoteProcessGroupEntity entity = updateResult.getResult();
populateRemainingRemoteProcessGroupEntityContent(entity);
if (updateResult.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
);
}
// setters
@ -559,4 +555,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
this.properties = properties;
}
public void setAuthorizer(Authorizer authorizer) {
this.authorizer = authorizer;
}
}

View File

@ -23,6 +23,9 @@ import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.ui.extension.UiExtension;
import org.apache.nifi.ui.extension.UiExtensionMapping;
@ -73,6 +76,7 @@ public class ReportingTaskResource extends ApplicationResource {
private NiFiServiceFacade serviceFacade;
private WebClusterManager clusterManager;
private NiFiProperties properties;
private Authorizer authorizer;
@Context
private ServletContext servletContext;
@ -142,9 +146,6 @@ public class ReportingTaskResource extends ApplicationResource {
/**
* Retrieves the specified reporting task.
*
* @param clientId Optional client id. If the client id is not specified, a
* new one will be generated. This value (whether specified or generated) is
* included in the response.
* @param id The id of the reporting task to retrieve
* @return A reportingTaskEntity.
*/
@ -172,22 +173,23 @@ public class ReportingTaskResource extends ApplicationResource {
}
)
public Response getReportingTask(
@ApiParam(
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
required = false
)
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@ApiParam(
value = "The reporting task id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable reportingTask = lookup.getRemoteProcessGroup(id);
reportingTask.authorize(authorizer, RequestAction.READ);
});
// get the reporting task
final ReportingTaskEntity reportingTask = serviceFacade.getReportingTask(id);
populateRemainingReportingTaskEntityContent(reportingTask);
@ -198,9 +200,6 @@ public class ReportingTaskResource extends ApplicationResource {
/**
* Returns the descriptor for the specified property.
*
* @param clientId Optional client id. If the client id is not specified, a
* new one will be generated. This value (whether specified or generated) is
* included in the response.
* @param id The id of the reporting task.
* @param propertyName The property
* @return a propertyDescriptorEntity
@ -229,21 +228,16 @@ public class ReportingTaskResource extends ApplicationResource {
}
)
public Response getPropertyDescriptor(
@ApiParam(
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
required = false
)
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@ApiParam(
value = "The reporting task id.",
required = true
)
@PathParam("id") String id,
@PathParam("id") final String id,
@ApiParam(
value = "The property name.",
required = true
)
@QueryParam("propertyName") String propertyName) {
@QueryParam("propertyName") final String propertyName) {
// ensure the property name is specified
if (propertyName == null) {
@ -255,6 +249,12 @@ public class ReportingTaskResource extends ApplicationResource {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable reportingTask = lookup.getRemoteProcessGroup(id);
reportingTask.authorize(authorizer, RequestAction.READ);
});
// get the property descriptor
final PropertyDescriptorDTO descriptor = serviceFacade.getReportingTaskPropertyDescriptor(id, propertyName);
@ -298,13 +298,19 @@ public class ReportingTaskResource extends ApplicationResource {
value = "The reporting task id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable reportingTask = lookup.getRemoteProcessGroup(id);
reportingTask.authorize(authorizer, RequestAction.WRITE);
});
// get the component state
final ComponentStateDTO state = serviceFacade.getReportingTaskState(id);
@ -345,16 +351,16 @@ public class ReportingTaskResource extends ApplicationResource {
}
)
public Response clearState(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The revision used to verify the client is working with the latest version of the flow.",
required = true
) ComponentStateEntity revisionEntity,
) final ComponentStateEntity revisionEntity,
@ApiParam(
value = "The reporting task id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -363,6 +369,11 @@ public class ReportingTaskResource extends ApplicationResource {
// handle expects request (usually from the cluster manager)
if (isValidationPhase(httpServletRequest)) {
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable reportingTask = lookup.getRemoteProcessGroup(id);
reportingTask.authorize(authorizer, RequestAction.WRITE);
});
serviceFacade.verifyCanClearReportingTaskState(id);
return generateContinueResponse().build();
}
@ -407,16 +418,16 @@ public class ReportingTaskResource extends ApplicationResource {
}
)
public Response updateReportingTask(
@Context HttpServletRequest httpServletRequest,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The reporting task id.",
required = true
)
@PathParam("id") String id,
@PathParam("id") final String id,
@ApiParam(
value = "The reporting task configuration details.",
required = true
) ReportingTaskEntity reportingTaskEntity) {
) final ReportingTaskEntity reportingTaskEntity) {
if (reportingTaskEntity == null || reportingTaskEntity.getComponent() == null) {
throw new IllegalArgumentException("Reporting task details must be specified.");
@ -440,27 +451,29 @@ public class ReportingTaskResource extends ApplicationResource {
// handle expects request (usually from the cluster manager)
final Revision revision = getRevision(reportingTaskEntity, id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
serviceFacade.verifyUpdateReportingTask(requestReportingTaskDTO);
return generateContinueResponse().build();
}
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable reportingTask = lookup.getRemoteProcessGroup(id);
reportingTask.authorize(authorizer, RequestAction.WRITE);
},
() -> serviceFacade.verifyUpdateReportingTask(requestReportingTaskDTO),
() -> {
// update the reporting task
final UpdateResult<ReportingTaskEntity> controllerResponse = serviceFacade.updateReportingTask(revision, requestReportingTaskDTO);
// update the reporting task
final UpdateResult<ReportingTaskEntity> controllerResponse = serviceFacade.updateReportingTask(revision, requestReportingTaskDTO);
// get the results
final ReportingTaskEntity entity = controllerResponse.getResult();
populateRemainingReportingTaskEntityContent(entity);
// get the results
final ReportingTaskEntity entity = controllerResponse.getResult();
populateRemainingReportingTaskEntityContent(entity);
if (controllerResponse.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
if (controllerResponse.isNew()) {
return clusterContext(generateCreatedResponse(URI.create(entity.getComponent().getUri()), entity)).build();
} else {
return clusterContext(generateOkResponse(entity)).build();
}
}
);
}
/**
@ -521,18 +534,20 @@ public class ReportingTaskResource extends ApplicationResource {
// handle expects request (usually from the cluster manager)
final Revision revision = new Revision(version == null ? null : version.getLong(), clientId.getClientId(), id);
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
serviceFacade.claimRevision(revision);
}
if (validationPhase) {
serviceFacade.verifyDeleteReportingTask(id);
return generateContinueResponse().build();
}
// delete the specified reporting task
final ReportingTaskEntity entity = serviceFacade.deleteReportingTask(revision, id);
return clusterContext(generateOkResponse(entity)).build();
return withWriteLock(
serviceFacade,
revision,
lookup -> {
final Authorizable reportingTask = lookup.getRemoteProcessGroup(id);
reportingTask.authorize(authorizer, RequestAction.WRITE);
},
() -> serviceFacade.verifyDeleteReportingTask(id),
() -> {
// delete the specified reporting task
final ReportingTaskEntity entity = serviceFacade.deleteReportingTask(revision, id);
return clusterContext(generateOkResponse(entity)).build();
}
);
}
// setters
@ -547,4 +562,8 @@ public class ReportingTaskResource extends ApplicationResource {
public void setProperties(NiFiProperties properties) {
this.properties = properties;
}
public void setAuthorizer(Authorizer authorizer) {
this.authorizer = authorizer;
}
}

View File

@ -21,6 +21,16 @@ import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.AccessDeniedException;
import org.apache.nifi.authorization.AuthorizationRequest;
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.ResourceFactory;
import org.apache.nifi.authorization.user.NiFiUser;
import org.apache.nifi.authorization.user.NiFiUserUtils;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
@ -49,6 +59,25 @@ public class ResourceResource extends ApplicationResource {
private NiFiServiceFacade serviceFacade;
private WebClusterManager clusterManager;
private NiFiProperties properties;
private Authorizer authorizer;
private void authorizeResource() {
final NiFiUser user = NiFiUserUtils.getNiFiUser();
final AuthorizationRequest request = new AuthorizationRequest.Builder()
.resource(ResourceFactory.getResourceResource())
.identity(user.getIdentity())
.anonymous(user.isAnonymous())
.accessAttempt(true)
.action(RequestAction.READ)
.build();
final AuthorizationResult result = authorizer.authorize(request);
if (!Result.Approved.equals(result.getResult())) {
final String message = StringUtils.isNotBlank(result.getExplanation()) ? result.getExplanation() : "Access is denied";
throw new AccessDeniedException(message);
}
}
/**
* Gets the available resources that support access/authorization policies.
@ -75,6 +104,8 @@ public class ResourceResource extends ApplicationResource {
)
public Response getResources() {
authorizeResource();
// replicate if the cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
@ -103,4 +134,8 @@ public class ResourceResource extends ApplicationResource {
public void setProperties(NiFiProperties properties) {
this.properties = properties;
}
public void setAuthorizer(Authorizer authorizer) {
this.authorizer = authorizer;
}
}

View File

@ -0,0 +1,327 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.web.api;
import com.wordnik.swagger.annotations.Api;
import com.wordnik.swagger.annotations.ApiOperation;
import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.controller.Snippet;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.api.dto.SnippetDTO;
import org.apache.nifi.web.api.entity.SnippetEntity;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
import javax.ws.rs.DELETE;
import javax.ws.rs.HttpMethod;
import javax.ws.rs.POST;
import javax.ws.rs.PUT;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import java.net.URI;
import java.util.Set;
import java.util.stream.Collectors;
/**
* RESTful endpoint for querying dataflow snippets.
*/
@Path("/snippets")
@Api(
value = "/snippets",
description = "Endpoint for accessing dataflow snippets."
)
public class SnippetResource extends ApplicationResource {
private NiFiServiceFacade serviceFacade;
private WebClusterManager clusterManager;
private NiFiProperties properties;
private Authorizer authorizer;
/**
* Populate the uri's for the specified snippet.
*
* @param entity processors
* @return dtos
*/
private SnippetEntity populateRemainingSnippetEntityContent(SnippetEntity entity) {
if (entity.getSnippet() != null) {
populateRemainingSnippetContent(entity.getSnippet());
}
return entity;
}
/**
* Populates the uri for the specified snippet.
*/
private SnippetDTO populateRemainingSnippetContent(SnippetDTO snippet) {
String snippetGroupId = snippet.getParentGroupId();
// populate the snippet href
snippet.setUri(generateResourceUri("process-groups", snippetGroupId, "snippets", snippet.getId()));
return snippet;
}
// --------
// snippets
// --------
/**
* Creates a snippet based off the specified configuration.
*
* @param httpServletRequest request
* @param snippetEntity A snippetEntity
* @return A snippetEntity
*/
@POST
@Consumes(MediaType.APPLICATION_JSON)
@Produces(MediaType.APPLICATION_JSON)
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
@ApiOperation(
value = "Creates a snippet",
response = SnippetEntity.class,
authorizations = {
@Authorization(value = "Read Only", type = "ROLE_MONITOR"),
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
@Authorization(value = "Administrator", type = "ROLE_ADMIN")
}
)
@ApiResponses(
value = {
@ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
@ApiResponse(code = 401, message = "Client could not be authenticated."),
@ApiResponse(code = 403, message = "Client is not authorized to make this request."),
@ApiResponse(code = 404, message = "The specified resource could not be found."),
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
}
)
public Response createSnippet(
@Context HttpServletRequest httpServletRequest,
@ApiParam(
value = "The snippet configuration details.",
required = true
)
final SnippetEntity snippetEntity) {
if (snippetEntity == null || snippetEntity.getSnippet() == null) {
throw new IllegalArgumentException("Snippet details must be specified.");
}
if (snippetEntity.getSnippet().getId() != null) {
throw new IllegalArgumentException("Snippet ID cannot be specified.");
}
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), snippetEntity, getHeaders()).getResponse();
}
// handle expects request (usually from the cluster manager)
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase || !isTwoPhaseRequest(httpServletRequest)) {
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final SnippetDTO snippet = snippetEntity.getSnippet();
authorizeSnippet(snippet, authorizer, lookup, RequestAction.READ);
});
}
if (validationPhase) {
return generateContinueResponse().build();
}
// set the processor id as appropriate
snippetEntity.getSnippet().setId(generateUuid());
// create the snippet
final SnippetEntity entity = serviceFacade.createSnippet(snippetEntity.getSnippet());
populateRemainingSnippetEntityContent(entity);
// build the response
return clusterContext(generateCreatedResponse(URI.create(entity.getSnippet().getUri()), entity)).build();
}
/**
* Updates the specified snippet. The contents of the snippet (component
* ids) cannot be updated once the snippet is created.
*
* @param httpServletRequest request
* @param snippetId The id of the snippet.
* @param snippetEntity A snippetEntity
* @return A snippetEntity
*/
@PUT
@Consumes(MediaType.APPLICATION_JSON)
@Produces(MediaType.APPLICATION_JSON)
@Path("{id}")
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
@ApiOperation(
value = "Updates a snippet",
response = SnippetEntity.class,
authorizations = {
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
}
)
@ApiResponses(
value = {
@ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
@ApiResponse(code = 401, message = "Client could not be authenticated."),
@ApiResponse(code = 403, message = "Client is not authorized to make this request."),
@ApiResponse(code = 404, message = "The specified resource could not be found."),
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
}
)
public Response updateSnippet(
@Context HttpServletRequest httpServletRequest,
@ApiParam(
value = "The snippet id.",
required = true
)
@PathParam("id") String snippetId,
@ApiParam(
value = "The snippet configuration details.",
required = true
) final SnippetEntity snippetEntity) {
if (snippetEntity == null || snippetEntity.getSnippet() == null) {
throw new IllegalArgumentException("Snippet details must be specified.");
}
// ensure the ids are the same
final SnippetDTO requestSnippetDTO = snippetEntity.getSnippet();
if (!snippetId.equals(requestSnippetDTO.getId())) {
throw new IllegalArgumentException(String.format("The snippet id (%s) in the request body does not equal the "
+ "snippet id of the requested resource (%s).", requestSnippetDTO.getId(), snippetId));
}
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), snippetEntity, getHeaders()).getResponse();
}
// get the revision from this snippet
final Set<Revision> revisions = serviceFacade.getRevisionsFromSnippet(snippetId);
return withWriteLock(
serviceFacade,
revisions,
lookup -> {
// ensure write access to the target process group
if (requestSnippetDTO.getParentGroupId() != null) {
lookup.getProcessGroup(requestSnippetDTO.getParentGroupId()).authorize(authorizer, RequestAction.WRITE);
}
// ensure read permission to every component in the snippet
final Snippet snippet = lookup.getSnippet(snippetId);
authorizeSnippet(snippet, authorizer, lookup, RequestAction.WRITE);
},
() -> serviceFacade.verifyUpdateSnippet(requestSnippetDTO, revisions.stream().map(rev -> rev.getComponentId()).collect(Collectors.toSet())),
() -> {
// update the snippet
final SnippetEntity entity = serviceFacade.updateSnippet(revisions, snippetEntity.getSnippet());
populateRemainingSnippetEntityContent(entity);
return clusterContext(generateOkResponse(entity)).build();
}
);
}
/**
* Removes the specified snippet.
*
* @param httpServletRequest request
* @param snippetId The id of the snippet to remove.
* @return A entity containing the client id and an updated revision.
*/
@DELETE
@Consumes(MediaType.WILDCARD)
@Produces(MediaType.APPLICATION_JSON)
@Path("{id}")
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
@ApiOperation(
value = "Deletes the components in a snippet and drops the snippet",
response = SnippetEntity.class,
authorizations = {
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
}
)
@ApiResponses(
value = {
@ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
@ApiResponse(code = 401, message = "Client could not be authenticated."),
@ApiResponse(code = 403, message = "Client is not authorized to make this request."),
@ApiResponse(code = 404, message = "The specified resource could not be found."),
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
}
)
public Response deleteSnippet(
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The snippet id.",
required = true
)
@PathParam("id") final String snippetId) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.DELETE, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// get the revision from this snippet
final Set<Revision> revisions = serviceFacade.getRevisionsFromSnippet(snippetId);
return withWriteLock(
serviceFacade,
revisions,
lookup -> {
// ensure read permission to every component in the snippet
final Snippet snippet = lookup.getSnippet(snippetId);
authorizeSnippet(snippet, authorizer, lookup, RequestAction.WRITE);
},
() -> serviceFacade.verifyDeleteSnippet(snippetId, revisions.stream().map(rev -> rev.getComponentId()).collect(Collectors.toSet())),
() -> {
// delete the specified snippet
final SnippetEntity snippetEntity = serviceFacade.deleteSnippet(revisions, snippetId);
return clusterContext(generateOkResponse(snippetEntity)).build();
}
);
}
/* setters */
public void setServiceFacade(NiFiServiceFacade serviceFacade) {
this.serviceFacade = serviceFacade;
}
public void setClusterManager(WebClusterManager clusterManager) {
this.clusterManager = clusterManager;
}
public void setProperties(NiFiProperties properties) {
this.properties = properties;
}
public void setAuthorizer(Authorizer authorizer) {
this.authorizer = authorizer;
}
}

View File

@ -22,7 +22,16 @@ import com.wordnik.swagger.annotations.ApiParam;
import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.AccessDeniedException;
import org.apache.nifi.authorization.AuthorizationRequest;
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.ResourceFactory;
import org.apache.nifi.authorization.user.NiFiUser;
import org.apache.nifi.authorization.user.NiFiUserUtils;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
@ -60,6 +69,24 @@ public class SystemDiagnosticsResource extends ApplicationResource {
private NiFiProperties properties;
private Authorizer authorizer;
private void authorizeSystem() {
final NiFiUser user = NiFiUserUtils.getNiFiUser();
final AuthorizationRequest request = new AuthorizationRequest.Builder()
.resource(ResourceFactory.getSystemResource())
.identity(user.getIdentity())
.anonymous(user.isAnonymous())
.accessAttempt(true)
.action(RequestAction.READ)
.build();
final AuthorizationResult result = authorizer.authorize(request);
if (!Result.Approved.equals(result.getResult())) {
final String message = StringUtils.isNotBlank(result.getExplanation()) ? result.getExplanation() : "Access is denied";
throw new AccessDeniedException(message);
}
}
/**
* Gets the system diagnostics for this NiFi instance.
*
@ -88,12 +115,14 @@ public class SystemDiagnosticsResource extends ApplicationResource {
value = "Whether or not to include the breakdown per node. Optional, defaults to false",
required = false
)
@QueryParam("nodewise") @DefaultValue(NODEWISE) Boolean nodewise,
@QueryParam("nodewise") @DefaultValue(NODEWISE) final Boolean nodewise,
@ApiParam(
value = "The id of the node where to get the status.",
required = false
)
@QueryParam("clusterNodeId") String clusterNodeId) {
@QueryParam("clusterNodeId") final String clusterNodeId) {
authorizeSystem();
// ensure a valid request
if (Boolean.TRUE.equals(nodewise) && clusterNodeId != null) {

View File

@ -23,25 +23,25 @@ import com.wordnik.swagger.annotations.ApiResponse;
import com.wordnik.swagger.annotations.ApiResponses;
import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.api.dto.TemplateDTO;
import org.apache.nifi.web.api.entity.TemplateEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
import javax.ws.rs.DELETE;
import javax.ws.rs.DefaultValue;
import javax.ws.rs.GET;
import javax.ws.rs.HttpMethod;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
@ -62,6 +62,7 @@ public class TemplateResource extends ApplicationResource {
private NiFiServiceFacade serviceFacade;
private WebClusterManager clusterManager;
private NiFiProperties properties;
private Authorizer authorizer;
/**
* Populates the uri for the specified templates.
@ -119,13 +120,19 @@ public class TemplateResource extends ApplicationResource {
value = "The template id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable template = lookup.getTemplate(id);
template.authorize(authorizer, RequestAction.READ);
});
// get the template
final TemplateDTO template = serviceFacade.exportTemplate(id);
@ -148,9 +155,6 @@ public class TemplateResource extends ApplicationResource {
* Removes the specified template.
*
* @param httpServletRequest request
* @param clientId Optional client id. If the client id is not specified, a
* new one will be generated. This value (whether specified or generated) is
* included in the response.
* @param id The id of the template to remove.
* @return A templateEntity.
*/
@ -176,17 +180,12 @@ public class TemplateResource extends ApplicationResource {
}
)
public Response removeTemplate(
@Context HttpServletRequest httpServletRequest,
@ApiParam(
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
required = false
)
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@Context final HttpServletRequest httpServletRequest,
@ApiParam(
value = "The template id.",
required = true
)
@PathParam("id") String id) {
@PathParam("id") final String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
@ -194,9 +193,13 @@ public class TemplateResource extends ApplicationResource {
}
// handle expects request (usually from the cluster manager)
// TODO: NEED VERSION FOR REVISION!
final boolean validationPhase = isValidationPhase(httpServletRequest);
if (validationPhase) {
// authorize access
serviceFacade.authorizeAccess(lookup -> {
final Authorizable template = lookup.getTemplate(id);
template.authorize(authorizer, RequestAction.WRITE);
});
return generateContinueResponse().build();
}
@ -221,4 +224,8 @@ public class TemplateResource extends ApplicationResource {
public void setProperties(NiFiProperties properties) {
this.properties = properties;
}
public void setAuthorizer(Authorizer authorizer) {
this.authorizer = authorizer;
}
}

View File

@ -133,6 +133,7 @@ import org.apache.nifi.web.api.dto.status.ProcessorStatusDTO;
import org.apache.nifi.web.api.dto.status.ProcessorStatusSnapshotDTO;
import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusDTO;
import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusSnapshotDTO;
import org.apache.nifi.web.api.entity.FlowBreadcrumbEntity;
import org.apache.nifi.web.revision.RevisionManager;
import javax.ws.rs.WebApplicationException;
@ -681,7 +682,6 @@ public final class DtoFactory {
final SnippetDTO dto = new SnippetDTO();
dto.setId(snippet.getId());
dto.setParentGroupId(snippet.getParentGroupId());
dto.setLinked(snippet.isLinked());
// populate the snippet contents ids
dto.setConnections(mapRevisionToDto(snippet.getConnections()));
@ -1398,27 +1398,51 @@ public final class DtoFactory {
}
/**
* Creates a FlowBreadcrumbDTO from the specified parent ProcessGroup.
* Creates a FlowBreadcrumbEntity from the specified parent ProcessGroup.
*
* @param parentGroup group
* @param group group
* @return dto
*/
private FlowBreadcrumbDTO createBreadcrumbDto(final ProcessGroup parentGroup) {
if (parentGroup == null) {
private FlowBreadcrumbEntity createBreadcrumbEntity(final ProcessGroup group) {
if (group == null) {
return null;
}
final FlowBreadcrumbDTO dto = createBreadcrumbDto(group);
final AccessPolicyDTO accessPolicy = createAccessPolicyDto(group);
final FlowBreadcrumbEntity entity = entityFactory.createFlowBreadcrumbEntity(dto, accessPolicy);
if (group.getParent() != null) {
entity.setParentBreadcrumb(createBreadcrumbEntity(group.getParent()));
}
return entity;
}
/**
* Creates a FlowBreadcrumbDTO from the specified parent ProcessGroup.
*
* @param group group
* @return dto
*/
private FlowBreadcrumbDTO createBreadcrumbDto(final ProcessGroup group) {
if (group == null) {
return null;
}
final FlowBreadcrumbDTO dto = new FlowBreadcrumbDTO();
dto.setId(parentGroup.getIdentifier());
dto.setName(parentGroup.getName());
if (parentGroup.getParent() != null) {
dto.setParentBreadcrumb(createBreadcrumbDto(parentGroup.getParent()));
}
dto.setId(group.getIdentifier());
dto.setName(group.getName());
return dto;
}
/**
* Creates the AccessPolicyDTO based on the specified Authorizable.
*
* @param authorizable authorizable
* @return dto
*/
public AccessPolicyDTO createAccessPolicyDto(final Authorizable authorizable) {
final AccessPolicyDTO dto = new AccessPolicyDTO();
dto.setCanRead(authorizable.isAuthorized(authorizer, RequestAction.READ));
@ -1440,7 +1464,7 @@ public final class DtoFactory {
final ProcessGroupFlowDTO dto = new ProcessGroupFlowDTO();
dto.setId(group.getIdentifier());
dto.setLastRefreshed(new Date());
dto.setBreadcrumb(createBreadcrumbDto(group));
dto.setBreadcrumb(createBreadcrumbEntity(group));
dto.setFlow(createFlowDto(group, groupStatus, revisionManager));
final ProcessGroup parent = group.getParent();
@ -1468,11 +1492,6 @@ public final class DtoFactory {
flow.getConnections().add(entityFactory.createConnectionEntity(connection, null, accessPolicy, status));
}
for (final ControllerServiceDTO controllerService : snippet.getControllerServices()) {
final RevisionDTO revision = createRevisionDTO(revisionManager.getRevision(controllerService.getId()));
flow.getControllerServices().add(entityFactory.createControllerServiceEntity(controllerService, revision, null));
}
for (final FunnelDTO funnel : snippet.getFunnels()) {
final RevisionDTO revision = createRevisionDTO(revisionManager.getRevision(funnel.getId()));
final AccessPolicyDTO accessPolicy = createAccessPolicyDto(group.getFunnel(funnel.getId()));
@ -1627,13 +1646,6 @@ public final class DtoFactory {
dto.getOutputPorts().add(entityFactory.createPortEntity(createPortDto(outputPort), revision, accessPolicy, status));
}
// TODO - controller services once they are accessible from the group
for (final ControllerServiceNode controllerService : group.getControllerServices(false)) {
final RevisionDTO revision = createRevisionDTO(revisionManager.getRevision(controllerService.getIdentifier()));
final AccessPolicyDTO accessPolicy = createAccessPolicyDto(controllerService);
dto.getControllerServices().add(entityFactory.createControllerServiceEntity(createControllerServiceDto(controllerService), revision, accessPolicy));
}
return dto;
}
@ -2522,7 +2534,6 @@ public final class DtoFactory {
copy.setOutputPortCount(original.getOutputPortCount());
copy.setParentGroupId(original.getParentGroupId());
copy.setRunning(original.isRunning());
copy.setRunningCount(original.getRunningCount());
copy.setStoppedCount(original.getStoppedCount());
copy.setDisabledCount(original.getDisabledCount());

View File

@ -16,6 +16,8 @@
*/
package org.apache.nifi.web.api.dto;
import org.apache.nifi.web.api.dto.flow.FlowBreadcrumbDTO;
import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO;
import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO;
import org.apache.nifi.web.api.dto.status.PortStatusDTO;
import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO;
@ -24,10 +26,12 @@ import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusDTO;
import org.apache.nifi.web.api.entity.ConnectionEntity;
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentEntity;
import org.apache.nifi.web.api.entity.FlowBreadcrumbEntity;
import org.apache.nifi.web.api.entity.FunnelEntity;
import org.apache.nifi.web.api.entity.LabelEntity;
import org.apache.nifi.web.api.entity.PortEntity;
import org.apache.nifi.web.api.entity.ProcessGroupEntity;
import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity;
import org.apache.nifi.web.api.entity.ProcessorEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupPortEntity;
@ -36,6 +40,13 @@ import org.apache.nifi.web.api.entity.SnippetEntity;
public final class EntityFactory {
public ProcessGroupFlowEntity createProcessGroupFlowEntity(final ProcessGroupFlowDTO dto, final AccessPolicyDTO accessPolicy) {
final ProcessGroupFlowEntity entity = new ProcessGroupFlowEntity();
entity.setProcessGroupFlow(dto);
entity.setAccessPolicy(accessPolicy);
return entity;
}
public ProcessorEntity createProcessorEntity(final ProcessorDTO dto, final RevisionDTO revision, final AccessPolicyDTO accessPolicy, final ProcessorStatusDTO status) {
final ProcessorEntity entity = new ProcessorEntity();
entity.setRevision(revision);
@ -178,17 +189,9 @@ public final class EntityFactory {
return entity;
}
public SnippetEntity createSnippetEntity(final SnippetDTO dto, final RevisionDTO revision, final AccessPolicyDTO accessPolicy) {
public SnippetEntity createSnippetEntity(final SnippetDTO dto) {
final SnippetEntity entity = new SnippetEntity();
entity.setRevision(revision);
if (dto != null) {
entity.setAccessPolicy(accessPolicy);
entity.setId(dto.getId());
if (accessPolicy != null && accessPolicy.getCanRead()) {
entity.setSnippet(dto);
}
}
entity.setSnippet(dto);
return entity;
}
@ -235,4 +238,16 @@ public final class EntityFactory {
return entity;
}
public FlowBreadcrumbEntity createFlowBreadcrumbEntity(final FlowBreadcrumbDTO dto, final AccessPolicyDTO accessPolicy) {
final FlowBreadcrumbEntity entity = new FlowBreadcrumbEntity();
if (dto != null) {
entity.setAccessPolicy(accessPolicy);
entity.setId(dto.getId());
if (accessPolicy != null && accessPolicy.getCanRead()) {
entity.setBreadcrumb(dto);
}
}
return entity;
}
}

View File

@ -16,6 +16,7 @@
*/
package org.apache.nifi.web.dao;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.web.api.dto.ProcessGroupDTO;
@ -58,9 +59,18 @@ public interface ProcessGroupDAO {
/**
* Verifies the specified process group can be modified.
*
* @param processGroupDTO dto
* @param groupId id
* @param state scheduled state
*/
void verifyUpdate(ProcessGroupDTO processGroupDTO);
void verifyScheduleComponents(String groupId, ScheduledState state, Set<String> componentIds);
/**
* Schedules the components in the specified process group.
*
* @param groupId id
* @param state scheduled state
*/
void scheduleComponents(String groupId, ScheduledState state, Set<String> componentIds);
/**
* Updates the specified process group.

View File

@ -50,6 +50,13 @@ public interface SnippetDAO {
*/
boolean hasSnippet(String snippetId);
/**
* Drops the specified snippet.
*
* @param snippetId snippet id
*/
void dropSnippet(String snippetId);
/**
* Gets the specified snippet.
*
@ -59,31 +66,31 @@ public interface SnippetDAO {
Snippet getSnippet(String snippetId);
/**
* Verifies the specified snippet can be updated.
* Verifies the components of the specified snippet can be updated.
*
* @param snippetDTO snippet
*/
void verifyUpdate(SnippetDTO snippetDTO);
void verifyUpdateSnippetComponent(SnippetDTO snippetDTO);
/**
* Updates the specified snippet.
* Updates the components in the specified snippet.
*
* @param snippetDTO snippet
* @return The snippet
*/
Snippet updateSnippet(SnippetDTO snippetDTO);
Snippet updateSnippetComponents(SnippetDTO snippetDTO);
/**
* Verifies the specified snippet can be removed.
* Verifies the components of the specified snippet can be removed.
*
* @param snippetId snippet id
*/
void verifyDelete(String snippetId);
void verifyDeleteSnippetComponents(String snippetId);
/**
* Deletes the specified snippet.
* Deletes the components in the specified snippet.
*
* @param snippetId The snippet id
*/
void deleteSnippet(String snippetId);
void deleteSnippetComponents(String snippetId);
}

View File

@ -16,12 +16,19 @@
*/
package org.apache.nifi.web.dao.impl;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.ConnectableType;
import org.apache.nifi.connectable.Port;
import org.apache.nifi.connectable.Position;
import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.web.ResourceNotFoundException;
import org.apache.nifi.web.api.dto.ProcessGroupDTO;
import org.apache.nifi.web.dao.ProcessGroupDAO;
import java.util.HashSet;
import java.util.Set;
public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGroupDAO {
@ -68,15 +75,51 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
}
@Override
public void verifyUpdate(ProcessGroupDTO processGroupDTO) {
final ProcessGroup group = locateProcessGroup(flowController, processGroupDTO.getId());
public void verifyScheduleComponents(final String groupId, final ScheduledState state,final Set<String> componentIds) {
final ProcessGroup group = locateProcessGroup(flowController, groupId);
// determine if any action is required
if (isNotNull(processGroupDTO.isRunning())) {
if (processGroupDTO.isRunning()) {
group.verifyCanStart();
final Set<Connectable> connectables = new HashSet<>(componentIds.size());
for (final String componentId : componentIds) {
final Connectable connectable = group.findConnectable(componentId);
if (connectable == null) {
throw new ResourceNotFoundException("Unable to find component with id " + componentId);
}
connectables.add(connectable);
}
// verify as appropriate
connectables.forEach(connectable -> {
if (ScheduledState.RUNNING.equals(state)) {
group.verifyCanStart(connectable);
} else {
group.verifyCanStop();
group.verifyCanStop(connectable);
}
});
}
@Override
public void scheduleComponents(final String groupId, final ScheduledState state, final Set<String> componentIds) {
final ProcessGroup group = locateProcessGroup(flowController, groupId);
for (final String componentId : componentIds) {
final Connectable connectable = group.findConnectable(componentId);
if (ScheduledState.RUNNING.equals(state)) {
if (ConnectableType.PROCESSOR.equals(connectable.getConnectableType())) {
group.startProcessor((ProcessorNode) connectable);
} else if (ConnectableType.INPUT_PORT.equals(connectable.getConnectableType())) {
group.startInputPort((Port) connectable);
} else if (ConnectableType.OUTPUT_PORT.equals(connectable.getConnectableType())) {
group.startOutputPort((Port) connectable);
}
} else {
if (ConnectableType.PROCESSOR.equals(connectable.getConnectableType())) {
group.stopProcessor((ProcessorNode) connectable);
} else if (ConnectableType.INPUT_PORT.equals(connectable.getConnectableType())) {
group.stopInputPort((Port) connectable);
} else if (ConnectableType.OUTPUT_PORT.equals(connectable.getConnectableType())) {
group.stopOutputPort((Port) connectable);
}
}
}
}
@ -98,15 +141,6 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
group.setComments(comments);
}
// determine if any action is required
if (isNotNull(processGroupDTO.isRunning())) {
if (processGroupDTO.isRunning()) {
group.startProcessing();
} else {
group.stopProcessing();
}
}
return group;
}

View File

@ -16,11 +16,7 @@
*/
package org.apache.nifi.web.dao.impl;
import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.ProcessorNode;
@ -41,7 +37,11 @@ import org.apache.nifi.web.api.dto.RevisionDTO;
import org.apache.nifi.web.api.dto.SnippetDTO;
import org.apache.nifi.web.dao.SnippetDAO;
import org.apache.nifi.web.util.SnippetUtils;
import org.apache.commons.lang3.StringUtils;
import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
public class StandardSnippetDAO implements SnippetDAO {
@ -119,7 +119,6 @@ public class StandardSnippetDAO implements SnippetDAO {
final StandardSnippet snippet = new StandardSnippet();
snippet.setId(snippetDTO.getId());
snippet.setParentGroupId(snippetDTO.getParentGroupId());
snippet.setLinked(snippetDTO.isLinked());
snippet.addProcessors(mapDtoToRevision(snippetDTO.getProcessors()));
snippet.addProcessGroups(mapDtoToRevision(snippetDTO.getProcessGroups()));
snippet.addRemoteProcessGroups(mapDtoToRevision(snippetDTO.getRemoteProcessGroups()));
@ -146,39 +145,31 @@ public class StandardSnippetDAO implements SnippetDAO {
}
@Override
public void verifyDelete(String snippetId) {
final StandardSnippet snippet = locateSnippet(snippetId);
public void verifyDeleteSnippetComponents(String snippetId) {
final Snippet snippet = locateSnippet(snippetId);
// only need to check if the snippet is linked
if (snippet.isLinked()) {
// ensure the parent group exist
final ProcessGroup processGroup = flowController.getGroup(snippet.getParentGroupId());
if (processGroup == null) {
throw new IllegalArgumentException("The specified parent process group could not be found.");
}
// verify the processGroup can remove the snippet
processGroup.verifyCanDelete(snippet);
// ensure the parent group exist
final ProcessGroup processGroup = flowController.getGroup(snippet.getParentGroupId());
if (processGroup == null) {
throw new IllegalArgumentException("The specified parent process group could not be found.");
}
// verify the processGroup can remove the snippet
processGroup.verifyCanDelete(snippet);
}
@Override
public void deleteSnippet(String snippetId) {
final StandardSnippet snippet = locateSnippet(snippetId);
public void deleteSnippetComponents(String snippetId) {
final Snippet snippet = locateSnippet(snippetId);
// if the snippet is linked, remove the contents
if (snippet.isLinked()) {
final ProcessGroup processGroup = flowController.getGroup(snippet.getParentGroupId());
if (processGroup == null) {
throw new IllegalArgumentException("The specified parent process group could not be found.");
}
// remove the underlying components
processGroup.remove(snippet);
// remove the contents
final ProcessGroup processGroup = flowController.getGroup(snippet.getParentGroupId());
if (processGroup == null) {
throw new IllegalArgumentException("The specified parent process group could not be found.");
}
// delete the snippet itself
flowController.getSnippetManager().removeSnippet(snippet);
// remove the underlying components
processGroup.remove(snippet);
}
@Override
@ -192,11 +183,18 @@ public class StandardSnippetDAO implements SnippetDAO {
}
@Override
public void verifyUpdate(SnippetDTO snippetDTO) {
final StandardSnippet snippet = locateSnippet(snippetDTO.getId());
public void dropSnippet(String snippetId) {
// drop the snippet itself
final StandardSnippet snippet = locateSnippet(snippetId);
flowController.getSnippetManager().removeSnippet(snippet);
}
// if attempting to move the snippet contents
if (snippetDTO.getParentGroupId() != null) {
@Override
public void verifyUpdateSnippetComponent(SnippetDTO snippetDTO) {
final Snippet snippet = locateSnippet(snippetDTO.getId());
// if the group is changing move it
if (snippetDTO.getParentGroupId() != null && snippet.getParentGroupId() != snippetDTO.getParentGroupId()) {
// get the current process group
final ProcessGroup processGroup = flowController.getGroup(snippet.getParentGroupId());
if (processGroup == null) {
@ -209,35 +207,17 @@ public class StandardSnippetDAO implements SnippetDAO {
throw new IllegalArgumentException("The new process group could not be found.");
}
boolean verificationRequired = false;
// verify if necessary
if (snippetDTO.isLinked() != null) {
if (snippetDTO.isLinked()) {
verificationRequired = true;
}
} else if (snippet.isLinked()) {
verificationRequired = true;
}
// perform the verification if necessary
if (verificationRequired) {
processGroup.verifyCanMove(snippet, newProcessGroup);
}
// perform the verification
processGroup.verifyCanMove(snippet, newProcessGroup);
}
}
@Override
public Snippet updateSnippet(final SnippetDTO snippetDTO) {
public Snippet updateSnippetComponents(final SnippetDTO snippetDTO) {
final StandardSnippet snippet = locateSnippet(snippetDTO.getId());
// update whether this snippet is linked to the data flow
if (snippetDTO.isLinked() != null) {
snippet.setLinked(snippetDTO.isLinked());
}
// if the group is changing and its linked to the data flow move it
if (snippetDTO.getParentGroupId() != null && snippet.isLinked()) {
// if the group is changing move it
if (snippetDTO.getParentGroupId() != null && snippet.getParentGroupId() != snippetDTO.getParentGroupId()) {
final ProcessGroup currentProcessGroup = flowController.getGroup(snippet.getParentGroupId());
if (currentProcessGroup == null) {
throw new IllegalArgumentException("The current process group could not be found.");

View File

@ -183,6 +183,7 @@
<property name="serviceFacade" ref="serviceFacade"/>
<property name="properties" ref="nifiProperties"/>
<property name="clusterManager" ref="clusterManager"/>
<property name="authorizer" ref="authorizer"/>
</bean>
<bean id="controllerResource" class="org.apache.nifi.web.api.ControllerResource" scope="singleton">
<property name="serviceFacade" ref="serviceFacade"/>
@ -195,20 +196,29 @@
<property name="properties" ref="nifiProperties"/>
<property name="clusterManager" ref="clusterManager"/>
</bean>
<bean id="snippetResource" class="org.apache.nifi.web.api.SnippetResource" scope="singleton">
<property name="serviceFacade" ref="serviceFacade"/>
<property name="properties" ref="nifiProperties"/>
<property name="clusterManager" ref="clusterManager"/>
<property name="authorizer" ref="authorizer"/>
</bean>
<bean id="templateResource" class="org.apache.nifi.web.api.TemplateResource" scope="singleton">
<property name="serviceFacade" ref="serviceFacade"/>
<property name="properties" ref="nifiProperties"/>
<property name="clusterManager" ref="clusterManager"/>
<property name="authorizer" ref="authorizer"/>
</bean>
<bean id="controllerServiceResource" class="org.apache.nifi.web.api.ControllerServiceResource" scope="singleton">
<property name="serviceFacade" ref="serviceFacade"/>
<property name="properties" ref="nifiProperties"/>
<property name="clusterManager" ref="clusterManager"/>
<property name="authorizer" ref="authorizer"/>
</bean>
<bean id="reportingTaskResource" class="org.apache.nifi.web.api.ReportingTaskResource" scope="singleton">
<property name="serviceFacade" ref="serviceFacade"/>
<property name="properties" ref="nifiProperties"/>
<property name="clusterManager" ref="clusterManager"/>
<property name="authorizer" ref="authorizer"/>
</bean>
<bean id="processGroupResource" class="org.apache.nifi.web.api.ProcessGroupResource" scope="singleton">
<property name="serviceFacade" ref="serviceFacade"/>
@ -223,41 +233,49 @@
<property name="connectionResource" ref="connectionResource"/>
<property name="templateResource" ref="templateResource"/>
<property name="controllerServiceResource" ref="controllerServiceResource"/>
<property name="authorizer" ref="authorizer"/>
</bean>
<bean id="processorResource" class="org.apache.nifi.web.api.ProcessorResource" scope="singleton">
<property name="serviceFacade" ref="serviceFacade"/>
<property name="properties" ref="nifiProperties"/>
<property name="clusterManager" ref="clusterManager"/>
<property name="authorizer" ref="authorizer"/>
</bean>
<bean id="connectionResource" class="org.apache.nifi.web.api.ConnectionResource" scope="singleton">
<property name="serviceFacade" ref="serviceFacade"/>
<property name="properties" ref="nifiProperties"/>
<property name="clusterManager" ref="clusterManager"/>
<property name="authorizer" ref="authorizer"/>
</bean>
<bean id="flowfileQueueResource" class="org.apache.nifi.web.api.FlowFileQueueResource" scope="singleton">
<property name="serviceFacade" ref="serviceFacade"/>
<property name="properties" ref="nifiProperties"/>
<property name="clusterManager" ref="clusterManager"/>
<property name="authorizer" ref="authorizer"/>
</bean>
<bean id="remoteProcessGroupResource" class="org.apache.nifi.web.api.RemoteProcessGroupResource" scope="singleton">
<property name="serviceFacade" ref="serviceFacade"/>
<property name="properties" ref="nifiProperties"/>
<property name="clusterManager" ref="clusterManager"/>
<property name="authorizer" ref="authorizer"/>
</bean>
<bean id="inputPortResource" class="org.apache.nifi.web.api.InputPortResource" scope="singleton">
<property name="serviceFacade" ref="serviceFacade"/>
<property name="properties" ref="nifiProperties"/>
<property name="clusterManager" ref="clusterManager"/>
<property name="authorizer" ref="authorizer"/>
</bean>
<bean id="outputPortResource" class="org.apache.nifi.web.api.OutputPortResource" scope="singleton">
<property name="serviceFacade" ref="serviceFacade"/>
<property name="properties" ref="nifiProperties"/>
<property name="clusterManager" ref="clusterManager"/>
<property name="authorizer" ref="authorizer"/>
</bean>
<bean id="labelResource" class="org.apache.nifi.web.api.LabelResource" scope="singleton">
<property name="serviceFacade" ref="serviceFacade"/>
<property name="properties" ref="nifiProperties"/>
<property name="clusterManager" ref="clusterManager"/>
<property name="authorizer" ref="authorizer"/>
</bean>
<bean id="funnelResource" class="org.apache.nifi.web.api.FunnelResource" scope="singleton">
<property name="serviceFacade" ref="serviceFacade"/>
@ -284,6 +302,7 @@
<property name="serviceFacade" ref="serviceFacade"/>
<property name="properties" ref="nifiProperties"/>
<property name="clusterManager" ref="clusterManager"/>
<property name="authorizer" ref="authorizer"/>
</bean>
<bean id="accessResource" class="org.apache.nifi.web.api.AccessResource" scope="singleton">
<property name="properties" ref="nifiProperties"/>

View File

@ -361,7 +361,6 @@ public class DfmAccessControlTest {
// get a processor and update its configuration state
ProcessGroupDTO processGroup = getRandomProcessGroup();
processGroup.setRunning(Boolean.FALSE);
// create the revision
final RevisionDTO revision = new RevisionDTO();

View File

@ -550,16 +550,16 @@ public class NaiveRevisionManager implements RevisionManager {
public boolean releaseClaimIfCurrentThread(final Revision revision) {
threadLock.writeLock().lock();
try {
if (revision != null && !getRevision().equals(revision)) {
throw new InvalidRevisionException("Cannot release claim because the provided Revision is not valid");
}
final LockStamp stamp = lockStamp.get();
if (stamp == null) {
logger.debug("Cannot cancel claim for {} because there is no claim held", getRevision());
return false;
}
if (revision != null && !getRevision().equals(revision)) {
throw new InvalidRevisionException("Cannot release claim because the provided Revision is not valid");
}
if (stamp.isObtainedByCurrentThread()) {
releaseClaim();
logger.debug("Successfully canceled claim for {}", getRevision());

View File

@ -21,34 +21,42 @@
<div id="component-container">
<button title="{{appCtrl.serviceProvider.headerCtrl.toolboxCtrl.config.type.processor}}"
id="processor-component" class="component-button"
ng-disabled="!appCtrl.nf.Canvas.canWrite();"
nf-draggable="appCtrl.serviceProvider.headerCtrl.toolboxCtrl.draggableComponentConfig(appCtrl.serviceProvider.headerCtrl.toolboxCtrl.processorComponent);">
<i class="icon icon-processor"></i></button>
<button title="{{appCtrl.serviceProvider.headerCtrl.toolboxCtrl.config.type.inputPort}}"
id="port-in-component" class="component-button"
ng-disabled="!appCtrl.nf.Canvas.canWrite();"
nf-draggable="appCtrl.serviceProvider.headerCtrl.toolboxCtrl.draggableComponentConfig(appCtrl.serviceProvider.headerCtrl.toolboxCtrl.inputPortComponent);">
<i class="icon icon-port-in"></i></button>
<button title="{{appCtrl.serviceProvider.headerCtrl.toolboxCtrl.config.type.outputPort}}"
id="port-out-component" class="component-button"
ng-disabled="!appCtrl.nf.Canvas.canWrite();"
nf-draggable="appCtrl.serviceProvider.headerCtrl.toolboxCtrl.draggableComponentConfig(appCtrl.serviceProvider.headerCtrl.toolboxCtrl.outputPortComponent);">
<i class="icon icon-port-out"></i></button>
<button title="{{appCtrl.serviceProvider.headerCtrl.toolboxCtrl.config.type.processGroup}}"
id="group-component" class="component-button"
ng-disabled="!appCtrl.nf.Canvas.canWrite();"
nf-draggable="appCtrl.serviceProvider.headerCtrl.toolboxCtrl.draggableComponentConfig(appCtrl.serviceProvider.headerCtrl.toolboxCtrl.groupComponent);">
<i class="icon icon-group"></i></button>
<button title="{{appCtrl.serviceProvider.headerCtrl.toolboxCtrl.config.type.remoteProcessGroup}}" hide
show-gt-sm id="group-remote-component" class="component-button"
ng-disabled="!appCtrl.nf.Canvas.canWrite();"
nf-draggable="appCtrl.serviceProvider.headerCtrl.toolboxCtrl.draggableComponentConfig(appCtrl.serviceProvider.headerCtrl.toolboxCtrl.remoteGroupComponent);">
<i class="icon icon-group-remote"></i></button>
<button title="{{appCtrl.serviceProvider.headerCtrl.toolboxCtrl.config.type.funnel}}"
id="funnel-component" class="component-button"
ng-disabled="!appCtrl.nf.Canvas.canWrite();"
nf-draggable="appCtrl.serviceProvider.headerCtrl.toolboxCtrl.draggableComponentConfig(appCtrl.serviceProvider.headerCtrl.toolboxCtrl.funnelComponent);">
<i class="icon icon-funnel"></i></button>
<button title="{{appCtrl.serviceProvider.headerCtrl.toolboxCtrl.config.type.template}}"
id="template-component" class="component-button"
ng-disabled="!appCtrl.nf.Canvas.canWrite();"
nf-draggable="appCtrl.serviceProvider.headerCtrl.toolboxCtrl.draggableComponentConfig(appCtrl.serviceProvider.headerCtrl.toolboxCtrl.templateComponent);">
<i class="icon icon-template"></i></button>
<button title="{{appCtrl.serviceProvider.headerCtrl.toolboxCtrl.config.type.label}}"
id="label-component" class="component-button"
ng-disabled="!appCtrl.nf.Canvas.canWrite();"
nf-draggable="appCtrl.serviceProvider.headerCtrl.toolboxCtrl.draggableComponentConfig(appCtrl.serviceProvider.headerCtrl.toolboxCtrl.labelComponent);">
<i class="icon icon-label"></i></button>
</div>

View File

@ -85,43 +85,43 @@
<div>
<div id="operate-enable" class="action-button" title="Enable">
<button ng-click="appCtrl.nf.Actions['enable'](appCtrl.nf.CanvasUtils.getSelection());"
ng-disabled="appCtrl.nf.CanvasUtils.getSelection().empty() || !appCtrl.nf.CanvasUtils.canEnable(appCtrl.nf.CanvasUtils.getSelection());">
ng-disabled="!appCtrl.nf.CanvasUtils.canEnable(appCtrl.nf.CanvasUtils.getSelection());">
<i class="graph-control-action-icon fa fa-flash"></i></button>
</div>
<div class="button-spacer-small">&nbsp;</div>
<div id="operate-disable" class="action-button" title="Disable">
<button ng-click="appCtrl.nf.Actions['disable'](appCtrl.nf.CanvasUtils.getSelection());"
ng-disabled="appCtrl.nf.CanvasUtils.getSelection().empty() || !appCtrl.nf.CanvasUtils.canDisable(appCtrl.nf.CanvasUtils.getSelection());">
ng-disabled="!appCtrl.nf.CanvasUtils.canDisable(appCtrl.nf.CanvasUtils.getSelection());">
<i class="graph-control-action-icon icon icon-enable-false"></i></button>
</div>
<div class="button-spacer-large">&nbsp;</div>
<div id="operate-start" class="action-button" title="Start">
<button ng-click="appCtrl.nf.Actions['start'](appCtrl.nf.CanvasUtils.getSelection());"
ng-disabled="!appCtrl.nf.Common.isDFM();">
ng-disabled="!appCtrl.nf.CanvasUtils.getSelection().empty() && !appCtrl.nf.CanvasUtils.canModify(appCtrl.nf.CanvasUtils.getSelection());">
<i class="graph-control-action-icon fa fa-play"></i></button>
</div>
<div class="button-spacer-small">&nbsp;</div>
<div id="operate-stop" class="action-button" title="Stop">
<button ng-click="appCtrl.nf.Actions['stop'](appCtrl.nf.CanvasUtils.getSelection());"
ng-disabled="!appCtrl.nf.Common.isDFM();">
ng-disabled="!appCtrl.nf.CanvasUtils.getSelection().empty() && !appCtrl.nf.CanvasUtils.canModify(appCtrl.nf.CanvasUtils.getSelection());">
<i class="graph-control-action-icon fa fa-stop"></i></button>
</div>
<div class="button-spacer-large">&nbsp;</div>
<div id="operate-template" class="action-button" title="Create Template">
<button ng-click="appCtrl.nf.Actions['template'](appCtrl.nf.CanvasUtils.getSelection());"
ng-disabled="!appCtrl.nf.Common.isDFM();">
ng-disabled="!(appCtrl.nf.Canvas.canWrite() && (appCtrl.nf.CanvasUtils.getSelection().empty() || appCtrl.nf.CanvasUtils.canRead(appCtrl.nf.CanvasUtils.getSelection())));">
<i class="graph-control-action-icon icon icon-template"></i></button>
</div>
<div class="button-spacer-large">&nbsp;</div>
<div id="operate-copy" class="action-button" title="Copy">
<button ng-click="appCtrl.nf.Actions['copy'](appCtrl.nf.CanvasUtils.getSelection());"
ng-disabled="appCtrl.nf.CanvasUtils.getSelection().empty() || !appCtrl.nf.CanvasUtils.isCopyable(appCtrl.nf.CanvasUtils.getSelection());">
ng-disabled="!appCtrl.nf.CanvasUtils.isCopyable(appCtrl.nf.CanvasUtils.getSelection()) || !appCtrl.nf.CanvasUtils.canRead(appCtrl.nf.CanvasUtils.getSelection());">
<i class="graph-control-action-icon fa fa-copy"></i></button>
</div>
<div class="button-spacer-small">&nbsp;</div>
<div id="operate-paste" class="action-button" title="Paste">
<button ng-click="appCtrl.nf.Actions['paste'](appCtrl.nf.CanvasUtils.getSelection());"
ng-disabled="!appCtrl.nf.Clipboard.isCopied() || !appCtrl.nf.CanvasUtils.canModify(appCtrl.nf.CanvasUtils.getSelection());">
ng-disabled="!appCtrl.nf.Clipboard.isCopied()">
<i class="graph-control-action-icon fa fa-paste"></i></button>
</div>
<div class="clear"></div>
@ -129,19 +129,19 @@
<div style="margin-top: 5px;">
<div id="operate-group" class="action-button" title="Group">
<button ng-click="appCtrl.nf.Actions['group'](appCtrl.nf.CanvasUtils.getSelection());"
ng-disabled="appCtrl.nf.CanvasUtils.getSelection().empty() || !appCtrl.nf.CanvasUtils.isDisconnected(appCtrl.nf.CanvasUtils.getSelection());">
ng-disabled="!appCtrl.nf.CanvasUtils.isDisconnected(appCtrl.nf.CanvasUtils.getSelection()) || !appCtrl.nf.CanvasUtils.canModify(appCtrl.nf.CanvasUtils.getSelection());">
<i class="graph-control-action-icon icon icon-group"></i></button>
</div>
<div class="button-spacer-large">&nbsp;</div>
<div id="operate-color" class="action-button" title="Fill Color">
<button ng-click="appCtrl.nf.Actions['fillColor'](appCtrl.nf.CanvasUtils.getSelection());"
ng-disabled="appCtrl.nf.CanvasUtils.getSelection().empty() || !appCtrl.nf.CanvasUtils.isColorable(appCtrl.nf.CanvasUtils.getSelection());">
ng-disabled="!appCtrl.nf.CanvasUtils.isColorable(appCtrl.nf.CanvasUtils.getSelection());">
<i class="graph-control-action-icon fa fa-paint-brush"></i></button>
</div>
<div class="button-spacer-large">&nbsp;</div>
<div id="operate-delete" class="action-button" title="Delete">
<button ng-click="appCtrl.nf.Actions['delete'](appCtrl.nf.CanvasUtils.getSelection());"
ng-disabled="appCtrl.nf.CanvasUtils.getSelection().empty();">
ng-disabled="!appCtrl.nf.CanvasUtils.areDeletable(appCtrl.nf.CanvasUtils.getSelection());">
<i class="graph-control-action-icon fa fa-trash"></i><span>Delete</span></button>
</div>
<div class="clear"></div>

View File

@ -74,6 +74,7 @@ div.graph-control {
div.graph-control button {
height: 24px;
width: 24px;
line-height: 19px;
border-radius: 2px;
border: 1px solid #CCDADB; /*tint link-color 80%*/
background-color: rgba(249, 250, 251, 0.97);

View File

@ -38,15 +38,21 @@ nf.ng.BreadcrumbsCtrl = function (serviceProvider, $sanitize) {
/**
* Generate the breadcrumbs.
*
* @param {object} breadcrumb The breadcrumb
* @param {object} breadcrumbEntity The breadcrumb
*/
generateBreadcrumbs: function(breadcrumb) {
generateBreadcrumbs: function(breadcrumbEntity) {
var label = breadcrumbEntity.id;
if (breadcrumbEntity.accessPolicy.canRead) {
label = breadcrumbEntity.breadcrumb.name;
}
//explicitly sanitize processGroup.name
breadcrumb.name = $sanitize(breadcrumb.name);
this.breadcrumbs.unshift(breadcrumb);
this.breadcrumbs.unshift($.extend({
'label': $sanitize(label)
}, breadcrumbEntity));
if (nf.Common.isDefinedAndNotNull(breadcrumb.parentBreadcrumb)) {
this.generateBreadcrumbs(breadcrumb.parentBreadcrumb);
if (nf.Common.isDefinedAndNotNull(breadcrumbEntity.parentBreadcrumb)) {
this.generateBreadcrumbs(breadcrumbEntity.parentBreadcrumb);
}
},

View File

@ -157,7 +157,9 @@ nf.Actions = (function () {
// reload the group's connections
var connections = nf.Connection.getComponentConnections(remoteProcessGroup.id);
$.each(connections, function (_, connection) {
nf.Connection.reload(connection.component);
if (connection.accessPolicy.canRead) {
nf.Connection.reload(connection.component);
}
});
}
});
@ -487,13 +489,11 @@ nf.Actions = (function () {
if (selection.empty()) {
// build the entity
var entity = {
'component': {
'id': nf.Canvas.getGroupId(),
'running': true
}
'id': nf.Canvas.getGroupId(),
'state': 'RUNNING'
};
updateResource(config.urls.api + '/process-groups/' + encodeURIComponent(nf.Canvas.getGroupId()), entity).done(updateProcessGroup);
updateResource(config.urls.api + '/flow/process-groups/' + encodeURIComponent(nf.Canvas.getGroupId()), entity).done(updateProcessGroup);
} else {
var componentsToStart = selection.filter(function (d) {
return nf.CanvasUtils.isRunnable(d3.select(this));
@ -535,7 +535,9 @@ nf.Actions = (function () {
// reload the group's connections
var connections = nf.Connection.getComponentConnections(response.id);
$.each(connections, function (_, connection) {
nf.Connection.reload(connection);
if (connection.accessPolicy.canRead) {
nf.Connection.reload(connection.component);
}
});
} else {
nf[d.type].set(response);
@ -562,13 +564,11 @@ nf.Actions = (function () {
if (selection.empty()) {
// build the entity
var entity = {
'component': {
'id': nf.Canvas.getGroupId(),
'running': false
}
'id': nf.Canvas.getGroupId(),
'state': 'STOPPED'
};
updateResource(config.urls.api + '/process-groups/' + encodeURIComponent(nf.Canvas.getGroupId()), entity).done(updateProcessGroup);
updateResource(config.urls.api + '/flow/process-groups/' + encodeURIComponent(nf.Canvas.getGroupId()), entity).done(updateProcessGroup);
} else {
var componentsToStop = selection.filter(function (d) {
return nf.CanvasUtils.isStoppable(d3.select(this));
@ -610,7 +610,9 @@ nf.Actions = (function () {
// reload the group's connections
var connections = nf.Connection.getComponentConnections(response.id);
$.each(connections, function (_, connection) {
nf.Connection.reload(connection);
if (connection.accessPolicy.canRead) {
nf.Connection.reload(connection.component);
}
});
} else {
nf[d.type].set(response);
@ -826,10 +828,10 @@ nf.Actions = (function () {
}).fail(nf.Common.handleAjaxError);
} else {
// create a snippet for the specified component and link to the data flow
var snippetDetails = nf.Snippet.marshal(selection, true);
nf.Snippet.create(snippetDetails).done(function (snippetEntity) {
var snippet = nf.Snippet.marshal(selection);
nf.Snippet.create(snippet).done(function (response) {
// remove the snippet, effectively removing the components
nf.Snippet.remove(snippetEntity).done(function () {
nf.Snippet.remove(response.snippet.id).done(function () {
var components = d3.map();
// add the id to the type's array
@ -870,18 +872,10 @@ nf.Actions = (function () {
// refresh the birdseye
nf.Birdseye.refresh();
// inform Angular app values have changed
nf.ng.Bridge.digest();
}).fail(function (xhr, status, error) {
// unable to actually remove the components so attempt to
// unlink and remove just the snippet - if unlinking fails
// just ignore
nf.Snippet.unlink(snippetEntity).done(function (unlinkedSnippetEntity) {
nf.Snippet.remove(unlinkedSnippetEntity);
});
nf.Common.handleAjaxError(xhr, status, error);
});
}).fail(nf.Common.handleAjaxError);
}).fail(nf.Common.handleAjaxError);
}
}
@ -1222,14 +1216,14 @@ nf.Actions = (function () {
var templateDescription = $('#new-template-description').val();
// create a snippet
var snippetDetails = nf.Snippet.marshal(selection, false);
var snippet = nf.Snippet.marshal(selection);
// create the snippet
nf.Snippet.create(snippetDetails).done(function (snippetEntity) {
nf.Snippet.create(snippet).done(function (response) {
var createSnippetEntity = {
'name': templateName,
'description': templateDescription,
'snippetId': snippetEntity.id
'snippetId': response.snippet.id
};
// create the template
@ -1246,9 +1240,6 @@ nf.Actions = (function () {
overlayBackground: false
});
}).always(function () {
// remove the snippet
nf.Snippet.remove(snippetEntity);
// clear the template dialog fields
$('#new-template-name').val('');
$('#new-template-description').val('');
@ -1284,7 +1275,7 @@ nf.Actions = (function () {
// copy the snippet details
nf.Clipboard.copy({
snippet: nf.Snippet.marshal(selection, false),
snippet: nf.Snippet.marshal(selection),
origin: origin
});
},
@ -1323,7 +1314,7 @@ nf.Actions = (function () {
};
// create a snippet from the details
nf.Snippet.create(data['snippet']).done(function (snippetEntity) {
nf.Snippet.create(data['snippet']).done(function (createResponse) {
// determine the origin of the bounding box of the copy
var origin = pasteLocation;
var snippetOrigin = data['origin'];
@ -1336,7 +1327,7 @@ nf.Actions = (function () {
}
// copy the snippet to the new location
nf.Snippet.copy(snippetEntity.id, origin).done(function (copyResponse) {
nf.Snippet.copy(createResponse.snippet.id, origin).done(function (copyResponse) {
var snippetFlow = copyResponse.flow;
// update the graph accordingly
@ -1349,9 +1340,6 @@ nf.Actions = (function () {
// refresh the birdseye/toolbar
nf.Birdseye.refresh();
// remove the original snippet
nf.Snippet.remove(snippetEntity).fail(reject);
}).fail(function () {
// an error occured while performing the copy operation, reload the
// graph in case it was a partial success

View File

@ -55,11 +55,11 @@ nf.CanvasUtils = (function () {
return $.Deferred(function (deferred) {
// ensure the current selection is eligible for move into the specified group
nf.CanvasUtils.eligibleForMove(components, groupId).done(function () {
// create a snippet for the specified components and link to the data flow
var snippetDetails = nf.Snippet.marshal(components, true);
nf.Snippet.create(snippetDetails).done(function (snippetEntity) {
// create a snippet for the specified components
var snippet = nf.Snippet.marshal(components);
nf.Snippet.create(snippet).done(function (response) {
// move the snippet into the target
nf.Snippet.move(snippetEntity, groupId).done(function () {
nf.Snippet.move(response.snippet.id, groupId).done(function () {
var componentMap = d3.map();
// add the id to the type's array
@ -85,12 +85,6 @@ nf.CanvasUtils = (function () {
deferred.resolve();
}).fail(nf.Common.handleAjaxError).fail(function () {
deferred.reject();
}).always(function () {
// unable to acutally move the components so attempt to
// unlink and remove just the snippet
nf.Snippet.unlink(snippetEntity).done(function (unlinkedSnippetEntity) {
nf.Snippet.remove(unlinkedSnippetEntity);
});
});
}).fail(nf.Common.handleAjaxError).fail(function () {
deferred.reject();
@ -240,6 +234,7 @@ nf.CanvasUtils = (function () {
getSelection: function () {
return d3.selectAll('g.component.selected, g.connection.selected');
},
/**
* Centers the specified bounding box.
*
@ -558,10 +553,12 @@ nf.CanvasUtils = (function () {
// determine if the current selection is entirely processors or labels
var selectedProcessors = selection.filter(function(d) {
return nf.CanvasUtils.isProcessor(d3.select(this));
var processor = d3.select(this);
return nf.CanvasUtils.isProcessor(processor) && nf.CanvasUtils.canModify(processor);
});
var selectedLabels = selection.filter(function(d) {
return nf.CanvasUtils.isLabel(d3.select(this));
var label = d3.select(this);
return nf.CanvasUtils.isLabel(label) && nf.CanvasUtils.canModify(label);
});
var allProcessors = selectedProcessors.size() === selection.size();
@ -753,7 +750,7 @@ nf.CanvasUtils = (function () {
return false;
}
return !nf.CanvasUtils.filterEnable(selection).empty();
return nf.CanvasUtils.filterEnable(selection).size() === selection.size();
},
/**
@ -783,7 +780,7 @@ nf.CanvasUtils = (function () {
return false;
}
return !nf.CanvasUtils.filterDisable(selection).empty();
return nf.CanvasUtils.filterDisable(selection).size() === selection.size();
},
@ -844,13 +841,33 @@ nf.CanvasUtils = (function () {
canStopTransmitting: function (selection) {
return nf.CanvasUtils.isRemoteProcessGroup(selection);
},
/**
* Determines whether the components in the specified selection are deletable.
*
* @argument {selection} selection The selection
* @return {boolean} Whether the selection is deletable
*/
areDeletable: function (selection) {
if (selection.empty()) {
return false;
}
var isDeletable = true;
selection.each(function () {
if (!nf.CanvasUtils.isDeletable(d3.select(this))) {
isDeletable = false;
}
});
return isDeletable;
},
/**
* Determines whether the component in the specified selection is deletable.
*
* @argument {selection} selection The selection
* @return {boolean} Whether the selection is deletable
*/
isDeletable: function (selection) {
if (selection.empty()) {
return false;
@ -895,6 +912,11 @@ nf.CanvasUtils = (function () {
* @argument {selection} selection The selection
*/
supportsModification: function (selection) {
if (selection.size() !== 1) {
return false;
}
// get the selection data
var selectionData = selection.datum();
// check access policies first
@ -1301,6 +1323,11 @@ nf.CanvasUtils = (function () {
* @argument {selection} selection The selection
*/
isDisconnected: function (selection) {
// if nothing is selected return
if (selection.empty()) {
return false;
}
var connections = d3.map();
var components = d3.map();
var isDisconnected = true;

View File

@ -111,6 +111,7 @@ nf.Canvas = (function () {
var polling = false;
var groupId = 'root';
var groupName = null;
var accessPolicy = null;
var parentGroupId = null;
var secureSiteToSite = false;
var clustered = false;
@ -621,6 +622,9 @@ nf.Canvas = (function () {
// set the group details
nf.Canvas.setGroupId(processGroupFlow.id);
// update the access policies
accessPolicy = flowResponse.accessPolicy;
// update the breadcrumbs
nf.ng.Bridge.get('appCtrl.serviceProvider.breadcrumbsCtrl').resetBreadcrumbs();
nf.ng.Bridge.get('appCtrl.serviceProvider.breadcrumbsCtrl').generateBreadcrumbs(processGroupFlow.breadcrumb);
@ -941,6 +945,19 @@ nf.Canvas = (function () {
return parentGroupId;
},
/**
* Whether the current user can write in this group.
*
* @returns {boolean} can write
*/
canWrite: function () {
if (accessPolicy === null) {
return false;
} else {
return accessPolicy.canWrite === true;
}
},
View: (function () {
/**

View File

@ -87,7 +87,7 @@ nf.ContextMenu = (function () {
return false;
}
return nf.CanvasUtils.isDeletable(selection);
return nf.CanvasUtils.areDeletable(selection);
};
/**

View File

@ -378,7 +378,7 @@ nf.ProcessorConfiguration = (function () {
$.each(connections, function (_, connection) {
if (connection.accessPolicy.canRead) {
if (connection.sourceId === processor.id) {
nf.Connection.reload(connection);
nf.Connection.reload(connection.component);
}
}
});

View File

@ -942,7 +942,9 @@ nf.RemoteProcessGroup = (function () {
// reload the group's connections
var connections = nf.Connection.getComponentConnections(remoteProcessGroup.id);
$.each(connections, function (_, connection) {
nf.Connection.reload(connection);
if (connection.accessPolicy.canRead) {
nf.Connection.reload(connection.component);
}
});
});
}

View File

@ -21,6 +21,7 @@ nf.Snippet = (function () {
var config = {
urls: {
snippets: '../nifi-api/snippets',
processGroups: '../nifi-api/process-groups'
}
};
@ -30,12 +31,10 @@ nf.Snippet = (function () {
* Marshals snippet from the specified selection.
*
* @argument {selection} selection The selection to marshal
* @argument {boolean} linked Whether this snippet should be linked to the flow
*/
marshal: function (selection, linked) {
marshal: function (selection) {
var snippet = {
parentGroupId: nf.Canvas.getGroupId(),
linked: nf.Common.isDefinedAndNotNull(linked) ? linked : false,
processors: {},
funnels: {},
inputPorts: {},
@ -84,7 +83,7 @@ nf.Snippet = (function () {
return $.ajax({
type: 'POST',
url: config.urls.processGroups + '/' + encodeURIComponent(nf.Canvas.getGroupId()) + '/snippets',
url: config.urls.snippets,
data: JSON.stringify(snippetEntity),
dataType: 'json',
contentType: 'application/json'
@ -116,88 +115,36 @@ nf.Snippet = (function () {
/**
* Removes the specified snippet.
*
* @argument {string} snippetEntity The snippet entity
* @argument {string} snippetId The snippet id
*/
remove: function (snippetEntity) {
var revision = nf.Client.getRevision(snippetEntity);
remove: function (snippetId) {
return $.ajax({
type: 'DELETE',
url: config.urls.processGroups + '/' + encodeURIComponent(nf.Canvas.getGroupId()) + '/snippets/' + encodeURIComponent(snippetEntity.id) + '?' + $.param({
version: revision.version,
clientId: revision.clientId
})
url: config.urls.snippets + '/' + encodeURIComponent(snippetId)
});
},
/**
* Moves the snippet into the specified group.
*
* @argument {object} snippetEntity The snippet entity
* @argument {string} snippetId The snippet id
* @argument {string} newGroupId The new group id
*/
move: function (snippetEntity, newGroupId) {
move: function (snippetId, newGroupId) {
var moveSnippetEntity = {
'revision': nf.Client.getRevision(snippetEntity),
'snippet': {
'id': snippetEntity.id,
'id': snippetId,
'parentGroupId': newGroupId
}
};
return $.ajax({
type: 'PUT',
url: config.urls.processGroups + '/' + encodeURIComponent(nf.Canvas.getGroupId()) + '/snippets/' + encodeURIComponent(snippetEntity.id),
url: config.urls.snippets + '/' + encodeURIComponent(snippetId),
data: JSON.stringify(moveSnippetEntity),
dataType: 'json',
contentType: 'application/json'
});
},
/**
* Unlinks the snippet from the actual data flow.
*
* @argument {object} snippetEntity The snippet enmtity
*/
unlink: function (snippetEntity) {
var unlinkSnippetEntity = {
'revision': nf.Client.getRevision(snippetEntity),
'snippet': {
'id': snippetEntity.id,
'linked': false
}
};
return $.ajax({
type: 'PUT',
url: config.urls.processGroups + '/' + encodeURIComponent(nf.Canvas.getGroupId()) + '/snippets/' + encodeURIComponent(snippetEntity.id),
data: JSON.stringify(unlinkSnippetEntity),
dataType: 'json',
contentType: 'application/json'
});
},
/**
* Links the snippet from the actual data flow.
*
* @argument {object} snippetEntity The snippet entity
*/
link: function (snippetEntity) {
var linkSnippetEntity = {
'revision': nf.Client.getRevision(snippetEntity),
'snippet': {
'id': snippetEntity.id,
'linked': true
}
};
return $.ajax({
type: 'PUT',
url: config.urls.processGroups + '/' + encodeURIComponent(nf.Canvas.getGroupId()) + '/snippets/' + encodeURIComponent(snippetEntity.id),
data: JSON.stringify(linkSnippetEntity),
dataType: 'json',
contentType: 'application/json'
});
}
};
}());

View File

@ -383,7 +383,7 @@ nf.ConnectionDetails = (function () {
var groupResponse = groupResult[0];
var connectionResponse = connectionResult[0];
if (nf.Common.isDefinedAndNotNull(groupResponse.processGroup) && nf.Common.isDefinedAndNotNull(connectionResponse.connection)) {
if (nf.Common.isDefinedAndNotNull(groupResponse.component) && nf.Common.isDefinedAndNotNull(connectionResponse.component)) {
var processGroup = groupResponse.component;
var connection = connectionResponse.component;

View File

@ -211,7 +211,7 @@ nf.ProcessorDetails = (function () {
// show the dialog once we have the processor and its history
$.when(getProcessor, getProcessorHistory).done(function (processorResponse, historyResponse) {
var processorResponse = processorResponse[0];
var processor = processorResponse.processor;
var processor = processorResponse.component;
var historyResponse = historyResponse[0];
var history = historyResponse.componentHistory;

View File

@ -25,7 +25,7 @@ limitations under the License.
<span class="link"
ng-class="(highlightCrumbId === crumb.id) ? 'link-bold' : ''"
ng-click="clickFunc(crumb.id)">
{{crumb.name}}
{{crumb.label}}
</span>
</span>
</div>