mirror of https://github.com/apache/nifi.git
NIFI-2123: Add authorization of provenance events; refactor core classes so that Authorizable is located within nifi-api. This closes #592
This commit is contained in:
parent
65d895827b
commit
ae9e2fdf0b
|
@ -20,6 +20,7 @@ package org.apache.nifi.authorization;
|
|||
* Represents any error that might occur while authorizing user requests.
|
||||
*/
|
||||
public class AccessDeniedException extends RuntimeException {
|
||||
private static final long serialVersionUID = -5683444815269084134L;
|
||||
|
||||
public AccessDeniedException(Throwable cause) {
|
||||
super(cause);
|
||||
|
@ -35,5 +36,4 @@ public class AccessDeniedException extends RuntimeException {
|
|||
|
||||
public AccessDeniedException() {
|
||||
}
|
||||
|
||||
}
|
|
@ -24,7 +24,6 @@ import org.apache.nifi.authorization.Authorizer;
|
|||
import org.apache.nifi.authorization.RequestAction;
|
||||
import org.apache.nifi.authorization.Resource;
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.authorization.user.NiFiUserUtils;
|
||||
|
||||
public interface Authorizable {
|
||||
|
||||
|
@ -51,21 +50,8 @@ public interface Authorizable {
|
|||
* @param action action
|
||||
* @return is authorized
|
||||
*/
|
||||
default boolean isAuthorized(Authorizer authorizer, RequestAction action) {
|
||||
return Result.Approved.equals(checkAuthorization(authorizer, action).getResult());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the result of an authorization request for the current user for the specified action on the specified
|
||||
* resource. This method does not imply the user is directly attempting to access the specified resource. If the user is
|
||||
* attempting a direct access use Authorizable.authorize().
|
||||
*
|
||||
* @param authorizer authorizer
|
||||
* @param action action
|
||||
* @return is authorized
|
||||
*/
|
||||
default AuthorizationResult checkAuthorization(Authorizer authorizer, RequestAction action) {
|
||||
return checkAuthorization(authorizer, action, NiFiUserUtils.getNiFiUser());
|
||||
default boolean isAuthorized(Authorizer authorizer, RequestAction action, NiFiUser user) {
|
||||
return Result.Approved.equals(checkAuthorization(authorizer, action, user).getResult());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -99,7 +85,7 @@ public interface Authorizable {
|
|||
if (parent == null) {
|
||||
return AuthorizationResult.denied();
|
||||
} else {
|
||||
return parent.checkAuthorization(authorizer, action);
|
||||
return parent.checkAuthorization(authorizer, action, user);
|
||||
}
|
||||
} else {
|
||||
return result;
|
||||
|
@ -113,9 +99,7 @@ public interface Authorizable {
|
|||
* @param authorizer authorizer
|
||||
* @param action action
|
||||
*/
|
||||
default void authorize(Authorizer authorizer, RequestAction action) throws AccessDeniedException {
|
||||
final NiFiUser user = NiFiUserUtils.getNiFiUser();
|
||||
|
||||
default void authorize(Authorizer authorizer, RequestAction action, NiFiUser user) throws AccessDeniedException {
|
||||
// TODO - include user details context
|
||||
|
||||
final AuthorizationRequest request = new AuthorizationRequest.Builder()
|
||||
|
@ -132,7 +116,7 @@ public interface Authorizable {
|
|||
if (parent == null) {
|
||||
throw new AccessDeniedException("Access is denied");
|
||||
} else {
|
||||
parent.authorize(authorizer, action);
|
||||
parent.authorize(authorizer, action, user);
|
||||
}
|
||||
} else if (Result.Denied.equals(result.getResult())) {
|
||||
throw new AccessDeniedException(result.getExplanation());
|
|
@ -0,0 +1,44 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.authorization.user;
|
||||
|
||||
/**
|
||||
* A representation of a NiFi user that has logged into the application
|
||||
*/
|
||||
public interface NiFiUser {
|
||||
|
||||
/**
|
||||
* @return the unique identity of this user
|
||||
*/
|
||||
String getIdentity();
|
||||
|
||||
/**
|
||||
* @return the user name for this user
|
||||
*/
|
||||
String getUserName();
|
||||
|
||||
/**
|
||||
* @return the next user in the proxied entities chain, or <code>null</code> if no more users exist in the chain.
|
||||
*/
|
||||
NiFiUser getChain();
|
||||
|
||||
/**
|
||||
* @return <code>true</code> if the user is the unauthenticated Anonymous user
|
||||
*/
|
||||
boolean isAnonymous();
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
/*
|
||||
* 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.provenance;
|
||||
|
||||
import org.apache.nifi.authorization.resource.Authorizable;
|
||||
import org.apache.nifi.web.ResourceNotFoundException;
|
||||
|
||||
public interface ProvenanceAuthorizableFactory {
|
||||
|
||||
/**
|
||||
* Generates an Authorizable object for the Provenance events of the component with the given ID
|
||||
*
|
||||
* @param componentId the ID of the component to which the Provenance events belong
|
||||
*
|
||||
* @return the Authorizable that can be use to authorize access to provenance events
|
||||
* @throws ResourceNotFoundException if no component can be found with the given ID
|
||||
*/
|
||||
Authorizable createProvenanceAuthorizable(String componentId);
|
||||
|
||||
}
|
|
@ -19,11 +19,15 @@ package org.apache.nifi.provenance;
|
|||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.nifi.authorization.AccessDeniedException;
|
||||
import org.apache.nifi.authorization.Authorizer;
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.events.EventReporter;
|
||||
import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
|
||||
import org.apache.nifi.provenance.search.Query;
|
||||
import org.apache.nifi.provenance.search.QuerySubmission;
|
||||
import org.apache.nifi.provenance.search.SearchableField;
|
||||
import org.apache.nifi.web.ResourceNotFoundException;
|
||||
|
||||
/**
|
||||
* This Repository houses Provenance Events. The repository is responsible for
|
||||
|
@ -38,9 +42,12 @@ public interface ProvenanceEventRepository {
|
|||
* framework.
|
||||
*
|
||||
* @param eventReporter to report to
|
||||
* @param authorizer the authorizer to use for authorizing individual events
|
||||
* @param resourceFactory the resource factory to use for generating Provenance Resource objects for authorization purposes
|
||||
*
|
||||
* @throws java.io.IOException if unable to initialize
|
||||
*/
|
||||
void initialize(EventReporter eventReporter) throws IOException;
|
||||
void initialize(EventReporter eventReporter, Authorizer authorizer, ProvenanceAuthorizableFactory resourceFactory) throws IOException;
|
||||
|
||||
/**
|
||||
* Returns a {@link ProvenanceEventBuilder} that is capable of building
|
||||
|
@ -74,7 +81,8 @@ public interface ProvenanceEventRepository {
|
|||
/**
|
||||
* Returns a List of all <code>ProvenanceEventRecord</code>s in the
|
||||
* repository starting with the given ID. The first ID in the repository
|
||||
* will always be 0 or higher.
|
||||
* will always be 0 or higher. This method performs no authorization of
|
||||
* the events.
|
||||
*
|
||||
* @param firstRecordId id of the first record to retrieve
|
||||
* @param maxRecords maximum number of records to retrieve
|
||||
|
@ -83,9 +91,25 @@ public interface ProvenanceEventRepository {
|
|||
*/
|
||||
List<ProvenanceEventRecord> getEvents(long firstRecordId, final int maxRecords) throws IOException;
|
||||
|
||||
/**
|
||||
* Returns a List of all <code>ProvenanceEventRecord</code>s in the
|
||||
* repository starting with the given ID. The first ID in the repository
|
||||
* will always be 0 or higher. Each event that is found will be authorized
|
||||
* against the given NiFiUser. If the user does not have authorization for
|
||||
* the event, the event will not be returned.
|
||||
*
|
||||
* @param firstRecordId id of the first record to retrieve
|
||||
* @param maxRecords maximum number of records to retrieve
|
||||
* @param user the NiFi user that the events should be authorized against
|
||||
*
|
||||
* @return records
|
||||
* @throws java.io.IOException if error reading from repository
|
||||
*/
|
||||
List<ProvenanceEventRecord> getEvents(long firstRecordId, final int maxRecords, NiFiUser user) throws IOException;
|
||||
|
||||
/**
|
||||
* @return the largest ID of any event that is queryable in the repository.
|
||||
* If no queryable events exists, returns null
|
||||
* If no queryable events exists, returns null
|
||||
*/
|
||||
Long getMaxEventId();
|
||||
|
||||
|
@ -94,70 +118,96 @@ public interface ProvenanceEventRepository {
|
|||
* identifier that can be used to fetch the results at a later time
|
||||
*
|
||||
* @param query to submit
|
||||
* @param user the NiFi User to authorize the events against
|
||||
*
|
||||
* @return an identifier that can be used to fetch the results at a later
|
||||
* time
|
||||
* time
|
||||
*/
|
||||
QuerySubmission submitQuery(Query query);
|
||||
QuerySubmission submitQuery(Query query, NiFiUser user);
|
||||
|
||||
/**
|
||||
* @param queryIdentifier of the query
|
||||
* @param user the user who is retrieving the query
|
||||
*
|
||||
* @return the QueryResult associated with the given identifier, if the
|
||||
* query has finished processing. If the query has not yet finished running,
|
||||
* returns <code>null</code>
|
||||
* query has finished processing. If the query has not yet finished running,
|
||||
* returns <code>null</code>
|
||||
*/
|
||||
QuerySubmission retrieveQuerySubmission(String queryIdentifier);
|
||||
QuerySubmission retrieveQuerySubmission(String queryIdentifier, NiFiUser user);
|
||||
|
||||
/**
|
||||
* Submits a Lineage Computation to be completed and returns the
|
||||
* AsynchronousLineageResult that indicates the status of the request and
|
||||
* the results, if the computation is complete.
|
||||
* the results, if the computation is complete. If the given user does not
|
||||
* have authorization to view one of the events in the lineage, a placeholder
|
||||
* event will be used instead that provides none of the event details except
|
||||
* for the identifier of the component that emitted the Provenance Event. It is
|
||||
* necessary to include this node in the lineage view so that the lineage makes
|
||||
* sense, rather than showing disconnected graphs when the user is not authorized
|
||||
* for all components' provenance events.
|
||||
*
|
||||
* @param flowFileUuid the UUID of the FlowFile for which the Lineage should
|
||||
* be calculated
|
||||
* be calculated
|
||||
* @param user the NiFi User to authorize events against
|
||||
*
|
||||
* @return a {@link ComputeLineageSubmission} object that can be used to
|
||||
* check if the computing is complete and if so get the results
|
||||
* check if the computing is complete and if so get the results
|
||||
*/
|
||||
ComputeLineageSubmission submitLineageComputation(String flowFileUuid);
|
||||
ComputeLineageSubmission submitLineageComputation(String flowFileUuid, NiFiUser user);
|
||||
|
||||
/**
|
||||
* @param lineageIdentifier identifier of lineage to compute
|
||||
* @param user the user who is retrieving the lineage submission
|
||||
*
|
||||
* @return the {@link ComputeLineageSubmission} associated with the given
|
||||
* identifier
|
||||
* identifier
|
||||
*/
|
||||
ComputeLineageSubmission retrieveLineageSubmission(String lineageIdentifier);
|
||||
ComputeLineageSubmission retrieveLineageSubmission(String lineageIdentifier, NiFiUser user);
|
||||
|
||||
/**
|
||||
* Retrieves the Provenance Event with the given ID. The event will be returned only
|
||||
* if the given user is authorized to access the event. Otherwise, an
|
||||
* AccessDeniedException or ResourceNotFoundException will be thrown, as appropriate
|
||||
*
|
||||
* @param id to lookup
|
||||
* @return the Provenance Event Record with the given ID, if it exists, or
|
||||
* {@code null} otherwise
|
||||
* {@code null} otherwise
|
||||
* @throws IOException if failure while retrieving event
|
||||
* @throws AccessDeniedException if the user does not have access to the component
|
||||
* @throws ResourceNotFoundException if the component that the event belongs to cannot be found
|
||||
*/
|
||||
ProvenanceEventRecord getEvent(long id) throws IOException;
|
||||
ProvenanceEventRecord getEvent(long id, NiFiUser user) throws IOException;
|
||||
|
||||
/**
|
||||
* Submits a request to expand the parents of the event with the given id
|
||||
* Submits a request to expand the parents of the event with the given id. If the given user
|
||||
* is not authorized to access any event, a placeholder will be used instead that contains only
|
||||
* the ID of the component that emitted the event.
|
||||
*
|
||||
* @param eventId the one-up id of the Event to expand
|
||||
* @param user the NiFi user to authorize events against
|
||||
* @return a submission which can be checked for status
|
||||
*
|
||||
* @throws IllegalArgumentException if the given identifier identifies a
|
||||
* Provenance Event that has a Type that is not expandable or if the
|
||||
* identifier cannot be found
|
||||
* Provenance Event that has a Type that is not expandable or if the
|
||||
* identifier cannot be found
|
||||
*/
|
||||
ComputeLineageSubmission submitExpandParents(long eventId);
|
||||
ComputeLineageSubmission submitExpandParents(long eventId, NiFiUser user);
|
||||
|
||||
/**
|
||||
* Submits a request to expand the children of the event with the given id
|
||||
* Submits a request to expand the children of the event with the given id. If the given user
|
||||
* is not authorized to access any event, a placeholder will be used instead that contains only
|
||||
* the ID of the component that emitted the event.
|
||||
*
|
||||
* @param eventId the one-up id of the Event
|
||||
* @param user the NiFi user to authorize events against
|
||||
*
|
||||
* @return a submission which can be checked for status
|
||||
*
|
||||
* @throws IllegalArgumentException if the given identifier identifies a
|
||||
* Provenance Event that has a Type that is not expandable or if the
|
||||
* identifier cannot be found
|
||||
* Provenance Event that has a Type that is not expandable or if the
|
||||
* identifier cannot be found
|
||||
*/
|
||||
ComputeLineageSubmission submitExpandChildren(long eventId);
|
||||
ComputeLineageSubmission submitExpandChildren(long eventId, NiFiUser user);
|
||||
|
||||
/**
|
||||
* Closes the repository, freeing any resources
|
||||
|
|
|
@ -117,5 +117,11 @@ public enum ProvenanceEventType {
|
|||
* that is the UUID of the a newly created FlowFile that will be re-queued
|
||||
* for processing.
|
||||
*/
|
||||
REPLAY
|
||||
REPLAY,
|
||||
|
||||
/**
|
||||
* Indicates that the type of the provenance event is unknown because the user
|
||||
* who is attempting to access the event is not authorize to know the type.
|
||||
*/
|
||||
UNKNOWN;
|
||||
}
|
||||
|
|
|
@ -33,6 +33,11 @@ public interface ComputeLineageSubmission {
|
|||
*/
|
||||
Date getSubmissionTime();
|
||||
|
||||
/**
|
||||
* @return the identity of the user who submitted the request
|
||||
*/
|
||||
String getSubmitterIdentity();
|
||||
|
||||
/**
|
||||
* @return the generated identifier for this lineage result
|
||||
*/
|
||||
|
|
|
@ -52,4 +52,9 @@ public interface QuerySubmission {
|
|||
* <code>false</code> otherwise
|
||||
*/
|
||||
boolean isCanceled();
|
||||
|
||||
/**
|
||||
* @return the identity of the user who submitted the query
|
||||
*/
|
||||
String getSubmitterIdentity();
|
||||
}
|
||||
|
|
|
@ -34,18 +34,25 @@ public class AsyncLineageSubmission implements ComputeLineageSubmission {
|
|||
private final LineageComputationType computationType;
|
||||
private final Long eventId;
|
||||
private final Collection<String> lineageFlowFileUuids;
|
||||
private final String submitterId;
|
||||
|
||||
private volatile boolean canceled = false;
|
||||
|
||||
private final StandardLineageResult result;
|
||||
|
||||
public AsyncLineageSubmission(final LineageComputationType computationType, final Long eventId, final Collection<String> lineageFlowFileUuids, final int numSteps) {
|
||||
public AsyncLineageSubmission(final LineageComputationType computationType, final Long eventId, final Collection<String> lineageFlowFileUuids, final int numSteps, final String submitterId) {
|
||||
this.computationType = computationType;
|
||||
this.eventId = eventId;
|
||||
this.lineageFlowFileUuids = lineageFlowFileUuids;
|
||||
this.submitterId = submitterId;
|
||||
this.result = new StandardLineageResult(numSteps, lineageFlowFileUuids);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getSubmitterIdentity() {
|
||||
return submitterId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StandardLineageResult getResult() {
|
||||
return result;
|
||||
|
|
|
@ -31,6 +31,7 @@ public class AsyncQuerySubmission implements QuerySubmission {
|
|||
|
||||
private volatile boolean canceled = false;
|
||||
private final StandardQueryResult queryResult;
|
||||
private final String submitterId;
|
||||
|
||||
/**
|
||||
* Constructs an AsyncQuerySubmission with the given query and the given
|
||||
|
@ -40,11 +41,17 @@ public class AsyncQuerySubmission implements QuerySubmission {
|
|||
* @param query the query to execute
|
||||
* @param numSteps how many steps to include
|
||||
*/
|
||||
public AsyncQuerySubmission(final Query query, final int numSteps) {
|
||||
public AsyncQuerySubmission(final Query query, final int numSteps, final String submitterId) {
|
||||
this.query = query;
|
||||
this.submitterId = submitterId;
|
||||
queryResult = new StandardQueryResult(query, numSteps);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getSubmitterIdentity() {
|
||||
return submitterId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Date getSubmissionTime() {
|
||||
return submissionTime;
|
||||
|
|
|
@ -0,0 +1,196 @@
|
|||
/*
|
||||
* 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.provenance;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* A Provenance Event that is used to replace another Provenance Event when authorizations
|
||||
* are not granted for the original Provenance Event
|
||||
*/
|
||||
public class PlaceholderProvenanceEvent implements ProvenanceEventRecord {
|
||||
private final String componentId;
|
||||
private final long eventId;
|
||||
private final long eventTime;
|
||||
private final String flowFileUuid;
|
||||
|
||||
public PlaceholderProvenanceEvent(final ProvenanceEventRecord original) {
|
||||
this.componentId = original.getComponentId();
|
||||
this.eventId = original.getEventId();
|
||||
this.eventTime = original.getEventTime();
|
||||
this.flowFileUuid = original.getFlowFileUuid();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getEventId() {
|
||||
return eventId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getEventTime() {
|
||||
return eventTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getFlowFileEntryDate() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLineageStartDate() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getLineageIdentifiers() {
|
||||
return Collections.emptySet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getFileSize() {
|
||||
return -1L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Long getPreviousFileSize() {
|
||||
return -1L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getEventDuration() {
|
||||
return -1L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ProvenanceEventType getEventType() {
|
||||
return ProvenanceEventType.UNKNOWN;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, String> getAttributes() {
|
||||
return Collections.emptyMap();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, String> getPreviousAttributes() {
|
||||
return Collections.emptyMap();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, String> getUpdatedAttributes() {
|
||||
return Collections.emptyMap();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getComponentId() {
|
||||
return componentId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getComponentType() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTransitUri() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getSourceSystemFlowFileIdentifier() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getFlowFileUuid() {
|
||||
return flowFileUuid;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getParentUuids() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getChildUuids() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getAlternateIdentifierUri() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getDetails() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRelationship() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getSourceQueueIdentifier() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getContentClaimSection() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPreviousContentClaimSection() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getContentClaimContainer() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPreviousContentClaimContainer() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getContentClaimIdentifier() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPreviousContentClaimIdentifier() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Long getContentClaimOffset() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Long getPreviousContentClaimOffset() {
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -268,7 +268,7 @@ public class StandardLineageResult implements ComputeLineageResult {
|
|||
final boolean isNewFlowFile = nodes.add(childNode);
|
||||
if (!isNewFlowFile) {
|
||||
final String msg = "Unable to generate Lineage Graph because multiple "
|
||||
+ "events were registered claiming to have generated the same FlowFile (UUID = " + childNode.getFlowFileUuid() + ")";
|
||||
+ "events were registered claiming to have generated the same FlowFile (UUID = " + childNode.getFlowFileUuid() + ")";
|
||||
logger.error(msg);
|
||||
setError(msg);
|
||||
return;
|
||||
|
@ -287,7 +287,7 @@ public class StandardLineageResult implements ComputeLineageResult {
|
|||
lastEventMap.put(parentUuid, lineageNode);
|
||||
}
|
||||
}
|
||||
break;
|
||||
break;
|
||||
case RECEIVE:
|
||||
case CREATE: {
|
||||
// for a receive event, we want to create a FlowFile Node that represents the FlowFile received
|
||||
|
@ -296,7 +296,7 @@ public class StandardLineageResult implements ComputeLineageResult {
|
|||
final boolean isNewFlowFile = nodes.add(flowFileNode);
|
||||
if (!isNewFlowFile) {
|
||||
final String msg = "Found cycle in graph. This indicates that multiple events "
|
||||
+ "were registered claiming to have generated the same FlowFile (UUID = " + flowFileNode.getFlowFileUuid() + ")";
|
||||
+ "were registered claiming to have generated the same FlowFile (UUID = " + flowFileNode.getFlowFileUuid() + ")";
|
||||
setError(msg);
|
||||
logger.error(msg);
|
||||
return;
|
||||
|
@ -304,7 +304,7 @@ public class StandardLineageResult implements ComputeLineageResult {
|
|||
edges.add(new EdgeNode(record.getFlowFileUuid(), lineageNode, flowFileNode));
|
||||
lastEventMap.put(record.getFlowFileUuid(), flowFileNode);
|
||||
}
|
||||
break;
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -90,7 +90,19 @@ public class StandardQueryResult implements QueryResult {
|
|||
public long getTotalHitCount() {
|
||||
readLock.lock();
|
||||
try {
|
||||
return totalHitCount;
|
||||
// Because we filter the results based on the user's permissions,
|
||||
// we don't want to indicate that the total hit count is 1,000+ when we
|
||||
// have 0 matching records, for instance. So, if we have fewer matching
|
||||
// records than the max specified by the query, it is either the case that
|
||||
// we truly don't have enough records to reach the max results, or that
|
||||
// the user is not authorized to see some of the results. Either way,
|
||||
// we want to report the number of events that we find AND that the user
|
||||
// is allowed to see, so we report matching record count, or up to max results.
|
||||
if (matchingRecords.size() < query.getMaxResults()) {
|
||||
return matchingRecords.size();
|
||||
} else {
|
||||
return query.getMaxResults();
|
||||
}
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
|
|
|
@ -22,6 +22,8 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.nifi.authorization.Authorizer;
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.events.EventReporter;
|
||||
import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
|
||||
import org.apache.nifi.provenance.search.Query;
|
||||
|
@ -54,7 +56,8 @@ public class MockProvenanceEventRepository implements ProvenanceEventRepository
|
|||
}
|
||||
|
||||
@Override
|
||||
public void initialize(EventReporter reporter) throws IOException {
|
||||
public void initialize(EventReporter eventReporter, Authorizer authorizer, ProvenanceAuthorizableFactory resourceFactory) throws IOException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -66,33 +69,38 @@ public class MockProvenanceEventRepository implements ProvenanceEventRepository
|
|||
return records.subList((int) firstRecordId, Math.min(records.size(), (int) (firstRecordId + maxRecords)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ProvenanceEventRecord> getEvents(long firstRecordId, int maxRecords, NiFiUser user) throws IOException {
|
||||
return getEvents(firstRecordId, maxRecords);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Long getMaxEventId() {
|
||||
return Long.valueOf(records.size() - 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public QuerySubmission submitQuery(Query query) {
|
||||
public QuerySubmission submitQuery(Query query, NiFiUser user) {
|
||||
throw new UnsupportedOperationException("MockProvenanceEventRepository does not support querying");
|
||||
}
|
||||
|
||||
@Override
|
||||
public QuerySubmission retrieveQuerySubmission(String queryIdentifier) {
|
||||
public QuerySubmission retrieveQuerySubmission(String queryIdentifier, NiFiUser user) {
|
||||
throw new UnsupportedOperationException("MockProvenanceEventRepository does not support querying");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ComputeLineageSubmission submitLineageComputation(String flowFileUuid) {
|
||||
public ComputeLineageSubmission submitLineageComputation(String flowFileUuid, NiFiUser user) {
|
||||
throw new UnsupportedOperationException("MockProvenanceEventRepository does not support Lineage Computation");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ComputeLineageSubmission retrieveLineageSubmission(String lineageIdentifier) {
|
||||
public ComputeLineageSubmission retrieveLineageSubmission(String lineageIdentifier, NiFiUser user) {
|
||||
throw new UnsupportedOperationException("MockProvenanceEventRepository does not support Lineage Computation");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ProvenanceEventRecord getEvent(long id) throws IOException {
|
||||
public ProvenanceEventRecord getEvent(long id, NiFiUser user) throws IOException {
|
||||
if (id > records.size()) {
|
||||
return null;
|
||||
}
|
||||
|
@ -101,12 +109,12 @@ public class MockProvenanceEventRepository implements ProvenanceEventRepository
|
|||
}
|
||||
|
||||
@Override
|
||||
public ComputeLineageSubmission submitExpandParents(long eventId) {
|
||||
public ComputeLineageSubmission submitExpandParents(long eventId, NiFiUser user) {
|
||||
throw new UnsupportedOperationException("MockProvenanceEventRepository does not support Lineage Computation");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ComputeLineageSubmission submitExpandChildren(long eventId) {
|
||||
public ComputeLineageSubmission submitExpandChildren(long eventId, NiFiUser user) {
|
||||
throw new UnsupportedOperationException("MockProvenanceEventRepository does not support Lineage Computation");
|
||||
}
|
||||
|
||||
|
|
|
@ -30,7 +30,6 @@ public class LineageDTO {
|
|||
|
||||
private String id;
|
||||
private String uri;
|
||||
private String clusterNodeId;
|
||||
|
||||
private Date submissionTime;
|
||||
private Date expiration;
|
||||
|
@ -68,20 +67,6 @@ public class LineageDTO {
|
|||
this.uri = uri;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return id of the node in the cluster where this lineage originated
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The id of the node where this lineage originated if clustered."
|
||||
)
|
||||
public String getClusterNodeId() {
|
||||
return clusterNodeId;
|
||||
}
|
||||
|
||||
public void setClusterNodeId(String clusterNodeId) {
|
||||
this.clusterNodeId = clusterNodeId;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return submission time for this lineage
|
||||
*/
|
||||
|
|
|
@ -41,6 +41,7 @@ public class LineageRequestDTO {
|
|||
private LineageRequestType lineageRequestType;
|
||||
|
||||
private String uuid;
|
||||
private String clusterNodeId;
|
||||
|
||||
/**
|
||||
* @return event id that was used to generate this lineage
|
||||
|
@ -73,6 +74,18 @@ public class LineageRequestDTO {
|
|||
this.lineageRequestType = lineageRequestType;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return id of the node in the cluster where this lineage originated
|
||||
*/
|
||||
@ApiModelProperty(value = "The id of the node where this lineage originated if clustered.")
|
||||
public String getClusterNodeId() {
|
||||
return clusterNodeId;
|
||||
}
|
||||
|
||||
public void setClusterNodeId(String clusterNodeId) {
|
||||
this.clusterNodeId = clusterNodeId;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return uuid that was used to generate this lineage
|
||||
*/
|
||||
|
|
|
@ -0,0 +1,41 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.authorization.resource;
|
||||
|
||||
import org.apache.nifi.authorization.Resource;
|
||||
|
||||
public class ProvenanceEventAuthorizable implements Authorizable {
|
||||
final Authorizable authorizable;
|
||||
|
||||
public ProvenanceEventAuthorizable(final Authorizable authorizable) {
|
||||
this.authorizable = authorizable;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Authorizable getParentAuthorizable() {
|
||||
if (authorizable.getParentAuthorizable() == null) {
|
||||
return null;
|
||||
} else {
|
||||
return new ProvenanceEventAuthorizable(authorizable.getParentAuthorizable());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Resource getResource() {
|
||||
return ResourceFactory.getProvenanceEventResource(authorizable.getResource());
|
||||
}
|
||||
}
|
|
@ -179,6 +179,18 @@ public final class ResourceFactory {
|
|||
}
|
||||
};
|
||||
|
||||
private final static Resource PROVENANCE_EVENT_RESOURCE = new Resource() {
|
||||
@Override
|
||||
public String getIdentifier() {
|
||||
return ResourceType.ProvenanceEvent.getValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return "Provenance Event";
|
||||
}
|
||||
};
|
||||
|
||||
private final static Resource PROXY_RESOURCE = new Resource() {
|
||||
@Override
|
||||
public String getIdentifier() {
|
||||
|
@ -581,22 +593,19 @@ public final class ResourceFactory {
|
|||
/**
|
||||
* Gets a Resource for accessing a component's provenance events.
|
||||
*
|
||||
* @param resourceType The type of resource being accessed
|
||||
* @param identifier The identifier of the component being accessed
|
||||
* @param name The name of the component being accessed
|
||||
* @return The resource
|
||||
* @param resource The resource for the component being accessed
|
||||
* @return The resource for the provenance of the component being accessed
|
||||
*/
|
||||
public static Resource getComponentProvenanceResource(final ResourceType resourceType, final String identifier, final String name) {
|
||||
final Resource componentResource = getComponentResource(resourceType, identifier, name);
|
||||
public static Resource getProvenanceEventResource(final Resource resource) {
|
||||
return new Resource() {
|
||||
@Override
|
||||
public String getIdentifier() {
|
||||
return String.format("%s/%s", componentResource.getIdentifier(), "provenance");
|
||||
return String.format("%s%s", PROVENANCE_EVENT_RESOURCE.getIdentifier(), resource.getIdentifier());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return componentResource.getName() + " provenance";
|
||||
return "Provenance Events for " + resource.getName();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ public enum ResourceType {
|
|||
Processor("/processors"),
|
||||
ProcessGroup("/process-groups"),
|
||||
Provenance("/provenance"),
|
||||
ProvenanceEvent("/provenance-events"),
|
||||
Proxy("/proxy"),
|
||||
RemoteProcessGroup("/remote-process-groups"),
|
||||
ReportingTask("/reporting-tasks"),
|
||||
|
|
|
@ -20,45 +20,48 @@ import java.io.Serializable;
|
|||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* An NiFiUser.
|
||||
* An implementation of NiFiUser.
|
||||
*/
|
||||
public class NiFiUser implements Serializable {
|
||||
public class StandardNiFiUser implements NiFiUser, Serializable {
|
||||
private static final long serialVersionUID = -5503790026187817496L;
|
||||
|
||||
public static final NiFiUser ANONYMOUS = new NiFiUser("anonymous");
|
||||
public static final StandardNiFiUser ANONYMOUS = new StandardNiFiUser("anonymous");
|
||||
|
||||
private String identity;
|
||||
private String userName;
|
||||
private final String identity;
|
||||
private final String userName;
|
||||
private final NiFiUser chain;
|
||||
|
||||
private NiFiUser chain;
|
||||
|
||||
public NiFiUser(String identity) {
|
||||
public StandardNiFiUser(String identity) {
|
||||
this(identity, identity, null);
|
||||
}
|
||||
|
||||
public NiFiUser(String identity, NiFiUser chain) {
|
||||
public StandardNiFiUser(String identity, NiFiUser chain) {
|
||||
this(identity, identity, chain);
|
||||
}
|
||||
|
||||
public NiFiUser(String identity, String userName, NiFiUser chain) {
|
||||
public StandardNiFiUser(String identity, String userName, NiFiUser chain) {
|
||||
this.identity = identity;
|
||||
this.userName = userName;
|
||||
this.chain = chain;
|
||||
}
|
||||
|
||||
/* getters / setters */
|
||||
|
||||
@Override
|
||||
public String getIdentity() {
|
||||
return identity;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getUserName() {
|
||||
return userName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public NiFiUser getChain() {
|
||||
return chain;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAnonymous() {
|
||||
return this == ANONYMOUS;
|
||||
}
|
||||
|
@ -68,14 +71,13 @@ public class NiFiUser implements Serializable {
|
|||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != obj.getClass()) {
|
||||
|
||||
if (!(obj instanceof NiFiUser)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final NiFiUser other = (NiFiUser) obj;
|
||||
if (!Objects.equals(this.identity, other.identity)) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
return Objects.equals(this.identity, other.getIdentity());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -89,5 +91,4 @@ public class NiFiUser implements Serializable {
|
|||
public String toString() {
|
||||
return String.format("identity[%s], userName[%s]", getIdentity(), getUserName(), ", ");
|
||||
}
|
||||
|
||||
}
|
|
@ -30,7 +30,6 @@ import org.apache.nifi.authorization.resource.Authorizable;
|
|||
import org.apache.nifi.authorization.resource.ResourceFactory;
|
||||
import org.apache.nifi.authorization.resource.ResourceType;
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.authorization.user.NiFiUserUtils;
|
||||
import org.apache.nifi.connectable.Connection;
|
||||
import org.apache.nifi.controller.label.Label;
|
||||
import org.apache.nifi.groups.ProcessGroup;
|
||||
|
@ -139,8 +138,8 @@ public class Template implements Authorizable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void authorize(final Authorizer authorizer, final RequestAction action) throws AccessDeniedException {
|
||||
final AuthorizationResult result = checkAuthorization(authorizer, action, true);
|
||||
public void authorize(final Authorizer authorizer, final RequestAction action, final NiFiUser user) throws AccessDeniedException {
|
||||
final AuthorizationResult result = checkAuthorization(authorizer, action, true, user);
|
||||
if (Result.Denied.equals(result)) {
|
||||
final String explanation = result.getExplanation() == null ? "Access is denied" : result.getExplanation();
|
||||
throw new AccessDeniedException(explanation);
|
||||
|
@ -148,13 +147,11 @@ public class Template implements Authorizable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public AuthorizationResult checkAuthorization(final Authorizer authorizer, final RequestAction action) {
|
||||
return checkAuthorization(authorizer, action, false);
|
||||
public AuthorizationResult checkAuthorization(final Authorizer authorizer, final RequestAction action, final NiFiUser user) {
|
||||
return checkAuthorization(authorizer, action, false, user);
|
||||
}
|
||||
|
||||
private AuthorizationResult checkAuthorization(final Authorizer authorizer, final RequestAction action, final boolean accessAttempt) {
|
||||
final NiFiUser user = NiFiUserUtils.getNiFiUser();
|
||||
|
||||
private AuthorizationResult checkAuthorization(final Authorizer authorizer, final RequestAction action, final boolean accessAttempt, final NiFiUser user) {
|
||||
// TODO - include user details context
|
||||
|
||||
// build the request
|
||||
|
@ -172,7 +169,7 @@ public class Template implements Authorizable {
|
|||
// verify the results
|
||||
if (Result.ResourceNotFound.equals(result.getResult())) {
|
||||
for (final Authorizable child : getAuthorizableComponents()) {
|
||||
final AuthorizationResult childResult = child.checkAuthorization(authorizer, action);
|
||||
final AuthorizationResult childResult = child.checkAuthorization(authorizer, action, user);
|
||||
if (Result.Denied.equals(childResult)) {
|
||||
return childResult;
|
||||
}
|
||||
|
|
|
@ -16,7 +16,39 @@
|
|||
*/
|
||||
package org.apache.nifi.controller;
|
||||
|
||||
import com.sun.jersey.api.client.ClientHandlerException;
|
||||
import static java.util.Objects.requireNonNull;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.LockSupport;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.action.Action;
|
||||
import org.apache.nifi.admin.service.AuditService;
|
||||
|
@ -29,7 +61,9 @@ import org.apache.nifi.annotation.notification.PrimaryNodeState;
|
|||
import org.apache.nifi.authorization.Authorizer;
|
||||
import org.apache.nifi.authorization.Resource;
|
||||
import org.apache.nifi.authorization.resource.Authorizable;
|
||||
import org.apache.nifi.authorization.resource.ProvenanceEventAuthorizable;
|
||||
import org.apache.nifi.authorization.resource.ResourceFactory;
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.cluster.HeartbeatPayload;
|
||||
import org.apache.nifi.cluster.coordination.heartbeat.HeartbeatMonitor;
|
||||
import org.apache.nifi.cluster.coordination.node.ClusterRoles;
|
||||
|
@ -151,6 +185,7 @@ import org.apache.nifi.processor.StandardValidationContextFactory;
|
|||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRepository;
|
||||
import org.apache.nifi.provenance.ProvenanceEventType;
|
||||
import org.apache.nifi.provenance.ProvenanceAuthorizableFactory;
|
||||
import org.apache.nifi.provenance.StandardProvenanceEventRecord;
|
||||
import org.apache.nifi.remote.HttpRemoteSiteListener;
|
||||
import org.apache.nifi.remote.RemoteGroupPort;
|
||||
|
@ -178,6 +213,7 @@ import org.apache.nifi.stream.io.StreamUtils;
|
|||
import org.apache.nifi.util.FormatUtils;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.nifi.util.ReflectionUtils;
|
||||
import org.apache.nifi.web.ResourceNotFoundException;
|
||||
import org.apache.nifi.web.api.dto.ConnectableDTO;
|
||||
import org.apache.nifi.web.api.dto.ConnectionDTO;
|
||||
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
|
||||
|
@ -198,39 +234,9 @@ import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.LockSupport;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
import com.sun.jersey.api.client.ClientHandlerException;
|
||||
|
||||
import static java.util.Objects.requireNonNull;
|
||||
|
||||
public class FlowController implements EventAccess, ControllerServiceProvider, ReportingTaskProvider, QueueProvider, Authorizable {
|
||||
public class FlowController implements EventAccess, ControllerServiceProvider, ReportingTaskProvider, QueueProvider, Authorizable, ProvenanceAuthorizableFactory {
|
||||
|
||||
// default repository implementations
|
||||
public static final String DEFAULT_FLOWFILE_REPO_IMPLEMENTATION = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository";
|
||||
|
@ -437,7 +443,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
|
||||
try {
|
||||
this.provenanceEventRepository = createProvenanceRepository(properties);
|
||||
this.provenanceEventRepository.initialize(createEventReporter(bulletinRepository));
|
||||
this.provenanceEventRepository.initialize(createEventReporter(bulletinRepository), authorizer, this);
|
||||
} catch (final Exception e) {
|
||||
throw new RuntimeException("Unable to create Provenance Repository", e);
|
||||
}
|
||||
|
@ -3575,17 +3581,17 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
return null;
|
||||
}
|
||||
|
||||
public ProvenanceEventRecord replayFlowFile(final long provenanceEventRecordId, final String requestor) throws IOException {
|
||||
final ProvenanceEventRecord record = provenanceEventRepository.getEvent(provenanceEventRecordId);
|
||||
public ProvenanceEventRecord replayFlowFile(final long provenanceEventRecordId, final NiFiUser user) throws IOException {
|
||||
final ProvenanceEventRecord record = provenanceEventRepository.getEvent(provenanceEventRecordId, user);
|
||||
if (record == null) {
|
||||
throw new IllegalStateException("Cannot find Provenance Event with ID " + provenanceEventRecordId);
|
||||
}
|
||||
|
||||
return replayFlowFile(record, requestor);
|
||||
return replayFlowFile(record, user);
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
public ProvenanceEventRecord replayFlowFile(final ProvenanceEventRecord event, final String requestor) throws IOException {
|
||||
public ProvenanceEventRecord replayFlowFile(final ProvenanceEventRecord event, final NiFiUser user) throws IOException {
|
||||
if (event == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
|
@ -3684,7 +3690,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
.setFlowFileUUID(parentUUID)
|
||||
.setAttributes(Collections.<String, String> emptyMap(), flowFileRecord.getAttributes())
|
||||
.setCurrentContentClaim(event.getContentClaimContainer(), event.getContentClaimSection(), event.getContentClaimIdentifier(), event.getContentClaimOffset(), event.getFileSize())
|
||||
.setDetails("Replay requested by " + requestor)
|
||||
.setDetails("Replay requested by " + user.getIdentity())
|
||||
.setEventTime(System.currentTimeMillis())
|
||||
.setFlowFileEntryDate(System.currentTimeMillis())
|
||||
.setLineageStartDate(event.getLineageStartDate())
|
||||
|
@ -3846,6 +3852,29 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
return new ArrayList<>(provenanceEventRepository.getEvents(firstEventId, maxRecords));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Authorizable createProvenanceAuthorizable(final String componentId) {
|
||||
final String rootGroupId = getRootGroupId();
|
||||
|
||||
// Provenance Events are generated only by connectable components, with the exception of DOWNLOAD events,
|
||||
// which have the root process group's identifier assigned as the component ID. So, we check if the component ID
|
||||
// is set to the root group and otherwise assume that the ID is that of a component.
|
||||
final ProvenanceEventAuthorizable authorizable;
|
||||
if (rootGroupId.equals(componentId)) {
|
||||
authorizable = new ProvenanceEventAuthorizable(rootGroup);
|
||||
} else {
|
||||
final Connectable connectable = rootGroup.findConnectable(componentId);
|
||||
|
||||
if (connectable == null) {
|
||||
throw new ResourceNotFoundException("The component that generated this event is no longer part of the data flow.");
|
||||
}
|
||||
|
||||
authorizable = new ProvenanceEventAuthorizable(connectable);
|
||||
}
|
||||
|
||||
return authorizable;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Action> getFlowChanges(final int firstActionId, final int maxActions) {
|
||||
final History history = auditService.getActions(firstActionId, maxActions);
|
||||
|
|
|
@ -2812,7 +2812,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
}
|
||||
|
||||
// perform the authorization
|
||||
final AuthorizationResult result = authorizable.checkAuthorization(authorizer, RequestAction.READ);
|
||||
final AuthorizationResult result = authorizable.checkAuthorization(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
return Result.Approved.equals(result.getResult());
|
||||
}
|
||||
|
||||
|
|
|
@ -134,7 +134,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable authorizable = lookup.getProcessor(requestContext.getId());
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
componentType = Component.Processor;
|
||||
|
@ -143,7 +143,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable authorizable = lookup.getControllerService(requestContext.getId());
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
componentType = Component.ControllerService;
|
||||
|
@ -152,7 +152,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable authorizable = lookup.getReportingTask(requestContext.getId());
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
componentType = Component.ReportingTask;
|
||||
|
@ -322,7 +322,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable authorizable = lookup.getProcessor(id);
|
||||
authorizable.authorize(authorizer, RequestAction.READ);
|
||||
authorizable.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
final ProcessorDTO processor;
|
||||
|
@ -374,7 +374,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable authorizable = lookup.getProcessor(id);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
final ProcessorDTO processor;
|
||||
|
@ -510,7 +510,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable authorizable = lookup.getControllerService(id);
|
||||
authorizable.authorize(authorizer, RequestAction.READ);
|
||||
authorizable.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// if the lookup has the service that means we are either a node or
|
||||
|
@ -568,7 +568,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable authorizable = lookup.getControllerService(id);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
final ControllerServiceDTO controllerService;
|
||||
|
@ -678,7 +678,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable authorizable = lookup.getReportingTask(id);
|
||||
authorizable.authorize(authorizer, RequestAction.READ);
|
||||
authorizable.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// if the provider has the service that means we are either a node or
|
||||
|
@ -736,7 +736,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable authorizable = lookup.getReportingTask(id);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
final ReportingTaskDTO reportingTask;
|
||||
|
|
|
@ -26,6 +26,7 @@ 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.NiFiUserUtils;
|
||||
import org.apache.nifi.cluster.coordination.ClusterCoordinator;
|
||||
import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
|
@ -205,7 +206,7 @@ public class AccessPolicyResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable accessPolicies = lookup.getAccessPoliciesAuthorizable();
|
||||
accessPolicies.authorize(authorizer, RequestAction.WRITE);
|
||||
accessPolicies.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
@ -270,7 +271,7 @@ public class AccessPolicyResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable accessPolicy = lookup.getAccessPolicyAuthorizable(id);
|
||||
accessPolicy.authorize(authorizer, RequestAction.READ);
|
||||
accessPolicy.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the access policy
|
||||
|
@ -347,7 +348,7 @@ public class AccessPolicyResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
Authorizable authorizable = lookup.getAccessPolicyAuthorizable(id);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
null,
|
||||
() -> {
|
||||
|
@ -422,7 +423,7 @@ public class AccessPolicyResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
final Authorizable accessPolicy = lookup.getAccessPolicyAuthorizable(id);
|
||||
accessPolicy.authorize(authorizer, RequestAction.READ);
|
||||
accessPolicy.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> {
|
||||
},
|
||||
|
|
|
@ -424,7 +424,7 @@ public abstract class ApplicationResource {
|
|||
* @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);
|
||||
final Consumer<Authorizable> authorize = authorizable -> authorizable.authorize(authorizer, action, NiFiUserUtils.getNiFiUser());
|
||||
|
||||
snippet.getProcessGroups().keySet().stream().map(id -> lookup.getProcessGroup(id)).forEach(authorize);
|
||||
snippet.getRemoteProcessGroups().keySet().stream().map(id -> lookup.getRemoteProcessGroup(id)).forEach(authorize);
|
||||
|
@ -444,7 +444,7 @@ public abstract class ApplicationResource {
|
|||
* @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);
|
||||
final Consumer<Authorizable> authorize = authorizable -> authorizable.authorize(authorizer, action, NiFiUserUtils.getNiFiUser());
|
||||
|
||||
snippet.getProcessGroups().keySet().stream().map(id -> lookup.getProcessGroup(id)).forEach(authorize);
|
||||
snippet.getRemoteProcessGroups().keySet().stream().map(id -> lookup.getRemoteProcessGroup(id)).forEach(authorize);
|
||||
|
|
|
@ -26,6 +26,7 @@ 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.NiFiUserUtils;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
import org.apache.nifi.web.Revision;
|
||||
import org.apache.nifi.web.api.dto.ConnectionDTO;
|
||||
|
@ -191,7 +192,7 @@ public class ConnectionResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable conn = lookup.getConnection(id);
|
||||
conn.authorize(authorizer, RequestAction.READ);
|
||||
conn.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the specified relationship
|
||||
|
@ -270,7 +271,7 @@ public class ConnectionResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
Authorizable authorizable = lookup.getConnection(id);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyUpdateConnection(connection),
|
||||
() -> {
|
||||
|
@ -345,7 +346,7 @@ public class ConnectionResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
final Authorizable conn = lookup.getConnection(id);
|
||||
conn.authorize(authorizer, RequestAction.WRITE);
|
||||
conn.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyDeleteConnection(id),
|
||||
() -> {
|
||||
|
|
|
@ -16,16 +16,34 @@
|
|||
*/
|
||||
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 java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import javax.servlet.ServletContext;
|
||||
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.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.core.Context;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.ws.rs.core.Response;
|
||||
|
||||
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.NiFiUserUtils;
|
||||
import org.apache.nifi.controller.ScheduledState;
|
||||
import org.apache.nifi.controller.service.ControllerServiceState;
|
||||
import org.apache.nifi.ui.extension.UiExtension;
|
||||
|
@ -47,27 +65,12 @@ import org.apache.nifi.web.api.request.LongParameter;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.servlet.ServletContext;
|
||||
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.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.core.Context;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.ws.rs.core.Response;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
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;
|
||||
|
||||
/**
|
||||
* RESTful endpoint for managing a Controller Service.
|
||||
|
@ -192,7 +195,7 @@ public class ControllerServiceResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable controllerService = lookup.getControllerService(id);
|
||||
controllerService.authorize(authorizer, RequestAction.READ);
|
||||
controllerService.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the controller service
|
||||
|
@ -256,7 +259,7 @@ public class ControllerServiceResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable controllerService = lookup.getControllerService(id);
|
||||
controllerService.authorize(authorizer, RequestAction.READ);
|
||||
controllerService.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the property descriptor
|
||||
|
@ -311,7 +314,7 @@ public class ControllerServiceResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable controllerService = lookup.getControllerService(id);
|
||||
controllerService.authorize(authorizer, RequestAction.WRITE);
|
||||
controllerService.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the component state
|
||||
|
@ -370,7 +373,7 @@ public class ControllerServiceResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processor = lookup.getControllerService(id);
|
||||
processor.authorize(authorizer, RequestAction.WRITE);
|
||||
processor.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
@ -431,7 +434,7 @@ public class ControllerServiceResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable controllerService = lookup.getControllerService(id);
|
||||
controllerService.authorize(authorizer, RequestAction.READ);
|
||||
controllerService.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the controller service
|
||||
|
@ -540,7 +543,7 @@ public class ControllerServiceResource extends ApplicationResource {
|
|||
lookup -> {
|
||||
referencingRevisions.entrySet().stream().forEach(e -> {
|
||||
final Authorizable controllerService = lookup.getControllerServiceReferencingComponent(id, e.getKey());
|
||||
controllerService.authorize(authorizer, RequestAction.WRITE);
|
||||
controllerService.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
},
|
||||
() -> serviceFacade.verifyUpdateControllerServiceReferencingComponents(updateReferenceRequest.getId(), scheduledState, controllerServiceState),
|
||||
|
@ -621,7 +624,7 @@ public class ControllerServiceResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
Authorizable authorizable = lookup.getControllerService(id);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyUpdateControllerService(requestControllerServiceDTO),
|
||||
() -> {
|
||||
|
@ -696,7 +699,7 @@ public class ControllerServiceResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
final Authorizable controllerService = lookup.getControllerService(id);
|
||||
controllerService.authorize(authorizer, RequestAction.WRITE);
|
||||
controllerService.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyDeleteControllerService(id),
|
||||
() -> {
|
||||
|
|
|
@ -26,6 +26,7 @@ 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.NiFiUserUtils;
|
||||
import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
|
@ -177,7 +178,7 @@ public class FlowFileQueueResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable connection = lookup.getConnection(connectionId);
|
||||
connection.authorize(authorizer, RequestAction.WRITE);
|
||||
connection.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the flowfile
|
||||
|
@ -265,7 +266,7 @@ public class FlowFileQueueResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable connection = lookup.getConnection(connectionId);
|
||||
connection.authorize(authorizer, RequestAction.WRITE);
|
||||
connection.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the uri of the request
|
||||
|
@ -344,7 +345,7 @@ public class FlowFileQueueResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable connection = lookup.getConnection(id);
|
||||
connection.authorize(authorizer, RequestAction.WRITE);
|
||||
connection.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
@ -415,7 +416,7 @@ public class FlowFileQueueResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable connection = lookup.getConnection(connectionId);
|
||||
connection.authorize(authorizer, RequestAction.WRITE);
|
||||
connection.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the listing request
|
||||
|
@ -481,7 +482,7 @@ public class FlowFileQueueResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable connection = lookup.getConnection(connectionId);
|
||||
connection.authorize(authorizer, RequestAction.WRITE);
|
||||
connection.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
@ -551,7 +552,7 @@ public class FlowFileQueueResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable connection = lookup.getConnection(id);
|
||||
connection.authorize(authorizer, RequestAction.WRITE);
|
||||
connection.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
@ -621,7 +622,7 @@ public class FlowFileQueueResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable connection = lookup.getConnection(connectionId);
|
||||
connection.authorize(authorizer, RequestAction.WRITE);
|
||||
connection.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the drop request
|
||||
|
@ -687,7 +688,7 @@ public class FlowFileQueueResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable connection = lookup.getConnection(connectionId);
|
||||
connection.authorize(authorizer, RequestAction.WRITE);
|
||||
connection.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
|
|
@ -16,13 +16,29 @@
|
|||
*/
|
||||
package org.apache.nifi.web.api;
|
||||
|
||||
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 java.util.Arrays;
|
||||
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;
|
||||
|
||||
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 org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.authorization.AccessDeniedException;
|
||||
import org.apache.nifi.authorization.AuthorizationRequest;
|
||||
|
@ -96,27 +112,13 @@ import org.apache.nifi.web.api.request.DateTimeParameter;
|
|||
import org.apache.nifi.web.api.request.IntegerParameter;
|
||||
import org.apache.nifi.web.api.request.LongParameter;
|
||||
|
||||
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.Arrays;
|
||||
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;
|
||||
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;
|
||||
|
||||
/**
|
||||
* RESTful endpoint for managing a Flow.
|
||||
|
@ -661,7 +663,7 @@ public class FlowResource extends ApplicationResource {
|
|||
// 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))
|
||||
.filter(processor -> processor.isAuthorized(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()))
|
||||
.forEach(processor -> {
|
||||
componentIds.add(processor.getIdentifier());
|
||||
});
|
||||
|
@ -669,7 +671,7 @@ public class FlowResource extends ApplicationResource {
|
|||
// 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))
|
||||
.filter(inputPort -> inputPort.isAuthorized(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()))
|
||||
.forEach(inputPort -> {
|
||||
componentIds.add(inputPort.getIdentifier());
|
||||
});
|
||||
|
@ -677,7 +679,7 @@ public class FlowResource extends ApplicationResource {
|
|||
// 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))
|
||||
.filter(outputPort -> outputPort.isAuthorized(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()))
|
||||
.forEach(outputPort -> {
|
||||
componentIds.add(outputPort.getIdentifier());
|
||||
});
|
||||
|
@ -713,7 +715,7 @@ public class FlowResource extends ApplicationResource {
|
|||
// ensure access to every component being scheduled
|
||||
componentsToSchedule.keySet().forEach(componentId -> {
|
||||
final Authorizable connectable = lookup.getConnectable(componentId);
|
||||
connectable.authorize(authorizer, RequestAction.WRITE);
|
||||
connectable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
},
|
||||
() -> serviceFacade.verifyScheduleComponents(id, state, componentRevisions.keySet()),
|
||||
|
|
|
@ -26,6 +26,7 @@ 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.NiFiUserUtils;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
import org.apache.nifi.web.Revision;
|
||||
import org.apache.nifi.web.api.dto.FunnelDTO;
|
||||
|
@ -153,7 +154,7 @@ public class FunnelResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable funnel = lookup.getFunnel(id);
|
||||
funnel.authorize(authorizer, RequestAction.READ);
|
||||
funnel.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the funnel
|
||||
|
@ -230,7 +231,7 @@ public class FunnelResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
Authorizable authorizable = lookup.getFunnel(id);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
null,
|
||||
() -> {
|
||||
|
@ -305,7 +306,7 @@ public class FunnelResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
final Authorizable funnel = lookup.getFunnel(id);
|
||||
funnel.authorize(authorizer, RequestAction.WRITE);
|
||||
funnel.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyDeleteFunnel(id),
|
||||
() -> {
|
||||
|
|
|
@ -26,6 +26,7 @@ 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.NiFiUserUtils;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
import org.apache.nifi.web.Revision;
|
||||
import org.apache.nifi.web.api.dto.PortDTO;
|
||||
|
@ -153,7 +154,7 @@ public class InputPortResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable inputPort = lookup.getInputPort(id);
|
||||
inputPort.authorize(authorizer, RequestAction.READ);
|
||||
inputPort.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the port
|
||||
|
@ -230,7 +231,7 @@ public class InputPortResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
Authorizable authorizable = lookup.getInputPort(id);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyUpdateInputPort(requestPortDTO),
|
||||
() -> {
|
||||
|
@ -302,7 +303,7 @@ public class InputPortResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
final Authorizable inputPort = lookup.getInputPort(id);
|
||||
inputPort.authorize(authorizer, RequestAction.WRITE);
|
||||
inputPort.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyDeleteInputPort(id),
|
||||
() -> {
|
||||
|
|
|
@ -26,6 +26,7 @@ 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.NiFiUserUtils;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
import org.apache.nifi.web.Revision;
|
||||
import org.apache.nifi.web.api.dto.LabelDTO;
|
||||
|
@ -153,7 +154,7 @@ public class LabelResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable label = lookup.getLabel(id);
|
||||
label.authorize(authorizer, RequestAction.READ);
|
||||
label.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the label
|
||||
|
@ -230,7 +231,7 @@ public class LabelResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
Authorizable authorizable = lookup.getLabel(id);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
null,
|
||||
() -> {
|
||||
|
@ -302,7 +303,7 @@ public class LabelResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
final Authorizable label = lookup.getLabel(id);
|
||||
label.authorize(authorizer, RequestAction.WRITE);
|
||||
label.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
null,
|
||||
() -> {
|
||||
|
|
|
@ -26,6 +26,7 @@ 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.NiFiUserUtils;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
import org.apache.nifi.web.Revision;
|
||||
import org.apache.nifi.web.api.dto.PortDTO;
|
||||
|
@ -153,7 +154,7 @@ public class OutputPortResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable outputPort = lookup.getOutputPort(id);
|
||||
outputPort.authorize(authorizer, RequestAction.READ);
|
||||
outputPort.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the port
|
||||
|
@ -230,7 +231,7 @@ public class OutputPortResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
Authorizable authorizable = lookup.getOutputPort(id);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyUpdateOutputPort(requestPortDTO),
|
||||
() -> {
|
||||
|
@ -302,7 +303,7 @@ public class OutputPortResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
final Authorizable outputPort = lookup.getOutputPort(id);
|
||||
outputPort.authorize(authorizer, RequestAction.WRITE);
|
||||
outputPort.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyDeleteOutputPort(id),
|
||||
() -> {
|
||||
|
|
|
@ -16,18 +16,41 @@
|
|||
*/
|
||||
package org.apache.nifi.web.api;
|
||||
|
||||
import com.sun.jersey.api.core.ResourceContext;
|
||||
import com.sun.jersey.multipart.FormDataParam;
|
||||
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.io.InputStream;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
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.DefaultValue;
|
||||
import javax.ws.rs.GET;
|
||||
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.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.xml.bind.JAXBContext;
|
||||
import javax.xml.bind.JAXBElement;
|
||||
import javax.xml.bind.JAXBException;
|
||||
import javax.xml.bind.Unmarshaller;
|
||||
import javax.xml.transform.stream.StreamSource;
|
||||
|
||||
import 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.NiFiUserUtils;
|
||||
import org.apache.nifi.controller.Snippet;
|
||||
import org.apache.nifi.web.AuthorizableLookup;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
|
@ -65,34 +88,14 @@ 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;
|
||||
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.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 javax.xml.bind.JAXBContext;
|
||||
import javax.xml.bind.JAXBElement;
|
||||
import javax.xml.bind.JAXBException;
|
||||
import javax.xml.bind.Unmarshaller;
|
||||
import javax.xml.transform.stream.StreamSource;
|
||||
import java.io.InputStream;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import com.sun.jersey.api.core.ResourceContext;
|
||||
import com.sun.jersey.multipart.FormDataParam;
|
||||
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;
|
||||
|
||||
/**
|
||||
* RESTful endpoint for managing a Group.
|
||||
|
@ -237,7 +240,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.READ);
|
||||
processGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get this process group contents
|
||||
|
@ -318,7 +321,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
Authorizable authorizable = lookup.getProcessGroup(id);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
null,
|
||||
() -> {
|
||||
|
@ -391,7 +394,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(id);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyDeleteProcessGroup(id),
|
||||
() -> {
|
||||
|
@ -473,7 +476,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
@ -535,7 +538,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.READ);
|
||||
processGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the process groups
|
||||
|
@ -633,7 +636,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
@ -696,7 +699,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.READ);
|
||||
processGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the processors
|
||||
|
@ -783,7 +786,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
@ -844,7 +847,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.READ);
|
||||
processGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get all the input ports
|
||||
|
@ -930,7 +933,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
@ -991,7 +994,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.READ);
|
||||
processGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get all the output ports
|
||||
|
@ -1078,7 +1081,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
@ -1139,7 +1142,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.READ);
|
||||
processGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get all the funnels
|
||||
|
@ -1226,7 +1229,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
@ -1287,7 +1290,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.READ);
|
||||
processGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get all the labels
|
||||
|
@ -1380,7 +1383,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
@ -1472,7 +1475,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.READ);
|
||||
processGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get all the remote process groups
|
||||
|
@ -1571,7 +1574,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
@ -1634,7 +1637,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.READ);
|
||||
processGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// all of the relationships for the specified source processor
|
||||
|
@ -1803,10 +1806,10 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
|
||||
final Authorizable template = lookup.getTemplate(instantiateTemplateRequestEntity.getTemplateId());
|
||||
template.authorize(authorizer, RequestAction.READ);
|
||||
template.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
@ -1837,7 +1840,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
|
||||
private void authorizeSnippetUsage(final AuthorizableLookup lookup, final String groupId, final String snippetId) {
|
||||
// ensure write access to the target process group
|
||||
lookup.getProcessGroup(groupId).authorize(authorizer, RequestAction.WRITE);
|
||||
lookup.getProcessGroup(groupId).authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
|
||||
// ensure read permission to every component in the snippet
|
||||
final Snippet snippet = lookup.getSnippet(snippetId);
|
||||
|
@ -1885,7 +1888,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.READ);
|
||||
processGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get all the templates
|
||||
|
@ -2077,7 +2080,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
@ -2187,7 +2190,7 @@ public class ProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processGroup = lookup.getProcessGroup(groupId);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE);
|
||||
processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
|
|
@ -26,6 +26,7 @@ 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.NiFiUserUtils;
|
||||
import org.apache.nifi.ui.extension.UiExtension;
|
||||
import org.apache.nifi.ui.extension.UiExtensionMapping;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
|
@ -191,7 +192,7 @@ public class ProcessorResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processor = lookup.getProcessor(id);
|
||||
processor.authorize(authorizer, RequestAction.READ);
|
||||
processor.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the specified processor
|
||||
|
@ -262,7 +263,7 @@ public class ProcessorResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processor = lookup.getProcessor(id);
|
||||
processor.authorize(authorizer, RequestAction.READ);
|
||||
processor.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the property descriptor
|
||||
|
@ -318,7 +319,7 @@ public class ProcessorResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processor = lookup.getProcessor(id);
|
||||
processor.authorize(authorizer, RequestAction.WRITE);
|
||||
processor.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the component state
|
||||
|
@ -378,7 +379,7 @@ public class ProcessorResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processor = lookup.getProcessor(id);
|
||||
processor.authorize(authorizer, RequestAction.WRITE);
|
||||
processor.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (isValidationPhase) {
|
||||
|
@ -464,7 +465,7 @@ public class ProcessorResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
Authorizable authorizable = lookup.getProcessor(id);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyUpdateProcessor(requestProcessorDTO),
|
||||
() -> {
|
||||
|
@ -536,7 +537,7 @@ public class ProcessorResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
final Authorizable processor = lookup.getProcessor(id);
|
||||
processor.authorize(authorizer, RequestAction.WRITE);
|
||||
processor.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyDeleteProcessor(id),
|
||||
() -> {
|
||||
|
|
|
@ -780,14 +780,14 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
|
||||
// replicate if cluster manager
|
||||
if (isReplicateRequest()) {
|
||||
if (lineageDto.getClusterNodeId() == null) {
|
||||
if (requestDto.getClusterNodeId() == null) {
|
||||
throw new IllegalArgumentException("The cluster node identifier must be specified.");
|
||||
}
|
||||
|
||||
// change content type to JSON for serializing entity
|
||||
final Map<String, String> headersToOverride = new HashMap<>();
|
||||
headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
|
||||
return replicate(HttpMethod.POST, lineageEntity, lineageDto.getClusterNodeId(), headersToOverride);
|
||||
return replicate(HttpMethod.POST, lineageEntity, requestDto.getClusterNodeId(), headersToOverride);
|
||||
}
|
||||
|
||||
// handle expects request (usually from the cluster manager)
|
||||
|
@ -798,7 +798,7 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
|
||||
// get the provenance event
|
||||
final LineageDTO dto = serviceFacade.submitLineage(lineageDto);
|
||||
dto.setClusterNodeId(lineageDto.getClusterNodeId());
|
||||
dto.getRequest().setClusterNodeId(requestDto.getClusterNodeId());
|
||||
populateRemainingLineageContent(dto);
|
||||
|
||||
// create a response entity
|
||||
|
@ -858,7 +858,7 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
|
||||
// get the lineage
|
||||
final LineageDTO dto = serviceFacade.getLineage(id);
|
||||
dto.setClusterNodeId(clusterNodeId);
|
||||
dto.getRequest().setClusterNodeId(clusterNodeId);
|
||||
populateRemainingLineageContent(dto);
|
||||
|
||||
// create the response entity
|
||||
|
|
|
@ -26,6 +26,7 @@ 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.NiFiUserUtils;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
import org.apache.nifi.web.Revision;
|
||||
import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
|
||||
|
@ -169,7 +170,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable remoteProcessGroup = lookup.getRemoteProcessGroup(id);
|
||||
remoteProcessGroup.authorize(authorizer, RequestAction.READ);
|
||||
remoteProcessGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the remote process group
|
||||
|
@ -245,7 +246,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
final Authorizable remoteProcessGroup = lookup.getRemoteProcessGroup(id);
|
||||
remoteProcessGroup.authorize(authorizer, RequestAction.WRITE);
|
||||
remoteProcessGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyDeleteRemoteProcessGroup(id),
|
||||
() -> {
|
||||
|
@ -323,7 +324,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
final Authorizable remoteProcessGroupInputPort = lookup.getRemoteProcessGroupInputPort(id, portId);
|
||||
remoteProcessGroupInputPort.authorize(authorizer, RequestAction.WRITE);
|
||||
remoteProcessGroupInputPort.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyUpdateRemoteProcessGroupInputPort(id, requestRemoteProcessGroupPort),
|
||||
() -> {
|
||||
|
@ -412,7 +413,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
final Authorizable remoteProcessGroupOutputPort = lookup.getRemoteProcessGroupOutputPort(id, portId);
|
||||
remoteProcessGroupOutputPort.authorize(authorizer, RequestAction.WRITE);
|
||||
remoteProcessGroupOutputPort.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyUpdateRemoteProcessGroupOutputPort(id, requestRemoteProcessGroupPort),
|
||||
() -> {
|
||||
|
@ -492,7 +493,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
Authorizable authorizable = lookup.getRemoteProcessGroup(id);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyUpdateRemoteProcessGroup(requestRemoteProcessGroup),
|
||||
() -> {
|
||||
|
|
|
@ -26,6 +26,7 @@ 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.NiFiUserUtils;
|
||||
import org.apache.nifi.ui.extension.UiExtension;
|
||||
import org.apache.nifi.ui.extension.UiExtensionMapping;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
|
@ -180,7 +181,7 @@ public class ReportingTaskResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable reportingTask = lookup.getReportingTask(id);
|
||||
reportingTask.authorize(authorizer, RequestAction.READ);
|
||||
reportingTask.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the reporting task
|
||||
|
@ -244,7 +245,7 @@ public class ReportingTaskResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable reportingTask = lookup.getReportingTask(id);
|
||||
reportingTask.authorize(authorizer, RequestAction.READ);
|
||||
reportingTask.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the property descriptor
|
||||
|
@ -299,7 +300,7 @@ public class ReportingTaskResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable reportingTask = lookup.getReportingTask(id);
|
||||
reportingTask.authorize(authorizer, RequestAction.WRITE);
|
||||
reportingTask.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the component state
|
||||
|
@ -358,7 +359,7 @@ public class ReportingTaskResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable processor = lookup.getReportingTask(id);
|
||||
processor.authorize(authorizer, RequestAction.WRITE);
|
||||
processor.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (isValidationPhase) {
|
||||
|
@ -443,7 +444,7 @@ public class ReportingTaskResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
Authorizable authorizable = lookup.getReportingTask(id);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE);
|
||||
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyUpdateReportingTask(requestReportingTaskDTO),
|
||||
() -> {
|
||||
|
@ -518,7 +519,7 @@ public class ReportingTaskResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
final Authorizable reportingTask = lookup.getReportingTask(id);
|
||||
reportingTask.authorize(authorizer, RequestAction.WRITE);
|
||||
reportingTask.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
() -> serviceFacade.verifyDeleteReportingTask(id),
|
||||
() -> {
|
||||
|
|
|
@ -35,6 +35,7 @@ import javax.ws.rs.core.Response;
|
|||
|
||||
import org.apache.nifi.authorization.Authorizer;
|
||||
import org.apache.nifi.authorization.RequestAction;
|
||||
import org.apache.nifi.authorization.user.NiFiUserUtils;
|
||||
import org.apache.nifi.controller.Snippet;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
import org.apache.nifi.web.Revision;
|
||||
|
@ -228,7 +229,7 @@ public class SnippetResource extends ApplicationResource {
|
|||
lookup -> {
|
||||
// ensure write access to the target process group
|
||||
if (requestSnippetDTO.getParentGroupId() != null) {
|
||||
lookup.getProcessGroup(requestSnippetDTO.getParentGroupId()).authorize(authorizer, RequestAction.WRITE);
|
||||
lookup.getProcessGroup(requestSnippetDTO.getParentGroupId()).authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
}
|
||||
|
||||
// ensure read permission to every component in the snippet
|
||||
|
|
|
@ -34,6 +34,7 @@ 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.NiFiUserUtils;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
import org.apache.nifi.web.api.dto.TemplateDTO;
|
||||
import org.apache.nifi.web.api.entity.TemplateEntity;
|
||||
|
@ -123,7 +124,7 @@ public class TemplateResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable template = lookup.getTemplate(id);
|
||||
template.authorize(authorizer, RequestAction.READ);
|
||||
template.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the template
|
||||
|
@ -190,7 +191,7 @@ public class TemplateResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable template = lookup.getTemplate(id);
|
||||
template.authorize(authorizer, RequestAction.WRITE);
|
||||
template.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
return generateContinueResponse().build();
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ 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.NiFiUserUtils;
|
||||
import org.apache.nifi.cluster.coordination.ClusterCoordinator;
|
||||
import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
|
@ -182,7 +183,7 @@ public class TenantsResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable users = lookup.getTenantAuthorizable();
|
||||
users.authorize(authorizer, RequestAction.WRITE);
|
||||
users.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
@ -247,7 +248,7 @@ public class TenantsResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable users = lookup.getTenantAuthorizable();
|
||||
users.authorize(authorizer, RequestAction.READ);
|
||||
users.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the user
|
||||
|
@ -294,7 +295,7 @@ public class TenantsResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable users = lookup.getTenantAuthorizable();
|
||||
users.authorize(authorizer, RequestAction.READ);
|
||||
users.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get all the users
|
||||
|
@ -375,7 +376,7 @@ public class TenantsResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
final Authorizable users = lookup.getTenantAuthorizable();
|
||||
users.authorize(authorizer, RequestAction.WRITE);
|
||||
users.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
null,
|
||||
() -> {
|
||||
|
@ -450,7 +451,7 @@ public class TenantsResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
final Authorizable users = lookup.getTenantAuthorizable();
|
||||
users.authorize(authorizer, RequestAction.READ);
|
||||
users.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
null,
|
||||
() -> {
|
||||
|
@ -567,7 +568,7 @@ public class TenantsResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable userGroups = lookup.getTenantAuthorizable();
|
||||
userGroups.authorize(authorizer, RequestAction.WRITE);
|
||||
userGroups.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
}
|
||||
if (validationPhase) {
|
||||
|
@ -632,7 +633,7 @@ public class TenantsResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable userGroups = lookup.getTenantAuthorizable();
|
||||
userGroups.authorize(authorizer, RequestAction.READ);
|
||||
userGroups.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get the user group
|
||||
|
@ -679,7 +680,7 @@ public class TenantsResource extends ApplicationResource {
|
|||
// authorize access
|
||||
serviceFacade.authorizeAccess(lookup -> {
|
||||
final Authorizable userGroups = lookup.getTenantAuthorizable();
|
||||
userGroups.authorize(authorizer, RequestAction.READ);
|
||||
userGroups.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
});
|
||||
|
||||
// get all the user groups
|
||||
|
@ -760,7 +761,7 @@ public class TenantsResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
final Authorizable userGroups = lookup.getTenantAuthorizable();
|
||||
userGroups.authorize(authorizer, RequestAction.WRITE);
|
||||
userGroups.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
null,
|
||||
() -> {
|
||||
|
@ -835,7 +836,7 @@ public class TenantsResource extends ApplicationResource {
|
|||
revision,
|
||||
lookup -> {
|
||||
final Authorizable userGroups = lookup.getTenantAuthorizable();
|
||||
userGroups.authorize(authorizer, RequestAction.READ);
|
||||
userGroups.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
|
||||
},
|
||||
null,
|
||||
() -> {
|
||||
|
|
|
@ -49,17 +49,13 @@ public class AccessDeniedExceptionMapper implements ExceptionMapper<AccessDenied
|
|||
status = Status.FORBIDDEN;
|
||||
}
|
||||
|
||||
if (user != null) {
|
||||
logger.info(String.format("%s does not have permission to access the requested resource. Returning %s response.", user.getIdentity(), status));
|
||||
} else {
|
||||
logger.info(String.format("User does not have permission to access the requested resource. Returning %s response.", status));
|
||||
}
|
||||
logger.info(String.format("%s does not have permission to access the requested resource. Returning %s response.", user.getIdentity(), status));
|
||||
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug(StringUtils.EMPTY, exception);
|
||||
}
|
||||
|
||||
return Response.status(status).entity("Unable to perform the desired action. Contact the system administrator.").type("text/plain").build();
|
||||
return Response.status(status).entity("Unable to perform the desired action due to insufficient permissions. Contact the system administrator.").type("text/plain").build();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.nifi.authorization.RequestAction;
|
|||
import org.apache.nifi.authorization.Resource;
|
||||
import org.apache.nifi.authorization.User;
|
||||
import org.apache.nifi.authorization.resource.Authorizable;
|
||||
import org.apache.nifi.authorization.user.NiFiUserUtils;
|
||||
import org.apache.nifi.cluster.coordination.heartbeat.NodeHeartbeat;
|
||||
import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus;
|
||||
import org.apache.nifi.cluster.event.NodeEvent;
|
||||
|
@ -1579,8 +1580,8 @@ public final class DtoFactory {
|
|||
*/
|
||||
public AccessPolicyDTO createAccessPolicyDto(final Authorizable authorizable) {
|
||||
final AccessPolicyDTO dto = new AccessPolicyDTO();
|
||||
dto.setCanRead(authorizable.isAuthorized(authorizer, RequestAction.READ));
|
||||
dto.setCanWrite(authorizable.isAuthorized(authorizer, RequestAction.WRITE));
|
||||
dto.setCanRead(authorizable.isAuthorized(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()));
|
||||
dto.setCanWrite(authorizable.isAuthorized(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()));
|
||||
return dto;
|
||||
}
|
||||
|
||||
|
@ -2461,7 +2462,7 @@ public final class DtoFactory {
|
|||
final List<AllowableValueDTO> allowableValues = new ArrayList<>();
|
||||
for (final String serviceIdentifier : controllerServiceProvider.getControllerServiceIdentifiers(serviceDefinition, groupId)) {
|
||||
final ControllerServiceNode service = controllerServiceProvider.getControllerServiceNode(serviceIdentifier);
|
||||
final String displayName = service.isAuthorized(authorizer, RequestAction.READ) ? service.getName() : serviceIdentifier;
|
||||
final String displayName = service.isAuthorized(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()) ? service.getName() : serviceIdentifier;
|
||||
|
||||
final AllowableValueDTO allowableValue = new AllowableValueDTO();
|
||||
allowableValue.setDisplayName(displayName);
|
||||
|
|
|
@ -19,10 +19,6 @@ package org.apache.nifi.web.controller;
|
|||
import org.apache.commons.collections4.CollectionUtils;
|
||||
import org.apache.commons.lang3.ClassUtils;
|
||||
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;
|
||||
|
@ -31,6 +27,7 @@ import org.apache.nifi.authorization.resource.ResourceFactory;
|
|||
import org.apache.nifi.authorization.resource.ResourceType;
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.authorization.user.NiFiUserUtils;
|
||||
import org.apache.nifi.authorization.user.StandardNiFiUser;
|
||||
import org.apache.nifi.cluster.coordination.ClusterCoordinator;
|
||||
import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
|
@ -116,7 +113,6 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.ws.rs.WebApplicationException;
|
||||
import javax.ws.rs.core.Response;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.text.Collator;
|
||||
|
@ -803,7 +799,7 @@ public class ControllerFacade implements Authorizable {
|
|||
// add each processor
|
||||
for (final ProcessorNode processor : root.findAllProcessors()) {
|
||||
resources.add(ResourceFactory.getComponentResource(ResourceType.Processor, processor.getIdentifier(), processor.getName()));
|
||||
resources.add(ResourceFactory.getComponentProvenanceResource(ResourceType.Processor, processor.getIdentifier(), processor.getName()));
|
||||
resources.add(ResourceFactory.getProvenanceEventResource(processor.getResource()));
|
||||
}
|
||||
|
||||
// add each connection
|
||||
|
@ -820,25 +816,25 @@ public class ControllerFacade implements Authorizable {
|
|||
// add each process group
|
||||
for (final ProcessGroup processGroup : root.findAllProcessGroups()) {
|
||||
resources.add(ResourceFactory.getComponentResource(ResourceType.ProcessGroup, processGroup.getIdentifier(), processGroup.getName()));
|
||||
resources.add(ResourceFactory.getComponentProvenanceResource(ResourceType.ProcessGroup, processGroup.getIdentifier(), processGroup.getName()));
|
||||
resources.add(ResourceFactory.getProvenanceEventResource(processGroup.getResource()));
|
||||
}
|
||||
|
||||
// add each remote process group
|
||||
for (final RemoteProcessGroup remoteProcessGroup : root.findAllRemoteProcessGroups()) {
|
||||
resources.add(ResourceFactory.getComponentResource(ResourceType.RemoteProcessGroup, remoteProcessGroup.getIdentifier(), remoteProcessGroup.getName()));
|
||||
resources.add(ResourceFactory.getComponentProvenanceResource(ResourceType.RemoteProcessGroup, remoteProcessGroup.getIdentifier(), remoteProcessGroup.getName()));
|
||||
resources.add(ResourceFactory.getProvenanceEventResource(remoteProcessGroup.getResource()));
|
||||
}
|
||||
|
||||
// add each input port
|
||||
for (final Port inputPort : root.findAllInputPorts()) {
|
||||
resources.add(ResourceFactory.getComponentResource(ResourceType.InputPort, inputPort.getIdentifier(), inputPort.getName()));
|
||||
resources.add(ResourceFactory.getComponentProvenanceResource(ResourceType.InputPort, inputPort.getIdentifier(), inputPort.getName()));
|
||||
resources.add(ResourceFactory.getProvenanceEventResource(inputPort.getResource()));
|
||||
}
|
||||
|
||||
// add each output port
|
||||
for (final Port outputPort : root.findAllOutputPorts()) {
|
||||
resources.add(ResourceFactory.getComponentResource(ResourceType.OutputPort, outputPort.getIdentifier(), outputPort.getName()));
|
||||
resources.add(ResourceFactory.getComponentProvenanceResource(ResourceType.OutputPort, outputPort.getIdentifier(), outputPort.getName()));
|
||||
resources.add(ResourceFactory.getProvenanceEventResource(outputPort.getResource()));
|
||||
}
|
||||
|
||||
// add each controller service
|
||||
|
@ -943,7 +939,7 @@ public class ControllerFacade implements Authorizable {
|
|||
|
||||
// submit the query to the provenance repository
|
||||
final ProvenanceEventRepository provenanceRepository = flowController.getProvenanceRepository();
|
||||
final QuerySubmission querySubmission = provenanceRepository.submitQuery(query);
|
||||
final QuerySubmission querySubmission = provenanceRepository.submitQuery(query, NiFiUserUtils.getNiFiUser());
|
||||
|
||||
// return the query with the results populated at this point
|
||||
return getProvenanceQuery(querySubmission.getQueryIdentifier());
|
||||
|
@ -959,7 +955,7 @@ public class ControllerFacade implements Authorizable {
|
|||
try {
|
||||
// get the query to the provenance repository
|
||||
final ProvenanceEventRepository provenanceRepository = flowController.getProvenanceRepository();
|
||||
final QuerySubmission querySubmission = provenanceRepository.retrieveQuerySubmission(provenanceId);
|
||||
final QuerySubmission querySubmission = provenanceRepository.retrieveQuerySubmission(provenanceId, NiFiUserUtils.getNiFiUser());
|
||||
|
||||
// ensure the query results could be found
|
||||
if (querySubmission == null) {
|
||||
|
@ -1056,13 +1052,13 @@ public class ControllerFacade implements Authorizable {
|
|||
// submit the event
|
||||
if (LineageRequestType.FLOWFILE.equals(requestDto.getLineageRequestType())) {
|
||||
// submit uuid
|
||||
result = provenanceRepository.submitLineageComputation(requestDto.getUuid());
|
||||
result = provenanceRepository.submitLineageComputation(requestDto.getUuid(), NiFiUserUtils.getNiFiUser());
|
||||
} else {
|
||||
// submit event... (parents or children)
|
||||
if (LineageRequestType.PARENTS.equals(requestDto.getLineageRequestType())) {
|
||||
result = provenanceRepository.submitExpandParents(requestDto.getEventId());
|
||||
result = provenanceRepository.submitExpandParents(requestDto.getEventId(), NiFiUserUtils.getNiFiUser());
|
||||
} else {
|
||||
result = provenanceRepository.submitExpandChildren(requestDto.getEventId());
|
||||
result = provenanceRepository.submitExpandChildren(requestDto.getEventId(), NiFiUserUtils.getNiFiUser());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1078,7 +1074,7 @@ public class ControllerFacade implements Authorizable {
|
|||
public LineageDTO getLineage(final String lineageId) {
|
||||
// get the query to the provenance repository
|
||||
final ProvenanceEventRepository provenanceRepository = flowController.getProvenanceRepository();
|
||||
final ComputeLineageSubmission computeLineageSubmission = provenanceRepository.retrieveLineageSubmission(lineageId);
|
||||
final ComputeLineageSubmission computeLineageSubmission = provenanceRepository.retrieveLineageSubmission(lineageId, NiFiUserUtils.getNiFiUser());
|
||||
|
||||
// ensure the submission was found
|
||||
if (computeLineageSubmission == null) {
|
||||
|
@ -1096,7 +1092,7 @@ public class ControllerFacade implements Authorizable {
|
|||
public void deleteProvenanceQuery(final String provenanceId) {
|
||||
// get the query to the provenance repository
|
||||
final ProvenanceEventRepository provenanceRepository = flowController.getProvenanceRepository();
|
||||
final QuerySubmission querySubmission = provenanceRepository.retrieveQuerySubmission(provenanceId);
|
||||
final QuerySubmission querySubmission = provenanceRepository.retrieveQuerySubmission(provenanceId, NiFiUserUtils.getNiFiUser());
|
||||
if (querySubmission != null) {
|
||||
querySubmission.cancel();
|
||||
}
|
||||
|
@ -1110,7 +1106,7 @@ public class ControllerFacade implements Authorizable {
|
|||
public void deleteLineage(final String lineageId) {
|
||||
// get the query to the provenance repository
|
||||
final ProvenanceEventRepository provenanceRepository = flowController.getProvenanceRepository();
|
||||
final ComputeLineageSubmission computeLineageSubmission = provenanceRepository.retrieveLineageSubmission(lineageId);
|
||||
final ComputeLineageSubmission computeLineageSubmission = provenanceRepository.retrieveLineageSubmission(lineageId, NiFiUserUtils.getNiFiUser());
|
||||
if (computeLineageSubmission != null) {
|
||||
computeLineageSubmission.cancel();
|
||||
}
|
||||
|
@ -1129,7 +1125,7 @@ public class ControllerFacade implements Authorizable {
|
|||
final NiFiUser user = NiFiUserUtils.getNiFiUser();
|
||||
|
||||
// get the event in order to get the filename
|
||||
final ProvenanceEventRecord event = flowController.getProvenanceRepository().getEvent(eventId);
|
||||
final ProvenanceEventRecord event = flowController.getProvenanceRepository().getEvent(eventId, NiFiUserUtils.getNiFiUser());
|
||||
if (event == null) {
|
||||
throw new ResourceNotFoundException("Unable to find the specified event.");
|
||||
}
|
||||
|
@ -1145,54 +1141,18 @@ public class ControllerFacade implements Authorizable {
|
|||
// calculate the dn chain
|
||||
final List<String> dnChain = ProxiedEntitiesUtils.buildProxiedEntitiesChain(user);
|
||||
dnChain.forEach(identity -> {
|
||||
final String rootGroupId = flowController.getRootGroupId();
|
||||
final ProcessGroup rootGroup = flowController.getGroup(rootGroupId);
|
||||
final Authorizable eventAuthorizable = flowController.createProvenanceAuthorizable(event.getComponentId());
|
||||
final NiFiUser chainUser = new StandardNiFiUser(identity) {
|
||||
private static final long serialVersionUID = 7589311627013017356L;
|
||||
|
||||
final Resource eventResource;
|
||||
if (rootGroupId.equals(event.getComponentId())) {
|
||||
eventResource = ResourceFactory.getComponentProvenanceResource(ResourceType.ProcessGroup, rootGroup.getIdentifier(), rootGroup.getName());
|
||||
} else {
|
||||
final Connectable connectable = rootGroup.findConnectable(event.getComponentId());
|
||||
|
||||
if (connectable == null) {
|
||||
throw new AccessDeniedException("The component that generated this event is no longer part of the data flow. Unable to determine access policy.");
|
||||
@Override
|
||||
public boolean isAnonymous() {
|
||||
// allow current user to drive anonymous flag as anonymous users are never chained... supports single user case
|
||||
return user.isAnonymous();
|
||||
}
|
||||
};
|
||||
|
||||
switch (connectable.getConnectableType()) {
|
||||
case PROCESSOR:
|
||||
eventResource = ResourceFactory.getComponentProvenanceResource(ResourceType.Processor, connectable.getIdentifier(), connectable.getName());
|
||||
break;
|
||||
case INPUT_PORT:
|
||||
case REMOTE_INPUT_PORT:
|
||||
eventResource = ResourceFactory.getComponentProvenanceResource(ResourceType.InputPort, connectable.getIdentifier(), connectable.getName());
|
||||
break;
|
||||
case OUTPUT_PORT:
|
||||
case REMOTE_OUTPUT_PORT:
|
||||
eventResource = ResourceFactory.getComponentProvenanceResource(ResourceType.OutputPort, connectable.getIdentifier(), connectable.getName());
|
||||
break;
|
||||
case FUNNEL:
|
||||
eventResource = ResourceFactory.getComponentProvenanceResource(ResourceType.Funnel, connectable.getIdentifier(), connectable.getName());
|
||||
break;
|
||||
default:
|
||||
throw new WebApplicationException(Response.serverError().entity("An unexpected type of component generated this event.").build());
|
||||
}
|
||||
}
|
||||
|
||||
// build the request
|
||||
final AuthorizationRequest request = new AuthorizationRequest.Builder()
|
||||
.identity(identity)
|
||||
.anonymous(user.isAnonymous()) // allow current user to drive anonymous flag as anonymous users are never chained... supports single user case
|
||||
.accessAttempt(false)
|
||||
.action(RequestAction.READ)
|
||||
.resource(eventResource)
|
||||
.eventAttributes(attributes)
|
||||
.build();
|
||||
|
||||
// perform the authorization
|
||||
final AuthorizationResult result = authorizer.authorize(request);
|
||||
if (!Result.Approved.equals(result.getResult())) {
|
||||
throw new AccessDeniedException(result.getExplanation());
|
||||
}
|
||||
eventAuthorizable.authorize(authorizer, RequestAction.READ, chainUser);
|
||||
});
|
||||
|
||||
// get the filename and fall back to the identifier (should never happen)
|
||||
|
@ -1229,13 +1189,13 @@ public class ControllerFacade implements Authorizable {
|
|||
}
|
||||
|
||||
// lookup the original event
|
||||
final ProvenanceEventRecord originalEvent = flowController.getProvenanceRepository().getEvent(eventId);
|
||||
final ProvenanceEventRecord originalEvent = flowController.getProvenanceRepository().getEvent(eventId, NiFiUserUtils.getNiFiUser());
|
||||
if (originalEvent == null) {
|
||||
throw new ResourceNotFoundException("Unable to find the specified event.");
|
||||
}
|
||||
|
||||
// replay the flow file
|
||||
final ProvenanceEventRecord event = flowController.replayFlowFile(originalEvent, user.getIdentity());
|
||||
final ProvenanceEventRecord event = flowController.replayFlowFile(originalEvent, user);
|
||||
|
||||
// convert the event record
|
||||
return createProvenanceEventDto(event);
|
||||
|
@ -1252,7 +1212,7 @@ public class ControllerFacade implements Authorizable {
|
|||
*/
|
||||
public ProvenanceEventDTO getProvenanceEvent(final Long eventId) {
|
||||
try {
|
||||
final ProvenanceEventRecord event = flowController.getProvenanceRepository().getEvent(eventId);
|
||||
final ProvenanceEventRecord event = flowController.getProvenanceRepository().getEvent(eventId, NiFiUserUtils.getNiFiUser());
|
||||
if (event == null) {
|
||||
throw new ResourceNotFoundException("Unable to find the specified event.");
|
||||
}
|
||||
|
|
|
@ -298,7 +298,7 @@ public class StandardConnectionDAO extends ComponentDAO implements ConnectionDAO
|
|||
}
|
||||
|
||||
// ensure the user has write access to the source component
|
||||
source.authorize(authorizer, RequestAction.WRITE);
|
||||
source.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
|
||||
// find the destination
|
||||
final Connectable destination;
|
||||
|
@ -324,7 +324,7 @@ public class StandardConnectionDAO extends ComponentDAO implements ConnectionDAO
|
|||
}
|
||||
|
||||
// ensure the user has write access to the source component
|
||||
destination.authorize(authorizer, RequestAction.WRITE);
|
||||
destination.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
|
||||
|
||||
// determine the relationships
|
||||
final Set<String> relationships = new HashSet<>();
|
||||
|
|
|
@ -20,6 +20,7 @@ import org.apache.nifi.authorization.*
|
|||
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.StandardNiFiUser
|
||||
import org.apache.nifi.authorization.user.NiFiUserDetails
|
||||
import org.apache.nifi.controller.service.ControllerServiceProvider
|
||||
import org.apache.nifi.web.api.dto.*
|
||||
|
@ -39,7 +40,7 @@ import spock.lang.Unroll
|
|||
class StandardNiFiServiceFacadeSpec extends Specification {
|
||||
|
||||
def setup() {
|
||||
final NiFiUser user = new NiFiUser("nifi-user");
|
||||
final NiFiUser user = new StandardNiFiUser("nifi-user");
|
||||
final NiFiAuthenticationToken auth = new NiFiAuthenticationToken(new NiFiUserDetails(user));
|
||||
SecurityContextHolder.getContext().setAuthentication(auth);
|
||||
}
|
||||
|
@ -867,17 +868,17 @@ class StandardNiFiServiceFacadeSpec extends Specification {
|
|||
}
|
||||
|
||||
@Override
|
||||
boolean isAuthorized(Authorizer authorzr, RequestAction action) {
|
||||
boolean isAuthorized(Authorizer authorzr, RequestAction action, NiFiUser user) {
|
||||
return isAuthorized
|
||||
}
|
||||
|
||||
@Override
|
||||
AuthorizationResult checkAuthorization(Authorizer authorzr, RequestAction action) {
|
||||
AuthorizationResult checkAuthorization(Authorizer authorzr, RequestAction action, NiFiUser user) {
|
||||
return authorizationResult
|
||||
}
|
||||
|
||||
@Override
|
||||
void authorize(Authorizer authorzr, RequestAction action) throws AccessDeniedException {
|
||||
void authorize(Authorizer authorzr, RequestAction action, NiFiUser user) throws AccessDeniedException {
|
||||
if (!isAuthorized) {
|
||||
throw new AccessDeniedException("test exception, access denied")
|
||||
}
|
||||
|
|
|
@ -39,6 +39,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.authorization.user.StandardNiFiUser;
|
||||
import org.apache.nifi.web.FlowModification;
|
||||
import org.apache.nifi.web.InvalidRevisionException;
|
||||
import org.apache.nifi.web.Revision;
|
||||
|
@ -49,7 +50,7 @@ import org.junit.Test;
|
|||
public class TestNaiveRevisionManager {
|
||||
private static final String CLIENT_1 = "client-1";
|
||||
private static final String COMPONENT_1 = "component-1";
|
||||
private static final NiFiUser USER_1 = new NiFiUser("user-1");
|
||||
private static final NiFiUser USER_1 = new StandardNiFiUser("user-1");
|
||||
|
||||
private RevisionUpdate<Object> components(final Revision revision) {
|
||||
return new StandardRevisionUpdate<Object>(null, new FlowModification(revision, null));
|
||||
|
@ -302,7 +303,7 @@ public class TestNaiveRevisionManager {
|
|||
final RevisionClaim firstClaim = revisionManager.requestClaim(firstRevision, USER_1);
|
||||
assertNotNull(firstClaim);
|
||||
|
||||
final NiFiUser user2 = new NiFiUser("user-2");
|
||||
final NiFiUser user2 = new StandardNiFiUser("user-2");
|
||||
try {
|
||||
revisionManager.updateRevision(firstClaim, user2, () -> null);
|
||||
Assert.fail("Expected updateRevision to fail with a different user but it succeeded");
|
||||
|
@ -318,7 +319,7 @@ public class TestNaiveRevisionManager {
|
|||
final RevisionClaim firstClaim = revisionManager.requestClaim(firstRevision, USER_1);
|
||||
assertNotNull(firstClaim);
|
||||
|
||||
final NiFiUser user2 = new NiFiUser("user-2");
|
||||
final NiFiUser user2 = new StandardNiFiUser("user-2");
|
||||
try {
|
||||
revisionManager.deleteRevision(firstClaim, user2, () -> null);
|
||||
Assert.fail("Expected deleteRevision to fail with a different user but it succeeded");
|
||||
|
|
|
@ -16,14 +16,14 @@
|
|||
*/
|
||||
package org.apache.nifi.web.security.anonymous;
|
||||
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
|
||||
import org.apache.nifi.authorization.user.NiFiUserDetails;
|
||||
import org.apache.nifi.authorization.user.StandardNiFiUser;
|
||||
import org.apache.nifi.web.security.token.NiFiAuthenticationToken;
|
||||
import org.springframework.security.core.Authentication;
|
||||
import org.springframework.security.web.authentication.AnonymousAuthenticationFilter;
|
||||
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
|
||||
public class NiFiAnonymousUserFilter extends AnonymousAuthenticationFilter {
|
||||
|
||||
private static final String ANONYMOUS_KEY = "anonymousNifiKey";
|
||||
|
@ -34,7 +34,7 @@ public class NiFiAnonymousUserFilter extends AnonymousAuthenticationFilter {
|
|||
|
||||
@Override
|
||||
protected Authentication createAuthentication(HttpServletRequest request) {
|
||||
return new NiFiAuthenticationToken(new NiFiUserDetails(NiFiUser.ANONYMOUS));
|
||||
return new NiFiAuthenticationToken(new NiFiUserDetails(StandardNiFiUser.ANONYMOUS));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -16,15 +16,17 @@
|
|||
*/
|
||||
package org.apache.nifi.web.security.jwt;
|
||||
|
||||
import io.jsonwebtoken.JwtException;
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.authorization.user.NiFiUserDetails;
|
||||
import org.apache.nifi.authorization.user.StandardNiFiUser;
|
||||
import org.apache.nifi.web.security.InvalidAuthenticationException;
|
||||
import org.apache.nifi.web.security.token.NiFiAuthenticationToken;
|
||||
import org.springframework.security.authentication.AuthenticationProvider;
|
||||
import org.springframework.security.core.Authentication;
|
||||
import org.springframework.security.core.AuthenticationException;
|
||||
|
||||
import io.jsonwebtoken.JwtException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
|
@ -42,7 +44,7 @@ public class JwtAuthenticationProvider implements AuthenticationProvider {
|
|||
|
||||
try {
|
||||
final String jwtPrincipal = jwtService.getAuthenticationFromToken(request.getToken());
|
||||
final NiFiUser user = new NiFiUser(jwtPrincipal);
|
||||
final NiFiUser user = new StandardNiFiUser(jwtPrincipal);
|
||||
return new NiFiAuthenticationToken(new NiFiUserDetails(user));
|
||||
} catch (JwtException e) {
|
||||
throw new InvalidAuthenticationException(e.getMessage(), e);
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.nifi.web.security.otp;
|
|||
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.authorization.user.NiFiUserDetails;
|
||||
import org.apache.nifi.authorization.user.StandardNiFiUser;
|
||||
import org.apache.nifi.web.security.InvalidAuthenticationException;
|
||||
import org.apache.nifi.web.security.token.NiFiAuthenticationToken;
|
||||
import org.springframework.security.authentication.AuthenticationProvider;
|
||||
|
@ -46,7 +47,7 @@ public class OtpAuthenticationProvider implements AuthenticationProvider {
|
|||
} else {
|
||||
otpPrincipal = otpService.getAuthenticationFromUiExtensionToken(request.getToken());
|
||||
}
|
||||
final NiFiUser user = new NiFiUser(otpPrincipal);
|
||||
final NiFiUser user = new StandardNiFiUser(otpPrincipal);
|
||||
return new NiFiAuthenticationToken(new NiFiUserDetails(user));
|
||||
} catch (OtpAuthenticationException e) {
|
||||
throw new InvalidAuthenticationException(e.getMessage(), e);
|
||||
|
|
|
@ -47,4 +47,9 @@ public class NiFiAuthenticationToken extends AbstractAuthenticationToken {
|
|||
public final void setAuthenticated(boolean authenticated) {
|
||||
throw new IllegalArgumentException("Cannot change the authenticated state.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return nifiUserDetails.getUsername();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ 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.NiFiUserDetails;
|
||||
import org.apache.nifi.authorization.user.StandardNiFiUser;
|
||||
import org.apache.nifi.web.security.InvalidAuthenticationException;
|
||||
import org.apache.nifi.web.security.ProxiedEntitiesUtils;
|
||||
import org.apache.nifi.web.security.UntrustedProxyException;
|
||||
|
@ -64,7 +65,7 @@ public class X509AuthenticationProvider implements AuthenticationProvider {
|
|||
}
|
||||
|
||||
if (StringUtils.isBlank(request.getProxiedEntitiesChain())) {
|
||||
return new NiFiAuthenticationToken(new NiFiUserDetails(new NiFiUser(authenticationResponse.getIdentity(), authenticationResponse.getUsername(), null)));
|
||||
return new NiFiAuthenticationToken(new NiFiUserDetails(new StandardNiFiUser(authenticationResponse.getIdentity(), authenticationResponse.getUsername(), null)));
|
||||
} else {
|
||||
// build the entire proxy chain if applicable - <end-user><proxy1><proxy2>
|
||||
final List<String> proxyChain = new ArrayList<>(ProxiedEntitiesUtils.tokenizeProxiedEntitiesChain(request.getProxiedEntitiesChain()));
|
||||
|
@ -91,7 +92,7 @@ public class X509AuthenticationProvider implements AuthenticationProvider {
|
|||
}
|
||||
}
|
||||
|
||||
proxy = new NiFiUser(chainIter.previous(), proxy);
|
||||
proxy = new StandardNiFiUser(chainIter.previous(), proxy);
|
||||
}
|
||||
|
||||
return new NiFiAuthenticationToken(new NiFiUserDetails(proxy));
|
||||
|
|
|
@ -145,9 +145,9 @@ nf.ProvenanceLineage = (function () {
|
|||
*/
|
||||
var getLineage = function (lineage) {
|
||||
var url = lineage.uri;
|
||||
if (nf.Common.isDefinedAndNotNull(lineage.clusterNodeId)) {
|
||||
if (nf.Common.isDefinedAndNotNull(lineage.request.clusterNodeId)) {
|
||||
url += '?' + $.param({
|
||||
clusterNodeId: lineage.clusterNodeId
|
||||
clusterNodeId: lineage.request.clusterNodeId
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -166,9 +166,9 @@ nf.ProvenanceLineage = (function () {
|
|||
*/
|
||||
var cancelLineage = function (lineage) {
|
||||
var url = lineage.uri;
|
||||
if (nf.Common.isDefinedAndNotNull(lineage.clusterNodeId)) {
|
||||
if (nf.Common.isDefinedAndNotNull(lineage.request.clusterNodeId)) {
|
||||
url += '?' + $.param({
|
||||
clusterNodeId: lineage.clusterNodeId
|
||||
clusterNodeId: lineage.request.clusterNodeId
|
||||
});
|
||||
}
|
||||
|
||||
|
|
|
@ -384,7 +384,7 @@ nf.ProvenanceTable = (function () {
|
|||
startTime = config.defaultStartTime;
|
||||
$('#provenance-search-start-time').val(startTime);
|
||||
}
|
||||
search['startDate'] = startDate + ' ' + startTime;
|
||||
search['startDate'] = startDate + ' ' + startTime + ' ' + $('.timezone:first').text();
|
||||
}
|
||||
|
||||
// extract the end date time
|
||||
|
@ -395,7 +395,7 @@ nf.ProvenanceTable = (function () {
|
|||
endTime = config.defaultEndTime;
|
||||
$('#provenance-search-end-time').val(endTime);
|
||||
}
|
||||
search['endDate'] = endDate + ' ' + endTime;
|
||||
search['endDate'] = endDate + ' ' + endTime + ' ' + $('.timezone:first').text();
|
||||
}
|
||||
|
||||
// extract the min/max file size
|
||||
|
|
|
@ -29,7 +29,7 @@ nf.Provenance = (function () {
|
|||
*/
|
||||
var config = {
|
||||
urls: {
|
||||
cluster: '../nifi-api/controller/cluster',
|
||||
flowConfig: '../nifi-api/flow/config',
|
||||
banners: '../nifi-api/flow/banners',
|
||||
about: '../nifi-api/flow/about',
|
||||
authorities: '../nifi-api/flow/authorities'
|
||||
|
@ -45,23 +45,12 @@ nf.Provenance = (function () {
|
|||
* Determines if this NiFi is clustered.
|
||||
*/
|
||||
var detectedCluster = function () {
|
||||
return $.Deferred(function (deferred) {
|
||||
$.ajax({
|
||||
type: 'HEAD',
|
||||
url: config.urls.cluster
|
||||
}).done(function () {
|
||||
isClustered = true;
|
||||
deferred.resolve();
|
||||
}).fail(function (xhr, status, error) {
|
||||
if (xhr.status === 404) {
|
||||
isClustered = false;
|
||||
deferred.resolve();
|
||||
} else {
|
||||
nf.Common.handleAjaxError(xhr, status, error);
|
||||
deferred.reject();
|
||||
}
|
||||
});
|
||||
}).promise();
|
||||
return $.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.flowConfig
|
||||
}).done(function (response) {
|
||||
isClustered = response.flowConfiguration.clustered;
|
||||
}).fail(nf.Common.handleAjaxError);
|
||||
};
|
||||
|
||||
/**
|
||||
|
|
|
@ -51,7 +51,7 @@ nf.Summary = (function () {
|
|||
urls: {
|
||||
banners: '../nifi-api/flow/banners',
|
||||
about: '../nifi-api/flow/about',
|
||||
cluster: '../nifi-api/controller/cluster'
|
||||
flowConfig: '../nifi-api/flow/config'
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -61,26 +61,15 @@ nf.Summary = (function () {
|
|||
var initializeSummaryTable = function () {
|
||||
return $.Deferred(function (deferred) {
|
||||
$.ajax({
|
||||
type: 'HEAD',
|
||||
url: config.urls.cluster
|
||||
}).done(function () {
|
||||
nf.SummaryTable.init(true).done(function () {
|
||||
type: 'GET',
|
||||
url: config.urls.flowConfig
|
||||
}).done(function (response) {
|
||||
nf.SummaryTable.init(response.flowConfiguration.clustered).done(function () {
|
||||
deferred.resolve();
|
||||
}).fail(function () {
|
||||
deferred.reject();
|
||||
});
|
||||
}).fail(function (xhr, status, error) {
|
||||
if (xhr.status === 404) {
|
||||
nf.SummaryTable.init(false).done(function () {
|
||||
deferred.resolve();
|
||||
}).fail(function () {
|
||||
deferred.reject();
|
||||
});
|
||||
} else {
|
||||
nf.Common.handleAjaxError(xhr, status, error);
|
||||
deferred.reject();
|
||||
}
|
||||
});
|
||||
}).fail(nf.Common.handleAjaxError);
|
||||
}).promise();
|
||||
};
|
||||
|
||||
|
|
|
@ -59,6 +59,7 @@ import java.util.concurrent.locks.Lock;
|
|||
import java.util.concurrent.locks.ReadWriteLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
|
@ -68,6 +69,13 @@ import org.apache.lucene.search.IndexSearcher;
|
|||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
import org.apache.nifi.authorization.AccessDeniedException;
|
||||
import org.apache.nifi.authorization.AuthorizationResult;
|
||||
import org.apache.nifi.authorization.AuthorizationResult.Result;
|
||||
import org.apache.nifi.authorization.Authorizer;
|
||||
import org.apache.nifi.authorization.RequestAction;
|
||||
import org.apache.nifi.authorization.resource.Authorizable;
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.events.EventReporter;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
import org.apache.nifi.provenance.expiration.ExpirationAction;
|
||||
|
@ -99,6 +107,7 @@ import org.apache.nifi.util.RingBuffer;
|
|||
import org.apache.nifi.util.RingBuffer.ForEachEvaluator;
|
||||
import org.apache.nifi.util.StopWatch;
|
||||
import org.apache.nifi.util.Tuple;
|
||||
import org.apache.nifi.web.ResourceNotFoundException;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -163,7 +172,9 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
// we keep the last 1000 records on hand so that when the UI is opened and it asks for the last 1000 records we don't need to
|
||||
// read them. Since this is a very cheap operation to keep them, it's worth the tiny expense for the improved user experience.
|
||||
private final RingBuffer<ProvenanceEventRecord> latestRecords = new RingBuffer<>(1000);
|
||||
private EventReporter eventReporter;
|
||||
private EventReporter eventReporter; // effectively final
|
||||
private Authorizer authorizer; // effectively final
|
||||
private ProvenanceAuthorizableFactory resourceFactory; // effectively final
|
||||
|
||||
public PersistentProvenanceRepository() throws IOException {
|
||||
this(createRepositoryConfiguration(), 10000);
|
||||
|
@ -207,7 +218,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
}
|
||||
|
||||
@Override
|
||||
public void initialize(final EventReporter eventReporter) throws IOException {
|
||||
public void initialize(final EventReporter eventReporter, final Authorizer authorizer, final ProvenanceAuthorizableFactory resourceFactory) throws IOException {
|
||||
writeLock.lock();
|
||||
try {
|
||||
if (initialized.getAndSet(true)) {
|
||||
|
@ -215,6 +226,8 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
}
|
||||
|
||||
this.eventReporter = eventReporter;
|
||||
this.authorizer = authorizer;
|
||||
this.resourceFactory = resourceFactory;
|
||||
|
||||
recover();
|
||||
|
||||
|
@ -391,8 +404,46 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
persistRecord(events);
|
||||
}
|
||||
|
||||
public boolean isAuthorized(final ProvenanceEventRecord event, final NiFiUser user) {
|
||||
if (authorizer == null || user == null) {
|
||||
return true;
|
||||
}
|
||||
|
||||
final Authorizable eventAuthorizable;
|
||||
try {
|
||||
eventAuthorizable = resourceFactory.createProvenanceAuthorizable(event.getComponentId());
|
||||
} catch (final ResourceNotFoundException rnfe) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final AuthorizationResult result = eventAuthorizable.checkAuthorization(authorizer, RequestAction.READ, user);
|
||||
return Result.Approved.equals(result.getResult());
|
||||
}
|
||||
|
||||
protected void authorize(final ProvenanceEventRecord event, final NiFiUser user) {
|
||||
if (authorizer == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final Authorizable eventAuthorizable = resourceFactory.createProvenanceAuthorizable(event.getComponentId());
|
||||
eventAuthorizable.authorize(authorizer, RequestAction.READ, user);
|
||||
}
|
||||
|
||||
private List<ProvenanceEventRecord> filterUnauthorizedEvents(final List<ProvenanceEventRecord> events, final NiFiUser user) {
|
||||
return events.stream().filter(event -> isAuthorized(event, user)).collect(Collectors.<ProvenanceEventRecord> toList());
|
||||
}
|
||||
|
||||
private Set<ProvenanceEventRecord> replaceUnauthorizedWithPlaceholders(final Set<ProvenanceEventRecord> events, final NiFiUser user) {
|
||||
return events.stream().map(event -> isAuthorized(event, user) ? event : new PlaceholderProvenanceEvent(event)).collect(Collectors.toSet());
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ProvenanceEventRecord> getEvents(final long firstRecordId, final int maxRecords) throws IOException {
|
||||
return getEvents(firstRecordId, maxRecords, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ProvenanceEventRecord> getEvents(final long firstRecordId, final int maxRecords, final NiFiUser user) throws IOException {
|
||||
final List<ProvenanceEventRecord> records = new ArrayList<>(maxRecords);
|
||||
|
||||
final List<Path> paths = getPathsForId(firstRecordId);
|
||||
|
@ -417,7 +468,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
|
||||
StandardProvenanceEventRecord record;
|
||||
while (records.size() < maxRecords && (record = reader.nextRecord()) != null) {
|
||||
if (record.getEventId() >= firstRecordId) {
|
||||
if (record.getEventId() >= firstRecordId && isAuthorized(record, user)) {
|
||||
records.add(record);
|
||||
}
|
||||
}
|
||||
|
@ -1807,8 +1858,8 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
return new ArrayList<>(configuration.getSearchableAttributes());
|
||||
}
|
||||
|
||||
QueryResult queryEvents(final Query query) throws IOException {
|
||||
final QuerySubmission submission = submitQuery(query);
|
||||
QueryResult queryEvents(final Query query, final NiFiUser user) throws IOException {
|
||||
final QuerySubmission submission = submitQuery(query, user);
|
||||
final QueryResult result = submission.getResult();
|
||||
while (!result.isFinished()) {
|
||||
try {
|
||||
|
@ -1826,8 +1877,10 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
}
|
||||
|
||||
@Override
|
||||
public QuerySubmission submitQuery(final Query query) {
|
||||
public QuerySubmission submitQuery(final Query query, final NiFiUser user) {
|
||||
final String userId = user.getIdentity();
|
||||
final int numQueries = querySubmissionMap.size();
|
||||
|
||||
if (numQueries > MAX_UNDELETED_QUERY_RESULTS) {
|
||||
throw new IllegalStateException("Cannot process query because there are currently " + numQueries + " queries whose results have not "
|
||||
+ "been deleted due to poorly behaving clients not issuing DELETE requests. Please try again later.");
|
||||
|
@ -1838,10 +1891,10 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
}
|
||||
|
||||
if (query.getSearchTerms().isEmpty() && query.getStartDate() == null && query.getEndDate() == null) {
|
||||
final AsyncQuerySubmission result = new AsyncQuerySubmission(query, 1);
|
||||
final AsyncQuerySubmission result = new AsyncQuerySubmission(query, 1, userId);
|
||||
|
||||
if (latestRecords.getSize() >= query.getMaxResults()) {
|
||||
final List<ProvenanceEventRecord> latestList = latestRecords.asList();
|
||||
final List<ProvenanceEventRecord> latestList = filterUnauthorizedEvents(latestRecords.asList(), user);
|
||||
final List<ProvenanceEventRecord> trimmed;
|
||||
if (latestList.size() > query.getMaxResults()) {
|
||||
trimmed = latestList.subList(latestList.size() - query.getMaxResults(), latestList.size());
|
||||
|
@ -1863,7 +1916,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
|
||||
result.getResult().update(trimmed, totalNumDocs);
|
||||
} else {
|
||||
queryExecService.submit(new GetMostRecentRunnable(query, result));
|
||||
queryExecService.submit(new GetMostRecentRunnable(query, result, user));
|
||||
}
|
||||
|
||||
querySubmissionMap.put(query.getIdentifier(), result);
|
||||
|
@ -1874,14 +1927,14 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
final List<File> indexDirectories = indexConfig.getIndexDirectories(
|
||||
query.getStartDate() == null ? null : query.getStartDate().getTime(),
|
||||
query.getEndDate() == null ? null : query.getEndDate().getTime());
|
||||
final AsyncQuerySubmission result = new AsyncQuerySubmission(query, indexDirectories.size());
|
||||
final AsyncQuerySubmission result = new AsyncQuerySubmission(query, indexDirectories.size(), userId);
|
||||
querySubmissionMap.put(query.getIdentifier(), result);
|
||||
|
||||
if (indexDirectories.isEmpty()) {
|
||||
result.getResult().update(Collections.<ProvenanceEventRecord>emptyList(), 0L);
|
||||
} else {
|
||||
for (final File indexDir : indexDirectories) {
|
||||
queryExecService.submit(new QueryRunnable(query, result, indexDir, retrievalCount));
|
||||
queryExecService.submit(new QueryRunnable(query, result, user, indexDir, retrievalCount));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2054,13 +2107,13 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
};
|
||||
}
|
||||
|
||||
Lineage computeLineage(final String flowFileUuid) throws IOException {
|
||||
return computeLineage(Collections.<String>singleton(flowFileUuid), LineageComputationType.FLOWFILE_LINEAGE, null, 0L, Long.MAX_VALUE);
|
||||
Lineage computeLineage(final String flowFileUuid, final NiFiUser user) throws IOException {
|
||||
return computeLineage(Collections.<String> singleton(flowFileUuid), user, LineageComputationType.FLOWFILE_LINEAGE, null, 0L, Long.MAX_VALUE);
|
||||
}
|
||||
|
||||
private Lineage computeLineage(final Collection<String> flowFileUuids, final LineageComputationType computationType, final Long eventId, final Long startTimestamp,
|
||||
private Lineage computeLineage(final Collection<String> flowFileUuids, final NiFiUser user, final LineageComputationType computationType, final Long eventId, final Long startTimestamp,
|
||||
final Long endTimestamp) throws IOException {
|
||||
final AsyncLineageSubmission submission = submitLineageComputation(flowFileUuids, computationType, eventId, startTimestamp, endTimestamp);
|
||||
final AsyncLineageSubmission submission = submitLineageComputation(flowFileUuids, user, computationType, eventId, startTimestamp, endTimestamp);
|
||||
final StandardLineageResult result = submission.getResult();
|
||||
while (!result.isFinished()) {
|
||||
try {
|
||||
|
@ -2077,29 +2130,31 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
}
|
||||
|
||||
@Override
|
||||
public AsyncLineageSubmission submitLineageComputation(final String flowFileUuid) {
|
||||
return submitLineageComputation(Collections.singleton(flowFileUuid), LineageComputationType.FLOWFILE_LINEAGE, null, 0L, Long.MAX_VALUE);
|
||||
public AsyncLineageSubmission submitLineageComputation(final String flowFileUuid, final NiFiUser user) {
|
||||
return submitLineageComputation(Collections.singleton(flowFileUuid), user, LineageComputationType.FLOWFILE_LINEAGE, null, 0L, Long.MAX_VALUE);
|
||||
}
|
||||
|
||||
private AsyncLineageSubmission submitLineageComputation(final Collection<String> flowFileUuids, final LineageComputationType computationType,
|
||||
private AsyncLineageSubmission submitLineageComputation(final Collection<String> flowFileUuids, final NiFiUser user, final LineageComputationType computationType,
|
||||
final Long eventId, final long startTimestamp, final long endTimestamp) {
|
||||
final List<File> indexDirs = indexConfig.getIndexDirectories(startTimestamp, endTimestamp);
|
||||
final AsyncLineageSubmission result = new AsyncLineageSubmission(computationType, eventId, flowFileUuids, indexDirs.size());
|
||||
final AsyncLineageSubmission result = new AsyncLineageSubmission(computationType, eventId, flowFileUuids, indexDirs.size(), user.getIdentity());
|
||||
lineageSubmissionMap.put(result.getLineageIdentifier(), result);
|
||||
|
||||
for (final File indexDir : indexDirs) {
|
||||
queryExecService.submit(new ComputeLineageRunnable(flowFileUuids, result, indexDir));
|
||||
queryExecService.submit(new ComputeLineageRunnable(flowFileUuids, user, result, indexDir));
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncLineageSubmission submitExpandChildren(final long eventId) {
|
||||
public AsyncLineageSubmission submitExpandChildren(final long eventId, final NiFiUser user) {
|
||||
final String userId = user.getIdentity();
|
||||
|
||||
try {
|
||||
final ProvenanceEventRecord event = getEvent(eventId);
|
||||
if (event == null) {
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String> emptyList(), 1, userId);
|
||||
lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
|
||||
submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList());
|
||||
return submission;
|
||||
|
@ -2110,15 +2165,15 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
case FORK:
|
||||
case JOIN:
|
||||
case REPLAY:
|
||||
return submitLineageComputation(event.getChildUuids(), LineageComputationType.EXPAND_CHILDREN, eventId, event.getEventTime(), Long.MAX_VALUE);
|
||||
return submitLineageComputation(event.getChildUuids(), user, LineageComputationType.EXPAND_CHILDREN, eventId, event.getEventTime(), Long.MAX_VALUE);
|
||||
default:
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String> emptyList(), 1, userId);
|
||||
lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
|
||||
submission.getResult().setError("Event ID " + eventId + " indicates an event of type " + event.getEventType() + " so its children cannot be expanded");
|
||||
return submission;
|
||||
}
|
||||
} catch (final IOException ioe) {
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String> emptyList(), 1, userId);
|
||||
lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
|
||||
|
||||
if (ioe.getMessage() == null) {
|
||||
|
@ -2132,11 +2187,13 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
}
|
||||
|
||||
@Override
|
||||
public AsyncLineageSubmission submitExpandParents(final long eventId) {
|
||||
public AsyncLineageSubmission submitExpandParents(final long eventId, final NiFiUser user) {
|
||||
final String userId = user.getIdentity();
|
||||
|
||||
try {
|
||||
final ProvenanceEventRecord event = getEvent(eventId);
|
||||
if (event == null) {
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String> emptyList(), 1, userId);
|
||||
lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
|
||||
submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList());
|
||||
return submission;
|
||||
|
@ -2147,16 +2204,16 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
case FORK:
|
||||
case CLONE:
|
||||
case REPLAY:
|
||||
return submitLineageComputation(event.getParentUuids(), LineageComputationType.EXPAND_PARENTS, eventId, event.getLineageStartDate(), event.getEventTime());
|
||||
return submitLineageComputation(event.getParentUuids(), user, LineageComputationType.EXPAND_PARENTS, eventId, event.getLineageStartDate(), event.getEventTime());
|
||||
default: {
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String>emptyList(), 1);
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String> emptyList(), 1, userId);
|
||||
lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
|
||||
submission.getResult().setError("Event ID " + eventId + " indicates an event of type " + event.getEventType() + " so its parents cannot be expanded");
|
||||
return submission;
|
||||
}
|
||||
}
|
||||
} catch (final IOException ioe) {
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String>emptyList(), 1);
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String> emptyList(), 1, userId);
|
||||
lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
|
||||
|
||||
if (ioe.getMessage() == null) {
|
||||
|
@ -2170,17 +2227,47 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
}
|
||||
|
||||
@Override
|
||||
public AsyncLineageSubmission retrieveLineageSubmission(final String lineageIdentifier) {
|
||||
return lineageSubmissionMap.get(lineageIdentifier);
|
||||
public AsyncLineageSubmission retrieveLineageSubmission(final String lineageIdentifier, final NiFiUser user) {
|
||||
final AsyncLineageSubmission submission = lineageSubmissionMap.get(lineageIdentifier);
|
||||
final String userId = submission.getSubmitterIdentity();
|
||||
|
||||
if (user == null && userId == null) {
|
||||
return submission;
|
||||
}
|
||||
|
||||
if (user == null) {
|
||||
throw new AccessDeniedException("Cannot retrieve Provenance Lineage Submission because no user id was provided");
|
||||
}
|
||||
|
||||
if (userId == null || userId.equals(user.getIdentity())) {
|
||||
return submission;
|
||||
}
|
||||
|
||||
throw new AccessDeniedException("Cannot retrieve Provenance Lineage Submission because " + user.getIdentity() + " is not the user who submitted the request");
|
||||
}
|
||||
|
||||
@Override
|
||||
public QuerySubmission retrieveQuerySubmission(final String queryIdentifier) {
|
||||
return querySubmissionMap.get(queryIdentifier);
|
||||
public QuerySubmission retrieveQuerySubmission(final String queryIdentifier, final NiFiUser user) {
|
||||
final QuerySubmission submission = querySubmissionMap.get(queryIdentifier);
|
||||
|
||||
final String userId = submission.getSubmitterIdentity();
|
||||
|
||||
if (user == null && userId == null) {
|
||||
return submission;
|
||||
}
|
||||
|
||||
if (user == null) {
|
||||
throw new AccessDeniedException("Cannot retrieve Provenance Query Submission because no user id was provided");
|
||||
}
|
||||
|
||||
if (userId == null || userId.equals(user.getIdentity())) {
|
||||
return submission;
|
||||
}
|
||||
|
||||
throw new AccessDeniedException("Cannot retrieve Provenance Query Submission because " + user.getIdentity() + " is not the user who submitted the request");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ProvenanceEventRecord getEvent(final long id) throws IOException {
|
||||
private ProvenanceEventRecord getEvent(final long id) throws IOException {
|
||||
final List<ProvenanceEventRecord> records = getEvents(id, 1);
|
||||
if (records.isEmpty()) {
|
||||
return null;
|
||||
|
@ -2192,6 +2279,17 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
return record;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ProvenanceEventRecord getEvent(final long id, final NiFiUser user) throws IOException {
|
||||
final ProvenanceEventRecord event = getEvent(id);
|
||||
if (event == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
authorize(event, user);
|
||||
return event;
|
||||
}
|
||||
|
||||
private boolean needToRollover() {
|
||||
final long writtenSinceRollover = bytesWrittenSinceRollover.get();
|
||||
|
||||
|
@ -2268,10 +2366,12 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
|
||||
private final Query query;
|
||||
private final AsyncQuerySubmission submission;
|
||||
private final NiFiUser user;
|
||||
|
||||
public GetMostRecentRunnable(final Query query, final AsyncQuerySubmission submission) {
|
||||
public GetMostRecentRunnable(final Query query, final AsyncQuerySubmission submission, final NiFiUser user) {
|
||||
this.query = query;
|
||||
this.submission = submission;
|
||||
this.user = user;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -2293,7 +2393,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
}
|
||||
final long totalNumDocs = maxEventId - minIndexedId;
|
||||
|
||||
final List<ProvenanceEventRecord> mostRecent = getEvents(startIndex, maxResults);
|
||||
final List<ProvenanceEventRecord> mostRecent = getEvents(startIndex, maxResults, user);
|
||||
submission.getResult().update(mostRecent, totalNumDocs);
|
||||
} catch (final IOException ioe) {
|
||||
logger.error("Failed to retrieve records from Provenance Repository: " + ioe.toString());
|
||||
|
@ -2314,12 +2414,14 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
|
||||
private final Query query;
|
||||
private final AsyncQuerySubmission submission;
|
||||
private final NiFiUser user;
|
||||
private final File indexDir;
|
||||
private final AtomicInteger retrievalCount;
|
||||
|
||||
public QueryRunnable(final Query query, final AsyncQuerySubmission submission, final File indexDir, final AtomicInteger retrievalCount) {
|
||||
public QueryRunnable(final Query query, final AsyncQuerySubmission submission, final NiFiUser user, final File indexDir, final AtomicInteger retrievalCount) {
|
||||
this.query = query;
|
||||
this.submission = submission;
|
||||
this.user = user;
|
||||
this.indexDir = indexDir;
|
||||
this.retrievalCount = retrievalCount;
|
||||
}
|
||||
|
@ -2328,7 +2430,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
public void run() {
|
||||
try {
|
||||
final IndexSearch search = new IndexSearch(PersistentProvenanceRepository.this, indexDir, indexManager, maxAttributeChars);
|
||||
final StandardQueryResult queryResult = search.search(query, retrievalCount, firstEventTimestamp);
|
||||
final StandardQueryResult queryResult = search.search(query, user, retrievalCount, firstEventTimestamp);
|
||||
submission.getResult().update(queryResult.getMatchingEvents(), queryResult.getTotalHitCount());
|
||||
} catch (final Throwable t) {
|
||||
logger.error("Failed to query Provenance Repository Index {} due to {}", indexDir, t.toString());
|
||||
|
@ -2348,11 +2450,13 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
private class ComputeLineageRunnable implements Runnable {
|
||||
|
||||
private final Collection<String> flowFileUuids;
|
||||
private final NiFiUser user;
|
||||
private final File indexDir;
|
||||
private final AsyncLineageSubmission submission;
|
||||
|
||||
public ComputeLineageRunnable(final Collection<String> flowFileUuids, final AsyncLineageSubmission submission, final File indexDir) {
|
||||
public ComputeLineageRunnable(final Collection<String> flowFileUuids, final NiFiUser user, final AsyncLineageSubmission submission, final File indexDir) {
|
||||
this.flowFileUuids = flowFileUuids;
|
||||
this.user = user;
|
||||
this.submission = submission;
|
||||
this.indexDir = indexDir;
|
||||
}
|
||||
|
@ -2368,7 +2472,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
|
|||
indexManager, indexDir, null, flowFileUuids, maxAttributeChars);
|
||||
|
||||
final StandardLineageResult result = submission.getResult();
|
||||
result.update(matchingRecords);
|
||||
result.update(replaceUnauthorizedWithPlaceholders(matchingRecords, user));
|
||||
|
||||
logger.info("Successfully created Lineage for FlowFiles with UUIDs {} in {} milliseconds; Lineage contains {} nodes and {} edges",
|
||||
flowFileUuids, result.getComputationTime(TimeUnit.MILLISECONDS), result.getNodes().size(), result.getEdges().size());
|
||||
|
|
|
@ -0,0 +1,24 @@
|
|||
/*
|
||||
* 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.provenance.authorization;
|
||||
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
|
||||
public interface AuthorizationCheck {
|
||||
boolean isAuthorized(ProvenanceEventRecord event);
|
||||
}
|
|
@ -33,6 +33,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.SearchableFields;
|
||||
import org.apache.nifi.provenance.StandardProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.authorization.AuthorizationCheck;
|
||||
import org.apache.nifi.provenance.serialization.RecordReader;
|
||||
import org.apache.nifi.provenance.serialization.RecordReaders;
|
||||
import org.apache.nifi.provenance.toc.TocReader;
|
||||
|
@ -47,7 +48,7 @@ import org.slf4j.LoggerFactory;
|
|||
class DocsReader {
|
||||
private final Logger logger = LoggerFactory.getLogger(DocsReader.class);
|
||||
|
||||
public Set<ProvenanceEventRecord> read(final TopDocs topDocs, final IndexReader indexReader, final Collection<Path> allProvenanceLogFiles,
|
||||
public Set<ProvenanceEventRecord> read(final TopDocs topDocs, final AuthorizationCheck authCheck, final IndexReader indexReader, final Collection<Path> allProvenanceLogFiles,
|
||||
final AtomicInteger retrievalCount, final int maxResults, final int maxAttributeChars) throws IOException {
|
||||
if (retrievalCount.get() >= maxResults) {
|
||||
return Collections.emptySet();
|
||||
|
@ -65,7 +66,7 @@ class DocsReader {
|
|||
|
||||
final long readDocuments = System.nanoTime() - start;
|
||||
logger.debug("Reading {} Lucene Documents took {} millis", docs.size(), TimeUnit.NANOSECONDS.toMillis(readDocuments));
|
||||
return read(docs, allProvenanceLogFiles, retrievalCount, maxResults, maxAttributeChars);
|
||||
return read(docs, authCheck, allProvenanceLogFiles, retrievalCount, maxResults, maxAttributeChars);
|
||||
}
|
||||
|
||||
|
||||
|
@ -104,7 +105,7 @@ class DocsReader {
|
|||
return record;
|
||||
}
|
||||
|
||||
public Set<ProvenanceEventRecord> read(final List<Document> docs, final Collection<Path> allProvenanceLogFiles,
|
||||
public Set<ProvenanceEventRecord> read(final List<Document> docs, final AuthorizationCheck authCheck, final Collection<Path> allProvenanceLogFiles,
|
||||
final AtomicInteger retrievalCount, final int maxResults, final int maxAttributeChars) throws IOException {
|
||||
|
||||
if (retrievalCount.get() >= maxResults) {
|
||||
|
@ -128,9 +129,9 @@ class DocsReader {
|
|||
|
||||
Iterator<Document> docIter = byStorageNameDocGroups.get(storageFileName).iterator();
|
||||
while (docIter.hasNext() && retrievalCount.getAndIncrement() < maxResults) {
|
||||
ProvenanceEventRecord eRec = this.getRecord(docIter.next(), reader);
|
||||
if (eRec != null) {
|
||||
matchingRecords.add(eRec);
|
||||
ProvenanceEventRecord event = this.getRecord(docIter.next(), reader);
|
||||
if (event != null && authCheck.isAuthorized(event)) {
|
||||
matchingRecords.add(event);
|
||||
eventsReadThisFile++;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,9 +28,11 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.provenance.PersistentProvenanceRepository;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.StandardQueryResult;
|
||||
import org.apache.nifi.provenance.authorization.AuthorizationCheck;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -48,7 +50,8 @@ public class IndexSearch {
|
|||
this.maxAttributeChars = maxAttributeChars;
|
||||
}
|
||||
|
||||
public StandardQueryResult search(final org.apache.nifi.provenance.search.Query provenanceQuery, final AtomicInteger retrievedCount, final long firstEventTimestamp) throws IOException {
|
||||
public StandardQueryResult search(final org.apache.nifi.provenance.search.Query provenanceQuery, final NiFiUser user, final AtomicInteger retrievedCount,
|
||||
final long firstEventTimestamp) throws IOException {
|
||||
if (retrievedCount.get() >= provenanceQuery.getMaxResults()) {
|
||||
final StandardQueryResult sqr = new StandardQueryResult(provenanceQuery, 1);
|
||||
sqr.update(Collections.<ProvenanceEventRecord> emptyList(), 0L);
|
||||
|
@ -102,7 +105,10 @@ public class IndexSearch {
|
|||
}
|
||||
|
||||
final DocsReader docsReader = new DocsReader();
|
||||
matchingRecords = docsReader.read(topDocs, searcher.getIndexReader(), repository.getAllLogFiles(), retrievedCount,
|
||||
|
||||
final AuthorizationCheck authCheck = event -> repository.isAuthorized(event, user);
|
||||
|
||||
matchingRecords = docsReader.read(topDocs, authCheck, searcher.getIndexReader(), repository.getAllLogFiles(), retrievedCount,
|
||||
provenanceQuery.getMaxResults(), maxAttributeChars);
|
||||
|
||||
final long readRecordsNanos = System.nanoTime() - finishSearch;
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.lucene.search.TopDocs;
|
|||
import org.apache.nifi.provenance.PersistentProvenanceRepository;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.SearchableFields;
|
||||
import org.apache.nifi.provenance.authorization.AuthorizationCheck;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -93,8 +94,12 @@ public class LineageQuery {
|
|||
final TopDocs uuidQueryTopDocs = searcher.search(query, MAX_QUERY_RESULTS);
|
||||
final long searchEnd = System.nanoTime();
|
||||
|
||||
// Always authorized. We do this because we need to pull back the event, regardless of whether or not
|
||||
// the user is truly authorized, because instead of ignoring unauthorized events, we want to replace them.
|
||||
final AuthorizationCheck authCheck = event -> true;
|
||||
|
||||
final DocsReader docsReader = new DocsReader();
|
||||
final Set<ProvenanceEventRecord> recs = docsReader.read(uuidQueryTopDocs, searcher.getIndexReader(), repo.getAllLogFiles(),
|
||||
final Set<ProvenanceEventRecord> recs = docsReader.read(uuidQueryTopDocs, authCheck, searcher.getIndexReader(), repo.getAllLogFiles(),
|
||||
new AtomicInteger(0), Integer.MAX_VALUE, maxAttributeChars);
|
||||
|
||||
final long readDocsEnd = System.nanoTime();
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.nifi.provenance;
|
|||
import static org.apache.nifi.provenance.TestUtil.createFlowFile;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
|
@ -39,6 +40,7 @@ import java.util.concurrent.TimeUnit;
|
|||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.zip.GZIPOutputStream;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
|
@ -51,6 +53,8 @@ import org.apache.lucene.search.IndexSearcher;
|
|||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
import org.apache.nifi.authorization.AccessDeniedException;
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.events.EventReporter;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.provenance.lineage.EventNode;
|
||||
|
@ -173,7 +177,7 @@ public class TestPersistentProvenanceRepository {
|
|||
config.setJournalCount(10);
|
||||
config.setQueryThreadPoolSize(10);
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("abc", "xyz");
|
||||
|
@ -221,7 +225,7 @@ public class TestPersistentProvenanceRepository {
|
|||
System.out.println("Closing and re-initializing");
|
||||
repo.close();
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
System.out.println("Re-initialized");
|
||||
|
||||
final long fetchStart = System.nanoTime();
|
||||
|
@ -241,7 +245,7 @@ public class TestPersistentProvenanceRepository {
|
|||
config.setMaxEventFileCapacity(1L);
|
||||
config.setMaxEventFileLife(1, TimeUnit.SECONDS);
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("abc", "xyz");
|
||||
|
@ -267,7 +271,7 @@ public class TestPersistentProvenanceRepository {
|
|||
Thread.sleep(500L); // Give the repo time to shutdown (i.e., close all file handles, etc.)
|
||||
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
final List<ProvenanceEventRecord> recoveredRecords = repo.getEvents(0L, 12);
|
||||
|
||||
assertEquals(10, recoveredRecords.size());
|
||||
|
@ -290,7 +294,7 @@ public class TestPersistentProvenanceRepository {
|
|||
config.setMaxEventFileLife(2, TimeUnit.SECONDS);
|
||||
config.setSearchableFields(searchableFields);
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("abc", "xyz");
|
||||
|
@ -331,7 +335,7 @@ public class TestPersistentProvenanceRepository {
|
|||
query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.ComponentID, "XXXX"));
|
||||
query.setMaxResults(100);
|
||||
|
||||
final QueryResult result = repo.queryEvents(query);
|
||||
final QueryResult result = repo.queryEvents(query, createUser());
|
||||
assertEquals(2, result.getMatchingEvents().size());
|
||||
for (final ProvenanceEventRecord match : result.getMatchingEvents()) {
|
||||
System.out.println(match);
|
||||
|
@ -344,7 +348,7 @@ public class TestPersistentProvenanceRepository {
|
|||
config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS);
|
||||
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final String uuid = "00000000-0000-0000-0000-000000000000";
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
|
@ -375,7 +379,7 @@ public class TestPersistentProvenanceRepository {
|
|||
query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.TransitURI, "nifi://*"));
|
||||
query.setMaxResults(100);
|
||||
|
||||
final QueryResult result = repo.queryEvents(query);
|
||||
final QueryResult result = repo.queryEvents(query, createUser());
|
||||
assertEquals(10, result.getMatchingEvents().size());
|
||||
for (final ProvenanceEventRecord match : result.getMatchingEvents()) {
|
||||
System.out.println(match);
|
||||
|
@ -388,7 +392,7 @@ public class TestPersistentProvenanceRepository {
|
|||
config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS);
|
||||
config.setCompressOnRollover(true);
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final String uuid = "00000000-0000-0000-0000-000000000000";
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
|
@ -426,7 +430,7 @@ public class TestPersistentProvenanceRepository {
|
|||
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
|
||||
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final String uuid = "10000000-0000-0000-0000-000000000000";
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
|
@ -458,7 +462,7 @@ public class TestPersistentProvenanceRepository {
|
|||
query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.TransitURI, "nifi://*"));
|
||||
query.setMaxResults(100);
|
||||
|
||||
final QueryResult result = repo.queryEvents(query);
|
||||
final QueryResult result = repo.queryEvents(query, createUser());
|
||||
assertEquals(10, result.getMatchingEvents().size());
|
||||
for (final ProvenanceEventRecord match : result.getMatchingEvents()) {
|
||||
System.out.println(match);
|
||||
|
@ -471,7 +475,7 @@ public class TestPersistentProvenanceRepository {
|
|||
repo.purgeOldEvents();
|
||||
Thread.sleep(2000L);
|
||||
|
||||
final QueryResult newRecordSet = repo.queryEvents(query);
|
||||
final QueryResult newRecordSet = repo.queryEvents(query, createUser());
|
||||
assertTrue(newRecordSet.getMatchingEvents().isEmpty());
|
||||
}
|
||||
|
||||
|
@ -486,7 +490,7 @@ public class TestPersistentProvenanceRepository {
|
|||
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
|
||||
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final String uuid = "00000000-0000-0000-0000-000000000000";
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
|
@ -525,7 +529,7 @@ public class TestPersistentProvenanceRepository {
|
|||
query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.TransitURI, "nifi://*"));
|
||||
query.setMaxResults(100);
|
||||
|
||||
final QuerySubmission submission = repo.submitQuery(query);
|
||||
final QuerySubmission submission = repo.submitQuery(query, createUser());
|
||||
while (!submission.getResult().isFinished()) {
|
||||
Thread.sleep(100L);
|
||||
}
|
||||
|
@ -555,7 +559,7 @@ public class TestPersistentProvenanceRepository {
|
|||
Thread.sleep(2000L); // purge is async. Give it time to do its job.
|
||||
|
||||
query.setMaxResults(100);
|
||||
final QuerySubmission noResultSubmission = repo.submitQuery(query);
|
||||
final QuerySubmission noResultSubmission = repo.submitQuery(query, createUser());
|
||||
while (!noResultSubmission.getResult().isFinished()) {
|
||||
Thread.sleep(10L);
|
||||
}
|
||||
|
@ -573,7 +577,7 @@ public class TestPersistentProvenanceRepository {
|
|||
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
|
||||
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final String uuid = "00000000-0000-0000-0000-000000000000";
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
|
@ -601,7 +605,7 @@ public class TestPersistentProvenanceRepository {
|
|||
final Query query = new Query(UUID.randomUUID().toString());
|
||||
query.setMaxResults(100);
|
||||
|
||||
final QueryResult result = repo.queryEvents(query);
|
||||
final QueryResult result = repo.queryEvents(query, createUser());
|
||||
assertEquals(10, result.getMatchingEvents().size());
|
||||
for (final ProvenanceEventRecord match : result.getMatchingEvents()) {
|
||||
System.out.println(match);
|
||||
|
@ -615,7 +619,7 @@ public class TestPersistentProvenanceRepository {
|
|||
|
||||
Thread.sleep(1000L);
|
||||
|
||||
final QueryResult newRecordSet = repo.queryEvents(query);
|
||||
final QueryResult newRecordSet = repo.queryEvents(query, createUser());
|
||||
assertTrue(newRecordSet.getMatchingEvents().isEmpty());
|
||||
}
|
||||
|
||||
|
@ -629,7 +633,7 @@ public class TestPersistentProvenanceRepository {
|
|||
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
|
||||
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final String uuid = "00000000-0000-0000-0000-000000000001";
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
|
@ -655,7 +659,7 @@ public class TestPersistentProvenanceRepository {
|
|||
|
||||
repo.waitForRollover();
|
||||
|
||||
final Lineage lineage = repo.computeLineage(uuid);
|
||||
final Lineage lineage = repo.computeLineage(uuid, createUser());
|
||||
assertNotNull(lineage);
|
||||
|
||||
// Nodes should consist of a RECEIVE followed by FlowFileNode, followed by a DROP
|
||||
|
@ -684,7 +688,7 @@ public class TestPersistentProvenanceRepository {
|
|||
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
|
||||
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final String uuid = "00000000-0000-0000-0000-000000000001";
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
|
@ -710,7 +714,7 @@ public class TestPersistentProvenanceRepository {
|
|||
|
||||
repo.waitForRollover();
|
||||
|
||||
final AsyncLineageSubmission submission = repo.submitLineageComputation(uuid);
|
||||
final AsyncLineageSubmission submission = repo.submitLineageComputation(uuid, createUser());
|
||||
while (!submission.getResult().isFinished()) {
|
||||
Thread.sleep(100L);
|
||||
}
|
||||
|
@ -743,7 +747,7 @@ public class TestPersistentProvenanceRepository {
|
|||
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
|
||||
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final String childId = "00000000-0000-0000-0000-000000000000";
|
||||
|
||||
|
@ -773,7 +777,7 @@ public class TestPersistentProvenanceRepository {
|
|||
|
||||
repo.waitForRollover();
|
||||
|
||||
final Lineage lineage = repo.computeLineage(childId);
|
||||
final Lineage lineage = repo.computeLineage(childId, createUser());
|
||||
assertNotNull(lineage);
|
||||
|
||||
// these are not necessarily accurate asserts....
|
||||
|
@ -793,7 +797,7 @@ public class TestPersistentProvenanceRepository {
|
|||
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
|
||||
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final String childId = "00000000-0000-0000-0000-000000000000";
|
||||
|
||||
|
@ -823,7 +827,7 @@ public class TestPersistentProvenanceRepository {
|
|||
|
||||
repo.waitForRollover();
|
||||
|
||||
final AsyncLineageSubmission submission = repo.submitLineageComputation(childId);
|
||||
final AsyncLineageSubmission submission = repo.submitLineageComputation(childId, createUser());
|
||||
while (!submission.getResult().isFinished()) {
|
||||
Thread.sleep(100L);
|
||||
}
|
||||
|
@ -840,7 +844,7 @@ public class TestPersistentProvenanceRepository {
|
|||
final RepositoryConfiguration config = createConfiguration();
|
||||
config.setMaxEventFileLife(1, TimeUnit.SECONDS);
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final String uuid = "00000000-0000-0000-0000-000000000000";
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
|
@ -866,13 +870,13 @@ public class TestPersistentProvenanceRepository {
|
|||
repo.close();
|
||||
|
||||
final PersistentProvenanceRepository secondRepo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
secondRepo.initialize(getEventReporter());
|
||||
secondRepo.initialize(getEventReporter(), null, null);
|
||||
|
||||
try {
|
||||
final ProvenanceEventRecord event11 = builder.build();
|
||||
secondRepo.registerEvent(event11);
|
||||
secondRepo.waitForRollover();
|
||||
final ProvenanceEventRecord event11Retrieved = secondRepo.getEvent(10L);
|
||||
final ProvenanceEventRecord event11Retrieved = secondRepo.getEvent(10L, null);
|
||||
assertNotNull(event11Retrieved);
|
||||
assertEquals(10, event11Retrieved.getEventId());
|
||||
} finally {
|
||||
|
@ -897,7 +901,7 @@ public class TestPersistentProvenanceRepository {
|
|||
in.writeInt(4);
|
||||
in.close();
|
||||
assertTrue(eventFile.exists());
|
||||
final QueryResult result = repo.queryEvents(query);
|
||||
final QueryResult result = repo.queryEvents(query, createUser());
|
||||
assertEquals(10, result.getMatchingEvents().size());
|
||||
}
|
||||
|
||||
|
@ -914,7 +918,7 @@ public class TestPersistentProvenanceRepository {
|
|||
query.setMaxResults(100);
|
||||
DataOutputStream in = new DataOutputStream(new GZIPOutputStream(new FileOutputStream(eventFile)));
|
||||
in.close();
|
||||
final QueryResult result = repo.queryEvents(query);
|
||||
final QueryResult result = repo.queryEvents(query, createUser());
|
||||
assertEquals(10, result.getMatchingEvents().size());
|
||||
}
|
||||
|
||||
|
@ -927,7 +931,7 @@ public class TestPersistentProvenanceRepository {
|
|||
config.setDesiredIndexSize(10);
|
||||
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
String uuid = UUID.randomUUID().toString();
|
||||
for (int i = 0; i < 20; i++) {
|
||||
|
@ -957,7 +961,7 @@ public class TestPersistentProvenanceRepository {
|
|||
config.setDesiredIndexSize(10); // force new index to be created for each rollover
|
||||
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final String uuid = "00000000-0000-0000-0000-000000000000";
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
|
@ -1003,7 +1007,7 @@ public class TestPersistentProvenanceRepository {
|
|||
query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.TransitURI, "nifi://*"));
|
||||
query.setMaxResults(100);
|
||||
|
||||
final QueryResult result = repo.queryEvents(query);
|
||||
final QueryResult result = repo.queryEvents(query, createUser());
|
||||
assertEquals(20, result.getMatchingEvents().size());
|
||||
|
||||
// Ensure index directories exists
|
||||
|
@ -1022,7 +1026,7 @@ public class TestPersistentProvenanceRepository {
|
|||
repo.purgeOldEvents();
|
||||
Thread.sleep(2000L);
|
||||
|
||||
final QueryResult newRecordSet = repo.queryEvents(query);
|
||||
final QueryResult newRecordSet = repo.queryEvents(query, createUser());
|
||||
assertEquals(10, newRecordSet.getMatchingEvents().size());
|
||||
|
||||
// Ensure that one index directory is gone
|
||||
|
@ -1030,6 +1034,291 @@ public class TestPersistentProvenanceRepository {
|
|||
assertEquals(1, indexDirs.length);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNotAuthorizedGetSpecificEvent() throws IOException {
|
||||
final RepositoryConfiguration config = createConfiguration();
|
||||
config.setMaxRecordLife(5, TimeUnit.MINUTES);
|
||||
config.setMaxStorageCapacity(1024L * 1024L);
|
||||
config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS);
|
||||
config.setMaxEventFileCapacity(1024L * 1024L);
|
||||
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
|
||||
config.setDesiredIndexSize(10); // force new index to be created for each rollover
|
||||
|
||||
final AccessDeniedException expectedException = new AccessDeniedException("Unit Test - Intentionally Thrown");
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS) {
|
||||
@Override
|
||||
protected void authorize(ProvenanceEventRecord event, NiFiUser user) {
|
||||
throw expectedException;
|
||||
}
|
||||
};
|
||||
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final String uuid = "00000000-0000-0000-0000-000000000000";
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("abc", "xyz");
|
||||
attributes.put("xyz", "abc");
|
||||
attributes.put("filename", "file-" + uuid);
|
||||
|
||||
final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder();
|
||||
builder.setEventTime(System.currentTimeMillis());
|
||||
builder.setEventType(ProvenanceEventType.RECEIVE);
|
||||
builder.setTransitUri("nifi://unit-test");
|
||||
builder.fromFlowFile(createFlowFile(3L, 3000L, attributes));
|
||||
builder.setComponentId("1234");
|
||||
builder.setComponentType("dummy processor");
|
||||
|
||||
for (int i = 0; i < 10; i++) {
|
||||
attributes.put("uuid", "00000000-0000-0000-0000-00000000000" + i);
|
||||
builder.fromFlowFile(createFlowFile(i, 3000L, attributes));
|
||||
builder.setEventTime(10L); // make sure the events are destroyed when we call purge
|
||||
repo.registerEvent(builder.build());
|
||||
}
|
||||
|
||||
repo.waitForRollover();
|
||||
|
||||
try {
|
||||
repo.getEvent(0L, null);
|
||||
Assert.fail("getEvent() did not throw an Exception");
|
||||
} catch (final Exception e) {
|
||||
Assert.assertSame(expectedException, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNotAuthorizedGetEventRange() throws IOException {
|
||||
final RepositoryConfiguration config = createConfiguration();
|
||||
config.setMaxRecordLife(5, TimeUnit.MINUTES);
|
||||
config.setMaxStorageCapacity(1024L * 1024L);
|
||||
config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS);
|
||||
config.setMaxEventFileCapacity(1024L * 1024L);
|
||||
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
|
||||
config.setDesiredIndexSize(10); // force new index to be created for each rollover
|
||||
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS) {
|
||||
@Override
|
||||
public boolean isAuthorized(ProvenanceEventRecord event, NiFiUser user) {
|
||||
return event.getEventId() > 2;
|
||||
}
|
||||
};
|
||||
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final String uuid = "00000000-0000-0000-0000-000000000000";
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("abc", "xyz");
|
||||
attributes.put("xyz", "abc");
|
||||
attributes.put("filename", "file-" + uuid);
|
||||
|
||||
final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder();
|
||||
builder.setEventTime(System.currentTimeMillis());
|
||||
builder.setEventType(ProvenanceEventType.RECEIVE);
|
||||
builder.setTransitUri("nifi://unit-test");
|
||||
builder.fromFlowFile(createFlowFile(3L, 3000L, attributes));
|
||||
builder.setComponentId("1234");
|
||||
builder.setComponentType("dummy processor");
|
||||
|
||||
for (int i = 0; i < 10; i++) {
|
||||
attributes.put("uuid", "00000000-0000-0000-0000-00000000000" + i);
|
||||
builder.fromFlowFile(createFlowFile(i, 3000L, attributes));
|
||||
builder.setEventTime(10L); // make sure the events are destroyed when we call purge
|
||||
repo.registerEvent(builder.build());
|
||||
}
|
||||
|
||||
repo.waitForRollover();
|
||||
|
||||
final List<ProvenanceEventRecord> events = repo.getEvents(0L, 10, null);
|
||||
|
||||
// Ensure that we gets events with ID's 3 through 10.
|
||||
assertEquals(7, events.size());
|
||||
final List<Long> eventIds = events.stream().map(event -> event.getEventId()).sorted().collect(Collectors.toList());
|
||||
for (int i = 0; i < 7; i++) {
|
||||
Assert.assertEquals(i + 3, eventIds.get(i).intValue());
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout = 10000)
|
||||
public void testNotAuthorizedQuery() throws IOException, InterruptedException {
|
||||
final RepositoryConfiguration config = createConfiguration();
|
||||
config.setMaxRecordLife(5, TimeUnit.MINUTES);
|
||||
config.setMaxStorageCapacity(1024L * 1024L);
|
||||
config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS);
|
||||
config.setMaxEventFileCapacity(1024L * 1024L);
|
||||
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
|
||||
config.setDesiredIndexSize(10); // force new index to be created for each rollover
|
||||
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS) {
|
||||
@Override
|
||||
public boolean isAuthorized(ProvenanceEventRecord event, NiFiUser user) {
|
||||
return event.getEventId() > 2;
|
||||
}
|
||||
};
|
||||
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final String uuid = "00000000-0000-0000-0000-000000000000";
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("abc", "xyz");
|
||||
attributes.put("xyz", "abc");
|
||||
attributes.put("filename", "file-" + uuid);
|
||||
|
||||
final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder();
|
||||
builder.setEventTime(System.currentTimeMillis());
|
||||
builder.setEventType(ProvenanceEventType.RECEIVE);
|
||||
builder.setTransitUri("nifi://unit-test");
|
||||
builder.fromFlowFile(createFlowFile(3L, 3000L, attributes));
|
||||
builder.setComponentId("1234");
|
||||
builder.setComponentType("dummy processor");
|
||||
|
||||
for (int i = 0; i < 10; i++) {
|
||||
attributes.put("uuid", "00000000-0000-0000-0000-00000000000" + i);
|
||||
builder.fromFlowFile(createFlowFile(i, 3000L, attributes));
|
||||
builder.setEventTime(10L); // make sure the events are destroyed when we call purge
|
||||
repo.registerEvent(builder.build());
|
||||
}
|
||||
|
||||
repo.waitForRollover();
|
||||
|
||||
final Query query = new Query("1234");
|
||||
query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.ComponentID, "1234"));
|
||||
final QuerySubmission submission = repo.submitQuery(query, createUser());
|
||||
|
||||
final QueryResult result = submission.getResult();
|
||||
while (!result.isFinished()) {
|
||||
Thread.sleep(100L);
|
||||
}
|
||||
|
||||
// Ensure that we gets events with ID's 3 through 10.
|
||||
final List<ProvenanceEventRecord> events = result.getMatchingEvents();
|
||||
assertEquals(7, events.size());
|
||||
final List<Long> eventIds = events.stream().map(event -> event.getEventId()).sorted().collect(Collectors.toList());
|
||||
for (int i = 0; i < 7; i++) {
|
||||
Assert.assertEquals(i + 3, eventIds.get(i).intValue());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test(timeout = 1000000)
|
||||
public void testNotAuthorizedLineage() throws IOException, InterruptedException {
|
||||
final RepositoryConfiguration config = createConfiguration();
|
||||
config.setMaxRecordLife(5, TimeUnit.MINUTES);
|
||||
config.setMaxStorageCapacity(1024L * 1024L);
|
||||
config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS);
|
||||
config.setMaxEventFileCapacity(1024L * 1024L);
|
||||
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
|
||||
config.setDesiredIndexSize(10); // force new index to be created for each rollover
|
||||
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS) {
|
||||
@Override
|
||||
public boolean isAuthorized(ProvenanceEventRecord event, NiFiUser user) {
|
||||
return event.getEventType() != ProvenanceEventType.ATTRIBUTES_MODIFIED;
|
||||
}
|
||||
};
|
||||
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final String uuid = "00000000-0000-0000-0000-000000000000";
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("abc", "xyz");
|
||||
attributes.put("xyz", "abc");
|
||||
attributes.put("filename", "file-" + uuid);
|
||||
attributes.put("uuid", uuid);
|
||||
|
||||
final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder();
|
||||
builder.setEventTime(System.currentTimeMillis());
|
||||
builder.setEventType(ProvenanceEventType.RECEIVE);
|
||||
builder.setTransitUri("nifi://unit-test");
|
||||
builder.fromFlowFile(createFlowFile(3L, 3000L, attributes));
|
||||
builder.setComponentId("1234");
|
||||
builder.setComponentType("dummy processor");
|
||||
builder.setEventTime(10L); // make sure the events are destroyed when we call purge
|
||||
|
||||
builder.fromFlowFile(createFlowFile(1, 3000L, attributes));
|
||||
repo.registerEvent(builder.build());
|
||||
|
||||
builder.setEventType(ProvenanceEventType.CONTENT_MODIFIED);
|
||||
builder.fromFlowFile(createFlowFile(2, 2000L, attributes));
|
||||
repo.registerEvent(builder.build());
|
||||
|
||||
builder.setEventType(ProvenanceEventType.CONTENT_MODIFIED);
|
||||
builder.fromFlowFile(createFlowFile(3, 2000L, attributes));
|
||||
repo.registerEvent(builder.build());
|
||||
|
||||
builder.setEventType(ProvenanceEventType.ATTRIBUTES_MODIFIED);
|
||||
attributes.put("new-attr", "yes");
|
||||
builder.fromFlowFile(createFlowFile(4, 2000L, attributes));
|
||||
repo.registerEvent(builder.build());
|
||||
|
||||
final Map<String, String> childAttributes = new HashMap<>(attributes);
|
||||
childAttributes.put("uuid", "00000000-0000-0000-0000-000000000001");
|
||||
builder.setEventType(ProvenanceEventType.FORK);
|
||||
builder.fromFlowFile(createFlowFile(4, 2000L, attributes));
|
||||
builder.addChildFlowFile(createFlowFile(5, 2000L, childAttributes));
|
||||
builder.addParentFlowFile(createFlowFile(4, 2000L, attributes));
|
||||
repo.registerEvent(builder.build());
|
||||
|
||||
builder.setEventType(ProvenanceEventType.ATTRIBUTES_MODIFIED);
|
||||
builder.fromFlowFile(createFlowFile(6, 2000L, childAttributes));
|
||||
repo.registerEvent(builder.build());
|
||||
|
||||
builder.setEventType(ProvenanceEventType.DROP);
|
||||
builder.fromFlowFile(createFlowFile(6, 2000L, childAttributes));
|
||||
repo.registerEvent(builder.build());
|
||||
|
||||
repo.waitForRollover();
|
||||
|
||||
final AsyncLineageSubmission originalLineage = repo.submitLineageComputation(uuid, createUser());
|
||||
|
||||
final StandardLineageResult result = originalLineage.getResult();
|
||||
while (!result.isFinished()) {
|
||||
Thread.sleep(100L);
|
||||
}
|
||||
|
||||
final List<LineageNode> lineageNodes = result.getNodes();
|
||||
assertEquals(6, lineageNodes.size());
|
||||
|
||||
assertEquals(1, lineageNodes.stream().map(node -> node.getNodeType()).filter(t -> t == LineageNodeType.FLOWFILE_NODE).count());
|
||||
assertEquals(5, lineageNodes.stream().map(node -> node.getNodeType()).filter(t -> t == LineageNodeType.PROVENANCE_EVENT_NODE).count());
|
||||
|
||||
final Set<EventNode> eventNodes = lineageNodes.stream()
|
||||
.filter(node -> node.getNodeType() == LineageNodeType.PROVENANCE_EVENT_NODE)
|
||||
.map(node -> (EventNode) node)
|
||||
.collect(Collectors.toSet());
|
||||
|
||||
final Map<ProvenanceEventType, List<EventNode>> nodesByType = eventNodes.stream().collect(Collectors.groupingBy(EventNode::getEventType));
|
||||
assertEquals(1, nodesByType.get(ProvenanceEventType.RECEIVE).size());
|
||||
assertEquals(2, nodesByType.get(ProvenanceEventType.CONTENT_MODIFIED).size());
|
||||
assertEquals(1, nodesByType.get(ProvenanceEventType.FORK).size());
|
||||
|
||||
assertEquals(1, nodesByType.get(ProvenanceEventType.UNKNOWN).size());
|
||||
assertNull(nodesByType.get(ProvenanceEventType.ATTRIBUTES_MODIFIED));
|
||||
|
||||
// Test filtering on expandChildren
|
||||
final AsyncLineageSubmission expandChild = repo.submitExpandChildren(4L, createUser());
|
||||
final StandardLineageResult expandChildResult = expandChild.getResult();
|
||||
while (!expandChildResult.isFinished()) {
|
||||
Thread.sleep(100L);
|
||||
}
|
||||
|
||||
final List<LineageNode> expandChildNodes = expandChildResult.getNodes();
|
||||
assertEquals(4, expandChildNodes.size());
|
||||
|
||||
assertEquals(1, expandChildNodes.stream().map(node -> node.getNodeType()).filter(t -> t == LineageNodeType.FLOWFILE_NODE).count());
|
||||
assertEquals(3, expandChildNodes.stream().map(node -> node.getNodeType()).filter(t -> t == LineageNodeType.PROVENANCE_EVENT_NODE).count());
|
||||
|
||||
final Set<EventNode> childEventNodes = expandChildNodes.stream()
|
||||
.filter(node -> node.getNodeType() == LineageNodeType.PROVENANCE_EVENT_NODE)
|
||||
.map(node -> (EventNode) node)
|
||||
.collect(Collectors.toSet());
|
||||
|
||||
final Map<ProvenanceEventType, List<EventNode>> childNodesByType = childEventNodes.stream().collect(Collectors.groupingBy(EventNode::getEventType));
|
||||
assertEquals(1, childNodesByType.get(ProvenanceEventType.FORK).size());
|
||||
assertEquals(1, childNodesByType.get(ProvenanceEventType.DROP).size());
|
||||
assertEquals(1, childNodesByType.get(ProvenanceEventType.UNKNOWN).size());
|
||||
assertNull(childNodesByType.get(ProvenanceEventType.ATTRIBUTES_MODIFIED));
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Test
|
||||
public void testBackPressure() throws IOException, InterruptedException {
|
||||
|
@ -1045,7 +1334,7 @@ public class TestPersistentProvenanceRepository {
|
|||
return journalCountRef.get();
|
||||
}
|
||||
};
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder();
|
||||
|
@ -1103,7 +1392,7 @@ public class TestPersistentProvenanceRepository {
|
|||
config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS);
|
||||
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final String uuid = "00000000-0000-0000-0000-000000000000";
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
|
@ -1170,7 +1459,7 @@ public class TestPersistentProvenanceRepository {
|
|||
final RepositoryConfiguration config = createConfiguration();
|
||||
config.setMaxEventFileLife(3, TimeUnit.SECONDS);
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
|
||||
|
@ -1221,7 +1510,7 @@ public class TestPersistentProvenanceRepository {
|
|||
config.setMaxAttributeChars(50);
|
||||
config.setMaxEventFileLife(3, TimeUnit.SECONDS);
|
||||
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("75chars", "123456789012345678901234567890123456789012345678901234567890123456789012345");
|
||||
|
@ -1240,7 +1529,7 @@ public class TestPersistentProvenanceRepository {
|
|||
repo.registerEvent(record);
|
||||
repo.waitForRollover();
|
||||
|
||||
final ProvenanceEventRecord retrieved = repo.getEvent(0L);
|
||||
final ProvenanceEventRecord retrieved = repo.getEvent(0L, null);
|
||||
assertNotNull(retrieved);
|
||||
assertEquals("12345678-0000-0000-0000-012345678912", retrieved.getAttributes().get("uuid"));
|
||||
assertEquals("12345678901234567890123456789012345678901234567890", retrieved.getAttributes().get("75chars"));
|
||||
|
@ -1269,7 +1558,7 @@ public class TestPersistentProvenanceRepository {
|
|||
};
|
||||
|
||||
// initialize with our event reporter
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
// create some events in the journal files.
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
|
@ -1346,7 +1635,7 @@ public class TestPersistentProvenanceRepository {
|
|||
return spiedWriters;
|
||||
}
|
||||
};
|
||||
repo.initialize(getEventReporter());
|
||||
repo.initialize(getEventReporter(), null, null);
|
||||
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("75chars", "123456789012345678901234567890123456789012345678901234567890123456789012345");
|
||||
|
@ -1407,7 +1696,7 @@ public class TestPersistentProvenanceRepository {
|
|||
};
|
||||
|
||||
try {
|
||||
recoveryRepo.initialize(getEventReporter());
|
||||
recoveryRepo.initialize(getEventReporter(), null, null);
|
||||
} finally {
|
||||
recoveryRepo.close();
|
||||
}
|
||||
|
@ -1437,4 +1726,29 @@ public class TestPersistentProvenanceRepository {
|
|||
return severity;
|
||||
}
|
||||
}
|
||||
|
||||
private NiFiUser createUser() {
|
||||
return new NiFiUser() {
|
||||
@Override
|
||||
public String getIdentity() {
|
||||
return "unit-test";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getUserName() {
|
||||
return "Unit Test";
|
||||
}
|
||||
|
||||
@Override
|
||||
public NiFiUser getChain() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAnonymous() {
|
||||
return false;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -37,6 +37,13 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.nifi.authorization.AccessDeniedException;
|
||||
import org.apache.nifi.authorization.AuthorizationResult;
|
||||
import org.apache.nifi.authorization.AuthorizationResult.Result;
|
||||
import org.apache.nifi.authorization.Authorizer;
|
||||
import org.apache.nifi.authorization.RequestAction;
|
||||
import org.apache.nifi.authorization.resource.Authorizable;
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.events.EventReporter;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
|
@ -55,6 +62,7 @@ import org.apache.nifi.util.RingBuffer;
|
|||
import org.apache.nifi.util.RingBuffer.Filter;
|
||||
import org.apache.nifi.util.RingBuffer.ForEachEvaluator;
|
||||
import org.apache.nifi.util.RingBuffer.IterationDirection;
|
||||
import org.apache.nifi.web.ResourceNotFoundException;
|
||||
|
||||
public class VolatileProvenanceRepository implements ProvenanceEventRepository {
|
||||
|
||||
|
@ -75,6 +83,9 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
|
|||
private final AtomicLong idGenerator = new AtomicLong(0L);
|
||||
private final AtomicBoolean initialized = new AtomicBoolean(false);
|
||||
|
||||
private Authorizer authorizer; // effectively final
|
||||
private ProvenanceAuthorizableFactory resourceFactory; // effectively final
|
||||
|
||||
public VolatileProvenanceRepository() {
|
||||
final NiFiProperties properties = NiFiProperties.getInstance();
|
||||
|
||||
|
@ -103,11 +114,14 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void initialize(final EventReporter eventReporter) {
|
||||
public void initialize(final EventReporter eventReporter, final Authorizer authorizer, final ProvenanceAuthorizableFactory resourceFactory) throws IOException {
|
||||
if (initialized.getAndSet(true)) {
|
||||
return;
|
||||
}
|
||||
|
||||
this.authorizer = authorizer;
|
||||
this.resourceFactory = resourceFactory;
|
||||
|
||||
scheduledExecService.scheduleWithFixedDelay(new RemoveExpiredQueryResults(), 30L, 30L, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
|
@ -131,9 +145,18 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
|
|||
|
||||
@Override
|
||||
public List<ProvenanceEventRecord> getEvents(final long firstRecordId, final int maxRecords) throws IOException {
|
||||
return getEvents(firstRecordId, maxRecords, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ProvenanceEventRecord> getEvents(final long firstRecordId, final int maxRecords, final NiFiUser user) throws IOException {
|
||||
return ringBuffer.getSelectedElements(new Filter<ProvenanceEventRecord>() {
|
||||
@Override
|
||||
public boolean select(final ProvenanceEventRecord value) {
|
||||
if (user != null && !isAuthorized(value, user)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return value.getEventId() >= firstRecordId;
|
||||
}
|
||||
}, maxRecords);
|
||||
|
@ -155,8 +178,7 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
|
|||
return records.isEmpty() ? null : records.get(0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ProvenanceEventRecord getEvent(final long id) {
|
||||
private ProvenanceEventRecord getEvent(final long id) {
|
||||
final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(new Filter<ProvenanceEventRecord>() {
|
||||
@Override
|
||||
public boolean select(final ProvenanceEventRecord event) {
|
||||
|
@ -167,6 +189,17 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
|
|||
return records.isEmpty() ? null : records.get(0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ProvenanceEventRecord getEvent(final long id, final NiFiUser user) {
|
||||
final ProvenanceEventRecord event = getEvent(id);
|
||||
if (event == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
authorize(event, user);
|
||||
return event;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
queryExecService.shutdownNow();
|
||||
|
@ -183,8 +216,8 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
|
|||
return searchableAttributes;
|
||||
}
|
||||
|
||||
public QueryResult queryEvents(final Query query) throws IOException {
|
||||
final QuerySubmission submission = submitQuery(query);
|
||||
public QueryResult queryEvents(final Query query, final NiFiUser user) throws IOException {
|
||||
final QuerySubmission submission = submitQuery(query, user);
|
||||
final QueryResult result = submission.getResult();
|
||||
while (!result.isFinished()) {
|
||||
try {
|
||||
|
@ -200,10 +233,40 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
|
|||
return result;
|
||||
}
|
||||
|
||||
private Filter<ProvenanceEventRecord> createFilter(final Query query) {
|
||||
|
||||
public boolean isAuthorized(final ProvenanceEventRecord event, final NiFiUser user) {
|
||||
if (authorizer == null) {
|
||||
return true;
|
||||
}
|
||||
|
||||
final Authorizable eventAuthorizable;
|
||||
try {
|
||||
eventAuthorizable = resourceFactory.createProvenanceAuthorizable(event.getComponentId());
|
||||
} catch (final ResourceNotFoundException rnfe) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final AuthorizationResult result = eventAuthorizable.checkAuthorization(authorizer, RequestAction.READ, user);
|
||||
return Result.Approved.equals(result.getResult());
|
||||
}
|
||||
|
||||
protected void authorize(final ProvenanceEventRecord event, final NiFiUser user) {
|
||||
if (authorizer == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final Authorizable eventAuthorizable = resourceFactory.createProvenanceAuthorizable(event.getComponentId());
|
||||
eventAuthorizable.authorize(authorizer, RequestAction.READ, user);
|
||||
}
|
||||
|
||||
private Filter<ProvenanceEventRecord> createFilter(final Query query, final NiFiUser user) {
|
||||
return new Filter<ProvenanceEventRecord>() {
|
||||
@Override
|
||||
public boolean select(final ProvenanceEventRecord event) {
|
||||
if (!isAuthorized(event, user)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (query.getStartDate() != null && query.getStartDate().getTime() > event.getEventTime()) {
|
||||
return false;
|
||||
}
|
||||
|
@ -348,36 +411,51 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
|
|||
}
|
||||
|
||||
@Override
|
||||
public QuerySubmission submitQuery(final Query query) {
|
||||
public QuerySubmission submitQuery(final Query query, final NiFiUser user) {
|
||||
if (query.getEndDate() != null && query.getStartDate() != null && query.getStartDate().getTime() > query.getEndDate().getTime()) {
|
||||
throw new IllegalArgumentException("Query End Time cannot be before Query Start Time");
|
||||
}
|
||||
|
||||
if (query.getSearchTerms().isEmpty() && query.getStartDate() == null && query.getEndDate() == null) {
|
||||
final AsyncQuerySubmission result = new AsyncQuerySubmission(query, 1);
|
||||
queryExecService.submit(new QueryRunnable(ringBuffer, createFilter(query), query.getMaxResults(), result));
|
||||
final AsyncQuerySubmission result = new AsyncQuerySubmission(query, 1, user.getIdentity());
|
||||
queryExecService.submit(new QueryRunnable(ringBuffer, createFilter(query, user), query.getMaxResults(), result));
|
||||
querySubmissionMap.put(query.getIdentifier(), result);
|
||||
return result;
|
||||
}
|
||||
|
||||
final AsyncQuerySubmission result = new AsyncQuerySubmission(query, 1);
|
||||
final AsyncQuerySubmission result = new AsyncQuerySubmission(query, 1, user.getIdentity());
|
||||
querySubmissionMap.put(query.getIdentifier(), result);
|
||||
queryExecService.submit(new QueryRunnable(ringBuffer, createFilter(query), query.getMaxResults(), result));
|
||||
queryExecService.submit(new QueryRunnable(ringBuffer, createFilter(query, user), query.getMaxResults(), result));
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public QuerySubmission retrieveQuerySubmission(final String queryIdentifier) {
|
||||
return querySubmissionMap.get(queryIdentifier);
|
||||
public QuerySubmission retrieveQuerySubmission(final String queryIdentifier, final NiFiUser user) {
|
||||
final QuerySubmission submission = querySubmissionMap.get(queryIdentifier);
|
||||
final String userId = submission.getSubmitterIdentity();
|
||||
|
||||
if (user == null && userId == null) {
|
||||
return submission;
|
||||
}
|
||||
|
||||
if (user == null) {
|
||||
throw new AccessDeniedException("Cannot retrieve Provenance Query Submission because no user id was provided");
|
||||
}
|
||||
|
||||
if (userId == null || userId.equals(user.getIdentity())) {
|
||||
return submission;
|
||||
}
|
||||
|
||||
throw new AccessDeniedException("Cannot retrieve Provenance Query Submission because " + user.getIdentity() + " is not the user who submitted the request");
|
||||
}
|
||||
|
||||
public Lineage computeLineage(final String flowFileUUID) throws IOException {
|
||||
return computeLineage(Collections.<String>singleton(flowFileUUID), LineageComputationType.FLOWFILE_LINEAGE, null);
|
||||
public Lineage computeLineage(final String flowFileUUID, final NiFiUser user) throws IOException {
|
||||
return computeLineage(Collections.<String> singleton(flowFileUUID), user, LineageComputationType.FLOWFILE_LINEAGE, null);
|
||||
}
|
||||
|
||||
private Lineage computeLineage(final Collection<String> flowFileUuids, final LineageComputationType computationType, final Long eventId) throws IOException {
|
||||
final AsyncLineageSubmission submission = submitLineageComputation(flowFileUuids, computationType, eventId);
|
||||
private Lineage computeLineage(final Collection<String> flowFileUuids, final NiFiUser user, final LineageComputationType computationType, final Long eventId) throws IOException {
|
||||
final AsyncLineageSubmission submission = submitLineageComputation(flowFileUuids, user, computationType, eventId);
|
||||
final StandardLineageResult result = submission.getResult();
|
||||
while (!result.isFinished()) {
|
||||
try {
|
||||
|
@ -394,13 +472,28 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
|
|||
}
|
||||
|
||||
@Override
|
||||
public AsyncLineageSubmission submitLineageComputation(final String flowFileUuid) {
|
||||
return submitLineageComputation(Collections.singleton(flowFileUuid), LineageComputationType.FLOWFILE_LINEAGE, null);
|
||||
public AsyncLineageSubmission submitLineageComputation(final String flowFileUuid, final NiFiUser user) {
|
||||
return submitLineageComputation(Collections.singleton(flowFileUuid), user, LineageComputationType.FLOWFILE_LINEAGE, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ComputeLineageSubmission retrieveLineageSubmission(String lineageIdentifier) {
|
||||
return lineageSubmissionMap.get(lineageIdentifier);
|
||||
public ComputeLineageSubmission retrieveLineageSubmission(String lineageIdentifier, final NiFiUser user) {
|
||||
final ComputeLineageSubmission submission = lineageSubmissionMap.get(lineageIdentifier);
|
||||
final String userId = submission.getSubmitterIdentity();
|
||||
|
||||
if (user == null && userId == null) {
|
||||
return submission;
|
||||
}
|
||||
|
||||
if (user == null) {
|
||||
throw new AccessDeniedException("Cannot retrieve Provenance Query Submission because no user id was provided");
|
||||
}
|
||||
|
||||
if (userId == null || userId.equals(user.getIdentity())) {
|
||||
return submission;
|
||||
}
|
||||
|
||||
throw new AccessDeniedException("Cannot retrieve Provenance Query Submission because " + user.getIdentity() + " is not the user who submitted the request");
|
||||
}
|
||||
|
||||
public Lineage expandSpawnEventParents(String identifier) throws IOException {
|
||||
|
@ -408,10 +501,12 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ComputeLineageSubmission submitExpandParents(final long eventId) {
|
||||
final ProvenanceEventRecord event = getEvent(eventId);
|
||||
public ComputeLineageSubmission submitExpandParents(final long eventId, final NiFiUser user) {
|
||||
final String userId = user.getIdentity();
|
||||
|
||||
final ProvenanceEventRecord event = getEvent(eventId, user);
|
||||
if (event == null) {
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String>emptyList(), 1);
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String> emptyList(), 1, userId);
|
||||
lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
|
||||
submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList());
|
||||
return submission;
|
||||
|
@ -422,9 +517,9 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
|
|||
case FORK:
|
||||
case REPLAY:
|
||||
case CLONE:
|
||||
return submitLineageComputation(event.getParentUuids(), LineageComputationType.EXPAND_PARENTS, eventId);
|
||||
return submitLineageComputation(event.getParentUuids(), user, LineageComputationType.EXPAND_PARENTS, eventId);
|
||||
default: {
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String>emptyList(), 1);
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String> emptyList(), 1, userId);
|
||||
lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
|
||||
submission.getResult().setError("Event ID " + eventId + " indicates an event of type " + event.getEventType() + " so its parents cannot be expanded");
|
||||
return submission;
|
||||
|
@ -437,10 +532,12 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ComputeLineageSubmission submitExpandChildren(final long eventId) {
|
||||
final ProvenanceEventRecord event = getEvent(eventId);
|
||||
public ComputeLineageSubmission submitExpandChildren(final long eventId, final NiFiUser user) {
|
||||
final String userId = user.getIdentity();
|
||||
|
||||
final ProvenanceEventRecord event = getEvent(eventId, user);
|
||||
if (event == null) {
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String> emptyList(), 1, userId);
|
||||
lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
|
||||
submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList());
|
||||
return submission;
|
||||
|
@ -451,9 +548,9 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
|
|||
case FORK:
|
||||
case REPLAY:
|
||||
case CLONE:
|
||||
return submitLineageComputation(event.getChildUuids(), LineageComputationType.EXPAND_CHILDREN, eventId);
|
||||
return submitLineageComputation(event.getChildUuids(), user, LineageComputationType.EXPAND_CHILDREN, eventId);
|
||||
default: {
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
|
||||
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String> emptyList(), 1, userId);
|
||||
lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
|
||||
submission.getResult().setError("Event ID " + eventId + " indicates an event of type " + event.getEventType() + " so its children cannot be expanded");
|
||||
return submission;
|
||||
|
@ -461,13 +558,18 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
|
|||
}
|
||||
}
|
||||
|
||||
private AsyncLineageSubmission submitLineageComputation(final Collection<String> flowFileUuids, final LineageComputationType computationType, final Long eventId) {
|
||||
final AsyncLineageSubmission result = new AsyncLineageSubmission(computationType, eventId, flowFileUuids, 1);
|
||||
private AsyncLineageSubmission submitLineageComputation(final Collection<String> flowFileUuids, final NiFiUser user, final LineageComputationType computationType, final Long eventId) {
|
||||
final String userId = user.getIdentity();
|
||||
final AsyncLineageSubmission result = new AsyncLineageSubmission(computationType, eventId, flowFileUuids, 1, userId);
|
||||
lineageSubmissionMap.put(result.getLineageIdentifier(), result);
|
||||
|
||||
final Filter<ProvenanceEventRecord> filter = new Filter<ProvenanceEventRecord>() {
|
||||
@Override
|
||||
public boolean select(final ProvenanceEventRecord event) {
|
||||
if (user != null && !isAuthorized(event, user)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (flowFileUuids.contains(event.getFlowFileUuid())) {
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.provenance.search.Query;
|
||||
import org.apache.nifi.provenance.search.QuerySubmission;
|
||||
|
@ -70,7 +71,7 @@ public class TestVolatileProvenanceRepository {
|
|||
assertEquals(10, retrieved.size());
|
||||
for (int i = 0; i < 10; i++) {
|
||||
final ProvenanceEventRecord recovered = retrieved.get(i);
|
||||
assertEquals((long) i, recovered.getEventId());
|
||||
assertEquals(i, recovered.getEventId());
|
||||
assertEquals("nifi://unit-test", recovered.getTransitUri());
|
||||
assertEquals(ProvenanceEventType.RECEIVE, recovered.getEventType());
|
||||
assertEquals(attributes, recovered.getAttributes());
|
||||
|
@ -108,7 +109,7 @@ public class TestVolatileProvenanceRepository {
|
|||
query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.TransitURI, "nifi://*"));
|
||||
query.setMaxResults(100);
|
||||
|
||||
final QuerySubmission submission = repo.submitQuery(query);
|
||||
final QuerySubmission submission = repo.submitQuery(query, createUser());
|
||||
while (!submission.getResult().isFinished()) {
|
||||
Thread.sleep(100L);
|
||||
}
|
||||
|
@ -175,4 +176,27 @@ public class TestVolatileProvenanceRepository {
|
|||
};
|
||||
}
|
||||
|
||||
private NiFiUser createUser() {
|
||||
return new NiFiUser() {
|
||||
@Override
|
||||
public String getIdentity() {
|
||||
return "unit-test";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getUserName() {
|
||||
return "Unit Test";
|
||||
}
|
||||
|
||||
@Override
|
||||
public NiFiUser getChain() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAnonymous() {
|
||||
return false;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue