NIFI-9009: Created VerifiableProcessor, VerifiableControllerService, VerifiableReportingTask components; implemented backend work to call the methods. Added REST APIs and created/updated data models for component configuration verification

Signed-off-by: Joe Gresock <jgresock@gmail.com>

This closes #5288
This commit is contained in:
Mark Payne 2021-07-29 14:52:24 -04:00 committed by Joe Gresock
parent 4240a9f285
commit baf29e59f1
No known key found for this signature in database
GPG Key ID: 37F5B9B6E258C8B7
108 changed files with 5086 additions and 222 deletions

View File

@ -0,0 +1,83 @@
/*
* 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.components;
public class ConfigVerificationResult {
private final Outcome outcome;
private final String verificationStepName;
private final String explanation;
private ConfigVerificationResult(final Builder builder) {
outcome = builder.outcome;
verificationStepName = builder.verificationStepName;
explanation = builder.explanation;
}
public Outcome getOutcome() {
return outcome;
}
public String getVerificationStepName() {
return verificationStepName;
}
public String getExplanation() {
return explanation;
}
@Override
public String toString() {
return "ConfigVerificationResult[" +
"outcome=" + outcome +
", verificationStepName='" + verificationStepName + "'" +
", explanation='" + explanation + "']";
}
public static class Builder {
private Outcome outcome = Outcome.SKIPPED;
private String verificationStepName = "Unknown Step Name";
private String explanation;
public Builder outcome(final Outcome outcome) {
this.outcome = outcome;
return this;
}
public Builder verificationStepName(final String verificationStepName) {
this.verificationStepName = verificationStepName;
return this;
}
public Builder explanation(final String explanation) {
this.explanation = explanation;
return this;
}
public ConfigVerificationResult build() {
return new ConfigVerificationResult(this);
}
}
public enum Outcome {
SUCCESSFUL,
FAILED,
SKIPPED;
}
}

View File

@ -16,6 +16,7 @@
*/
package org.apache.nifi.components;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceLookup;
@ -132,4 +133,18 @@ public interface ValidationContext extends PropertyContext {
* @return <code>true</code> if all dependencies of the given property descriptor are satisfied, <code>false</code> otherwise
*/
boolean isDependencySatisfied(PropertyDescriptor propertyDescriptor, Function<String, PropertyDescriptor> propertyDescriptorLookup);
/**
* Determines whether or not incoming and outgoing connections should be validated.
* If <code>true</code>, then the validation should verify that all Relationships either have one or more connections that include the Relationship,
* or that the Relationship is auto-terminated.
* Additionally, if <code>true</code>, then any Processor with an {@link InputRequirement} of {@link InputRequirement.Requirement#INPUT_REQUIRED}
* should be invalid unless it has an incoming (non-looping) connection, and any Processor with an {@link InputRequirement} of {@link InputRequirement.Requirement#INPUT_FORBIDDEN}
* should be invalid if it does have any incoming connection.
*
* @return <code>true</code> if Connections should be validated, <code>false</code> if Connections should be ignored
*/
default boolean isValidateConnections() {
return true;
}
}

View File

@ -16,11 +16,12 @@
*/
package org.apache.nifi.controller;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.context.PropertyContext;
import java.util.Map;
import java.util.concurrent.TimeUnit;
/**
* This context is passed to ControllerServices and Reporting Tasks in order
* to expose their configuration to them.
@ -33,6 +34,11 @@ public interface ConfigurationContext extends PropertyContext {
*/
Map<PropertyDescriptor, String> getProperties();
/**
* @return the annotation data configured for the component
*/
String getAnnotationData();
/**
* @return a String representation of the scheduling period, or <code>null</code> if
* the component does not have a scheduling period (e.g., for ControllerServices)

View File

@ -0,0 +1,66 @@
/*
* 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.controller;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.components.ConfigVerificationResult;
import java.util.List;
import java.util.Map;
/**
* <p>
* Any Controller Service that implements this interface will be provided the opportunity to verify
* a given configuration of the Controller Service. This allows the Controller Service to provide meaningful feedback
* to users when configuring the dataflow.
* </p>
*
* <p>
* Generally speaking, verification differs from validation in that validation is expected to be very
* quick and run often. If a Controller Service is not valid, it cannot be started. However, verification may be
* more expensive or time-consuming to complete. For example, validation may ensure that a username is
* provided for connecting to an external service but should not perform any sort of network connection
* in order to verify that the username is accurate. Verification, on the other hand, may create resources
* such as network connections, may be more expensive to complete, and may be run only when a user invokes
* the action (though verification may later occur at other stages, such as when starting a component).
* </p>
*
* <p>
* Verification is allowed to be run only when a Controller Service is fully disabled.
* Therefore, any initialization logic that may need to be performed
* before the Controller Service is triggered may also be required for verification. However, the framework is not responsible
* for triggering the Lifecycle management stages, such as @OnScheduled before triggering the verification. Such
* methods should be handled by the {@link #verify(ConfigurationContext, ComponentLog, Map)} itself.
* The {@link #verify(ConfigurationContext, ComponentLog, Map)} method will only be called if the configuration is valid according to the
* validation rules (i.e., all Property Descriptors' validators and customValidate methods have indicated that the configuration is valid).
* </p>
*/
public interface VerifiableControllerService {
/**
* Verifies that the configuration defined by the given ConfigurationContext is valid.
* @param context the ProcessContext that contains the necessary configuration
* @param verificationLogger a logger that can be used during verification. While the typical logger can be used, doing so may result
* in producing bulletins, which can be confusing.
* @param variables a Map of key/value pairs that can be used to resolve variables referenced in property values via Expression Language
*
* @return a List of ConfigVerificationResults, each illustrating one step of the verification process that was completed
*/
List<ConfigVerificationResult> verify(ConfigurationContext context, ComponentLog verificationLogger, Map<String, String> variables);
}

View File

@ -0,0 +1,65 @@
/*
* 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.processor;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.logging.ComponentLog;
import java.util.List;
import java.util.Map;
/**
* <p>
* Any Processor that implements this interface will be provided the opportunity to verify
* a given configuration of the Processor. This allows the Processor to provide meaningful feedback
* to users when configuring the dataflow.
* </p>
*
* <p>
* Generally speaking, verification differs from validation in that validation is expected to be very
* quick and run often. If a Processor is not valid, it cannot be started. However, verification may be
* more expensive or time-consuming to complete. For example, validation may ensure that a username is
* provided for connecting to an external service but should not perform any sort of network connection
* in order to verify that the username is accurate. Verification, on the other hand, may create resources
* such as network connections, may be more expensive to complete, and may be run only when a user invokes
* the action (though verification may later occur at other stages, such as when starting a component).
* </p>
*
* <p>
* Verification is allowed to be run only when a Processor is fully stopped. I.e., it has no active threads
* and currently has a state of STOPPED. Therefore, any initialization logic that may need to be performed
* before the Processor is triggered may also be required for verification. However, the framework is not responsible
* for triggering the Lifecycle management stages, such as @OnScheduled before triggering the verification. Such
* methods should be handled by the {@link #verify(ProcessContext, ComponentLog, Map)} itself.
* The {@link #verify(ProcessContext, ComponentLog, Map)} method will only be called if the configuration is valid according to the
* validation rules (i.e., all Property Descriptors' validators and customValidate methods have indicated that the configuration is valid).
* </p>
*/
public interface VerifiableProcessor {
/**
* Verifies that the configuration defined by the given ProcessContext is valid.
* @param context the ProcessContext that contains the necessary configuration
* @param verificationLogger a logger that can be used during verification. While the typical logger can be used, doing so may result
* in producing bulletins, which can be confusing.
* @param attributes a mapping of values that can be used as FlowFile attributes for the purpose of evaluating Expression Language for resolving property values
* @return a List of ConfigVerificationResults, each illustrating one step of the verification process that was completed
*/
List<ConfigVerificationResult> verify(ProcessContext context, ComponentLog verificationLogger, Map<String, String> attributes);
}

View File

@ -0,0 +1,66 @@
/*
* 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.reporting;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.components.ConfigVerificationResult;
import java.util.List;
/**
* <p>
* Any Reporting Task that implements this interface will be provided the opportunity to verify
* a given configuration of the Reporting Task. This allows the Reporting Task to provide meaningful feedback
* to users when configuring the dataflow.
* </p>
*
* <p>
* Generally speaking, verification differs from validation in that validation is expected to be very
* quick and run often. If a Reporting Task is not valid, it cannot be started. However, verification may be
* more expensive or time-consuming to complete. For example, validation may ensure that a username is
* provided for connecting to an external service but should not perform any sort of network connection
* in order to verify that the username is accurate. Verification, on the other hand, may create resources
* such as network connections, may be more expensive to complete, and may be run only when a user invokes
* the action (though verification may later occur at other stages, such as when starting a component).
* </p>
*
* <p>
* Verification is allowed to be run only when a Reporting Task is fully stopped. I.e., it has no active threads
* and currently has a state of STOPPED. Therefore, any initialization logic that may need to be performed
* before the Reporting Task is triggered may also be required for verification. However, the framework is not responsible
* for triggering the Lifecycle management stages, such as @OnScheduled before triggering the verification. Such
* methods should be handled by the {@link #verify(ConfigurationContext, ComponentLog)} itself.
* The {@link #verify(ConfigurationContext, ComponentLog)} method will only be called if the configuration is valid according to the
* validation rules (i.e., all Property Descriptors' validators and customValidate methods have indicated that the configuration is valid).
* </p>
*/
public interface VerifiableReportingTask {
/**
* Verifies that the configuration defined by the given ConfigurationContext is valid.
*
* @param context the Configuration Context that contains the necessary configuration
* @param verificationLogger a logger that can be used during verification. While the typical logger can be used, doing so may result
* in producing bulletins, which can be confusing.
*
* @return a List of ConfigVerificationResults, each illustrating one step of the verification process that was completed
*/
List<ConfigVerificationResult> verify(ConfigurationContext context, ComponentLog verificationLogger);
}

View File

@ -16,10 +16,12 @@
*/
package org.apache.nifi.attribute.expression.language;
import org.apache.nifi.expression.AttributeValueDecorator;
import org.apache.nifi.processor.exception.ProcessException;
import java.util.Collections;
import java.util.Set;
public class EmptyPreparedQuery implements PreparedQuery {
private final String value;
@ -42,4 +44,9 @@ public class EmptyPreparedQuery implements PreparedQuery {
public VariableImpact getVariableImpact() {
return VariableImpact.NEVER_IMPACTED;
}
@Override
public Set<String> getExplicitlyReferencedAttributes() {
return Collections.emptySet();
}
}

View File

@ -21,6 +21,9 @@ import org.apache.nifi.attribute.expression.language.exception.AttributeExpressi
import org.apache.nifi.expression.AttributeValueDecorator;
import org.apache.nifi.processor.exception.ProcessException;
import java.util.Collections;
import java.util.Set;
/**
* An implementation of PreparedQuery that throws an
* {@link AttributeExpressionLanguageException} when attempting to evaluate the
@ -51,4 +54,9 @@ public class InvalidPreparedQuery implements PreparedQuery {
public VariableImpact getVariableImpact() {
return VariableImpact.NEVER_IMPACTED;
}
@Override
public Set<String> getExplicitlyReferencedAttributes() {
return Collections.emptySet();
}
}

View File

@ -16,10 +16,11 @@
*/
package org.apache.nifi.attribute.expression.language;
import org.apache.nifi.expression.AttributeValueDecorator;
import org.apache.nifi.processor.exception.ProcessException;
import java.util.Set;
public interface PreparedQuery {
String evaluateExpressions(EvaluationContext evaluationContext, AttributeValueDecorator decorator) throws ProcessException;
@ -34,4 +35,14 @@ public interface PreparedQuery {
* variable impacts this Expression.
*/
VariableImpact getVariableImpact();
/**
* Returns a Set of all attributes that are explicitly referenced by the Prepared Query.
* There are some expressions, however, such as <code>${allMatchingAttributes('a.*'):gt(4)}</code>
* that reference multiple attributes, but those attributes' names cannot be determined a priori. As a result,
* those attributes will not be included in the returned set.
*
* @return a Set of all attributes that are explicitly referenced by the Prepared Query
*/
Set<String> getExplicitlyReferencedAttributes();
}

View File

@ -19,7 +19,6 @@ package org.apache.nifi.attribute.expression.language;
import org.antlr.runtime.tree.Tree;
import org.apache.nifi.attribute.expression.language.compile.ExpressionCompiler;
import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
import org.apache.nifi.attribute.expression.language.evaluation.EvaluatorState;
import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
import org.apache.nifi.attribute.expression.language.evaluation.selection.AttributeEvaluator;
import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
@ -50,7 +49,6 @@ public class Query {
private final Tree tree;
private final Evaluator<?> evaluator;
private final AtomicBoolean evaluated = new AtomicBoolean(false);
private final EvaluatorState context = new EvaluatorState();
private Query(final String query, final Tree tree, final Evaluator<?> evaluator) {
this.query = query;

View File

@ -77,6 +77,53 @@ public class StandardPreparedQuery implements PreparedQuery {
return false;
}
@Override
public Set<String> getExplicitlyReferencedAttributes() {
final Set<String> attributes = new HashSet<>();
for (final Expression expression : expressions) {
if (!(expression instanceof CompiledExpression)) {
continue;
}
final CompiledExpression compiled = (CompiledExpression) expression;
for (final Evaluator<?> evaluator : compiled.getAllEvaluators()) {
if (evaluator instanceof AttributeEvaluator) {
final AttributeEvaluator attributeEval = (AttributeEvaluator) evaluator;
final Evaluator<String> nameEval = attributeEval.getNameEvaluator();
if (nameEval instanceof StringLiteralEvaluator) {
final String referencedVar = nameEval.evaluate(new StandardEvaluationContext(Collections.emptyMap())).getValue();
attributes.add(referencedVar);
}
} else if (evaluator instanceof AllAttributesEvaluator) {
final AllAttributesEvaluator allAttrsEval = (AllAttributesEvaluator) evaluator;
final MultiAttributeEvaluator iteratingEval = allAttrsEval.getVariableIteratingEvaluator();
if (iteratingEval instanceof MultiNamedAttributeEvaluator) {
attributes.addAll(((MultiNamedAttributeEvaluator) iteratingEval).getAttributeNames());
}
} else if (evaluator instanceof AnyAttributeEvaluator) {
final AnyAttributeEvaluator allAttrsEval = (AnyAttributeEvaluator) evaluator;
final MultiAttributeEvaluator iteratingEval = allAttrsEval.getVariableIteratingEvaluator();
if (iteratingEval instanceof MultiNamedAttributeEvaluator) {
attributes.addAll(((MultiNamedAttributeEvaluator) iteratingEval).getAttributeNames());
}
} else if (evaluator instanceof MappingEvaluator) {
final MappingEvaluator<?> allAttrsEval = (MappingEvaluator<?>) evaluator;
final MultiAttributeEvaluator iteratingEval = allAttrsEval.getVariableIteratingEvaluator();
if (iteratingEval instanceof MultiNamedAttributeEvaluator) {
attributes.addAll(((MultiNamedAttributeEvaluator) iteratingEval).getAttributeNames());
}
}
}
}
return attributes;
}
@Override
public VariableImpact getVariableImpact() {
final VariableImpact existing = this.variableImpact;

View File

@ -78,6 +78,11 @@ public class MockConfigurationContext implements ConfigurationContext {
return new HashMap<>(this.properties);
}
@Override
public String getAnnotationData() {
return null;
}
@Override
public Map<String, String> getAllProperties() {
final Map<String,String> propValueMap = new LinkedHashMap<>();

View File

@ -71,8 +71,7 @@ public abstract class AbstractAWSCredentialsProviderProcessor<ClientType extends
*/
protected void onScheduledUsingControllerService(ProcessContext context) {
this.client = createClient(context, getCredentialsProvider(context), createConfiguration(context));
super.initializeRegionAndEndpoint(context);
super.initializeRegionAndEndpoint(context, this.client);
}
@OnShutdown

View File

@ -35,6 +35,7 @@ import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
import org.apache.nifi.processor.ProcessContext;
@ -154,8 +155,8 @@ public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceCl
protected volatile ClientType client;
protected volatile Region region;
private static final String VPCE_ENDPOINT_SUFFIX = ".vpce.amazonaws.com";
private static final Pattern VPCE_ENDPOINT_PATTERN = Pattern.compile("^(?:.+[vpce-][a-z0-9-]+\\.)?([a-z0-9-]+)$");
protected static final String VPCE_ENDPOINT_SUFFIX = ".vpce.amazonaws.com";
protected static final Pattern VPCE_ENDPOINT_PATTERN = Pattern.compile("^(?:.+[vpce-][a-z0-9-]+\\.)?([a-z0-9-]+)$");
// If protocol is changed to be a property, ensure other uses are also changed
protected static final Protocol DEFAULT_PROTOCOL = Protocol.HTTPS;
@ -219,8 +220,12 @@ public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceCl
}
protected ClientConfiguration createConfiguration(final ProcessContext context) {
return createConfiguration(context, context.getMaxConcurrentTasks());
}
protected ClientConfiguration createConfiguration(final PropertyContext context, final int maxConcurrentTasks) {
final ClientConfiguration config = new ClientConfiguration();
config.setMaxConnections(context.getMaxConcurrentTasks());
config.setMaxConnections(maxConcurrentTasks);
config.setMaxErrorRetry(0);
config.setUserAgent(DEFAULT_USER_AGENT);
// If this is changed to be a property, ensure other uses are also changed
@ -272,7 +277,7 @@ public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceCl
@OnScheduled
public void onScheduled(final ProcessContext context) {
this.client = createClient(context, getCredentials(context), createConfiguration(context));
initializeRegionAndEndpoint(context);
initializeRegionAndEndpoint(context, this.client);
}
/*
@ -297,7 +302,7 @@ public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceCl
*/
public abstract void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException;
protected void initializeRegionAndEndpoint(ProcessContext context) {
protected void initializeRegionAndEndpoint(final ProcessContext context, final AmazonWebServiceClient client) {
// if the processor supports REGION, get the configured region.
if (getSupportedPropertyDescriptors().contains(REGION)) {
final String region = context.getProperty(REGION).getValue();
@ -324,11 +329,11 @@ public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceCl
// falling back to the configured region if the parse fails
// e.g. in case of https://vpce-***-***.sqs.{region}.vpce.amazonaws.com
String region = parseRegionForVPCE(urlstr, this.region.getName());
this.client.setEndpoint(urlstr, this.client.getServiceName(), region);
client.setEndpoint(urlstr, this.client.getServiceName(), region);
} else {
// handling non-vpce custom endpoints where the AWS library can parse the region out
// e.g. https://sqs.{region}.***.***.***.gov
this.client.setEndpoint(urlstr);
client.setEndpoint(urlstr);
}
}
}
@ -376,7 +381,7 @@ public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceCl
return region;
}
protected AWSCredentials getCredentials(final ProcessContext context) {
protected AWSCredentials getCredentials(final PropertyContext context) {
final String accessKey = context.getProperty(ACCESS_KEY).evaluateAttributeExpressions().getValue();
final String secretKey = context.getProperty(SECRET_KEY).evaluateAttributeExpressions().getValue();

View File

@ -16,23 +16,10 @@
*/
package org.apache.nifi.processors.aws.s3;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import com.amazonaws.auth.AWSStaticCredentialsProvider;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.AWSStaticCredentialsProvider;
import com.amazonaws.regions.Region;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.S3ClientOptions;
@ -43,6 +30,18 @@ import com.amazonaws.services.s3.model.EmailAddressGrantee;
import com.amazonaws.services.s3.model.Grantee;
import com.amazonaws.services.s3.model.Owner;
import com.amazonaws.services.s3.model.Permission;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
public abstract class AbstractS3Processor extends AbstractAWSCredentialsProviderProcessor<AmazonS3Client> {

View File

@ -17,6 +17,7 @@
package org.apache.nifi.processors.aws.s3;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.internal.Constants;
import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
import com.amazonaws.services.s3.model.GetObjectTaggingRequest;
@ -43,6 +44,8 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyDescriptor.Builder;
import org.apache.nifi.components.ValidationContext;
@ -57,6 +60,7 @@ import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.VerifiableProcessor;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.schema.access.SchemaNotFoundException;
import org.apache.nifi.serialization.RecordSetWriter;
@ -111,7 +115,7 @@ import java.util.concurrent.atomic.AtomicReference;
@WritesAttribute(attribute = "s3.user.metadata.___", description = "If 'Write User Metadata' is set to 'True', the user defined metadata associated to the S3 object that is being listed " +
"will be written as part of the flowfile attributes")})
@SeeAlso({FetchS3Object.class, PutS3Object.class, DeleteS3Object.class})
public class ListS3 extends AbstractS3Processor {
public class ListS3 extends AbstractS3Processor implements VerifiableProcessor {
public static final PropertyDescriptor DELIMITER = new Builder()
.name("delimiter")
@ -884,4 +888,49 @@ public class ListS3 extends AbstractS3Processor {
return keys;
}
}
@Override
public List<ConfigVerificationResult> verify(final ProcessContext context, final ComponentLog logger, final Map<String, String> attributes) {
final AmazonS3Client client = createClient(context, getCredentials(context), createConfiguration(context));
initializeRegionAndEndpoint(context, client);
final List<ConfigVerificationResult> results = new ArrayList<>();
final String bucketName = context.getProperty(BUCKET).evaluateAttributeExpressions().getValue();
if (bucketName == null || bucketName.trim().isEmpty()) {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Perform Listing")
.outcome(Outcome.FAILED)
.explanation("Bucket Name must be specified")
.build());
return results;
}
final String prefix = context.getProperty(PREFIX).getValue();
// Attempt to perform a listing of objects in the S3 bucket
try {
final ObjectListing listing = client.listObjects(bucketName, prefix);
final int count = listing.getObjectSummaries().size();
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Perform Listing")
.outcome(Outcome.SUCCESSFUL)
.explanation("Successfully listed contents of bucket '" + bucketName + "', finding " + count + " objects" + (prefix == null ? "" : " with a prefix of '" + prefix + "'"))
.build());
logger.info("Successfully verified configuration");
} catch (final Exception e) {
logger.warn("Failed to verify configuration. Could not list contents of bucket '{}'", bucketName, e);
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Perform Listing")
.outcome(Outcome.FAILED)
.explanation("Failed to list contents of bucket '" + bucketName + "': " + e.getMessage())
.build());
}
return results;
}
}

View File

@ -0,0 +1,53 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.web.api.dto;
import io.swagger.annotations.ApiModelProperty;
public class ConfigVerificationResultDTO {
private String outcome;
private String verificationStepName;
private String explanation;
@ApiModelProperty(value = "The outcome of the verification", allowableValues = "SUCCESSFUL, FAILED, SKIPPED")
public String getOutcome() {
return outcome;
}
public void setOutcome(final String outcome) {
this.outcome = outcome;
}
@ApiModelProperty("The name of the verification step")
public String getVerificationStepName() {
return verificationStepName;
}
public void setVerificationStepName(final String verificationStepName) {
this.verificationStepName = verificationStepName;
}
@ApiModelProperty("An explanation of why the step was or was not successful")
public String getExplanation() {
return explanation;
}
public void setExplanation(final String explanation) {
this.explanation = explanation;
}
}

View File

@ -17,6 +17,7 @@
package org.apache.nifi.web.api.dto;
import io.swagger.annotations.ApiModelProperty;
import io.swagger.annotations.ApiModelProperty.AccessMode;
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentEntity;
import javax.xml.bind.annotation.XmlType;
@ -45,6 +46,7 @@ public class ControllerServiceDTO extends ComponentDTO {
private Boolean deprecated;
private Boolean isExtensionMissing;
private Boolean multipleVersionsAvailable;
private Set<String> referencedAttributes;
private Map<String, String> properties;
private Map<String, PropertyDescriptorDTO> descriptors;
@ -313,6 +315,15 @@ public class ControllerServiceDTO extends ComponentDTO {
this.validationStatus = validationStatus;
}
@ApiModelProperty(value = "The set of FlowFile Attributes that are referenced via Expression Language by the configured properties", accessMode = AccessMode.READ_ONLY)
public Set<String> getReferencedAttributes() {
return referencedAttributes;
}
public void setReferencedAttributes(final Set<String> referencedAttributes) {
this.referencedAttributes = referencedAttributes;
}
@Override
public int hashCode() {
final String id = getId();

View File

@ -17,6 +17,7 @@
package org.apache.nifi.web.api.dto;
import io.swagger.annotations.ApiModelProperty;
import io.swagger.annotations.ApiModelProperty.AccessMode;
import javax.xml.bind.annotation.XmlType;
import java.util.Map;
@ -50,6 +51,7 @@ public class ProcessorConfigDTO {
private Map<String, String> defaultConcurrentTasks;
private Map<String, String> defaultSchedulingPeriod;
private Set<String> referencedAttributes;
public ProcessorConfigDTO() {
@ -308,4 +310,12 @@ public class ProcessorConfigDTO {
this.defaultSchedulingPeriod = defaultSchedulingPeriod;
}
@ApiModelProperty(value = "The set of FlowFile Attributes that are referenced via Expression Language by the configured properties", accessMode = AccessMode.READ_ONLY)
public Set<String> getReferencedAttributes() {
return referencedAttributes;
}
public void setReferencedAttributes(final Set<String> referencedAttributes) {
this.referencedAttributes = referencedAttributes;
}
}

View File

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

View File

@ -0,0 +1,69 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.web.api.dto;
import io.swagger.annotations.ApiModelProperty;
import io.swagger.annotations.ApiModelProperty.AccessMode;
import javax.xml.bind.annotation.XmlType;
import java.util.List;
import java.util.Map;
@XmlType(name = "verifyControllerServiceConfigRequest")
public class VerifyControllerServiceConfigRequestDTO extends AsynchronousRequestDTO<VerifyConfigUpdateStepDTO> {
private String controllerServiceId;
private ControllerServiceDTO controllerService;
private Map<String, String> attributes;
private List<ConfigVerificationResultDTO> results;
@ApiModelProperty("The ID of the Controller Service whose configuration was verified")
public String getControllerServiceId() {
return controllerServiceId;
}
public void setControllerServiceId(final String controllerServiceId) {
this.controllerServiceId = controllerServiceId;
}
@ApiModelProperty("The Controller Service")
public ControllerServiceDTO getControllerService() {
return controllerService;
}
public void setControllerService(final ControllerServiceDTO controllerService) {
this.controllerService = controllerService;
}
@ApiModelProperty("FlowFile Attributes that should be used to evaluate Expression Language for resolving property values")
public Map<String, String> getAttributes() {
return attributes;
}
public void setAttributes(final Map<String, String> attributes) {
this.attributes = attributes;
}
@ApiModelProperty(value="The Results of the verification", accessMode = AccessMode.READ_ONLY)
public List<ConfigVerificationResultDTO> getResults() {
return results;
}
public void setResults(final List<ConfigVerificationResultDTO> results) {
this.results = results;
}
}

View File

@ -0,0 +1,69 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.web.api.dto;
import io.swagger.annotations.ApiModelProperty;
import io.swagger.annotations.ApiModelProperty.AccessMode;
import javax.xml.bind.annotation.XmlType;
import java.util.List;
import java.util.Map;
@XmlType(name = "verifyProcessorConfigRequest")
public class VerifyProcessorConfigRequestDTO extends AsynchronousRequestDTO<VerifyConfigUpdateStepDTO> {
private String processorId;
private ProcessorConfigDTO processorConfigDTO;
private Map<String, String> attributes;
private List<ConfigVerificationResultDTO> results;
@ApiModelProperty("The ID of the Processor whose configuration was verified")
public String getProcessorId() {
return processorId;
}
public void setProcessorId(final String processorId) {
this.processorId = processorId;
}
@ApiModelProperty("The configuration for the Processor")
public ProcessorConfigDTO getProcessorConfig() {
return processorConfigDTO;
}
public void setProcessorConfig(final ProcessorConfigDTO processorConfigDTO) {
this.processorConfigDTO = processorConfigDTO;
}
@ApiModelProperty("FlowFile Attributes that should be used to evaluate Expression Language for resolving property values")
public Map<String, String> getAttributes() {
return attributes;
}
public void setAttributes(final Map<String, String> attributes) {
this.attributes = attributes;
}
@ApiModelProperty(value="The Results of the verification", accessMode = AccessMode.READ_ONLY)
public List<ConfigVerificationResultDTO> getResults() {
return results;
}
public void setResults(final List<ConfigVerificationResultDTO> results) {
this.results = results;
}
}

View File

@ -0,0 +1,58 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.web.api.dto;
import io.swagger.annotations.ApiModelProperty;
import io.swagger.annotations.ApiModelProperty.AccessMode;
import javax.xml.bind.annotation.XmlType;
import java.util.List;
@XmlType(name = "verifyReportingTaskConfigRequest")
public class VerifyReportingTaskConfigRequestDTO extends AsynchronousRequestDTO<VerifyConfigUpdateStepDTO> {
private String reportingTaskId;
private ReportingTaskDTO reportingTask;
private List<ConfigVerificationResultDTO> results;
@ApiModelProperty("The ID of the Controller Service whose configuration was verified")
public String getReportingTaskId() {
return reportingTaskId;
}
public void setReportingTaskId(final String reportingTaskId) {
this.reportingTaskId = reportingTaskId;
}
@ApiModelProperty("The Controller Service")
public ReportingTaskDTO getReportingTask() {
return reportingTask;
}
public void setReportingTask(final ReportingTaskDTO reportingTask) {
this.reportingTask = reportingTask;
}
@ApiModelProperty(value="The Results of the verification", accessMode = AccessMode.READ_ONLY)
public List<ConfigVerificationResultDTO> getResults() {
return results;
}
public void setResults(final List<ConfigVerificationResultDTO> results) {
this.results = results;
}
}

View File

@ -0,0 +1,37 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.web.api.entity;
import io.swagger.annotations.ApiModelProperty;
import org.apache.nifi.web.api.dto.VerifyControllerServiceConfigRequestDTO;
import javax.xml.bind.annotation.XmlRootElement;
@XmlRootElement(name="verifyControllerServiceConfigRequest")
public class VerifyControllerServiceConfigRequestEntity extends Entity {
private VerifyControllerServiceConfigRequestDTO request;
@ApiModelProperty("The request")
public VerifyControllerServiceConfigRequestDTO getRequest() {
return request;
}
public void setRequest(final VerifyControllerServiceConfigRequestDTO request) {
this.request = request;
}
}

View File

@ -0,0 +1,37 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.web.api.entity;
import io.swagger.annotations.ApiModelProperty;
import org.apache.nifi.web.api.dto.VerifyProcessorConfigRequestDTO;
import javax.xml.bind.annotation.XmlRootElement;
@XmlRootElement(name="verifyProcessorConfigRequest")
public class VerifyProcessorConfigRequestEntity extends Entity {
private VerifyProcessorConfigRequestDTO request;
@ApiModelProperty("The request")
public VerifyProcessorConfigRequestDTO getRequest() {
return request;
}
public void setRequest(final VerifyProcessorConfigRequestDTO request) {
this.request = request;
}
}

View File

@ -0,0 +1,37 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.web.api.entity;
import io.swagger.annotations.ApiModelProperty;
import org.apache.nifi.web.api.dto.VerifyReportingTaskConfigRequestDTO;
import javax.xml.bind.annotation.XmlRootElement;
@XmlRootElement(name="verifyReportingTaskConfigRequest")
public class VerifyReportingTaskConfigRequestEntity extends Entity {
private VerifyReportingTaskConfigRequestDTO request;
@ApiModelProperty("The request")
public VerifyReportingTaskConfigRequestDTO getRequest() {
return request;
}
public void setRequest(final VerifyReportingTaskConfigRequestDTO request) {
this.request = request;
}
}

View File

@ -77,6 +77,9 @@ import org.apache.nifi.cluster.coordination.http.endpoints.UserGroupEndpointMerg
import org.apache.nifi.cluster.coordination.http.endpoints.UserGroupsEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.UsersEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.VariableRegistryEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.VerifyControllerServiceConfigEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.VerifyProcessorConfigEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.VerifyReportingTaskConfigEndpointMerger;
import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.stream.io.NullOutputStream;
@ -169,6 +172,9 @@ public class StandardHttpResponseMapper implements HttpResponseMapper {
endpointMergers.add(new ParameterContextsEndpointMerger());
endpointMergers.add(new ParameterContextEndpointMerger());
endpointMergers.add(new ParameterContextUpdateEndpointMerger());
endpointMergers.add(new VerifyProcessorConfigEndpointMerger());
endpointMergers.add(new VerifyControllerServiceConfigEndpointMerger());
endpointMergers.add(new VerifyReportingTaskConfigEndpointMerger());
}
@Override

View File

@ -0,0 +1,120 @@
/*
* 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.cluster.coordination.http.endpoints;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class ConfigVerificationResultMerger {
private final Map<String, List<ConfigVerificationResultDTO>> verificationResultDtos = new HashMap<>();
/**
* Adds the config verification results for one of the nodes in the cluster
* @param nodeId the ID of the node in the cluster
* @param nodeResults the results for the config verification
*/
public void addNodeResults(final NodeIdentifier nodeId, final List<ConfigVerificationResultDTO> nodeResults) {
if (nodeResults == null || nodeResults.isEmpty()) {
return;
}
verificationResultDtos.put(nodeId.getApiAddress() + ":" + nodeId.getApiPort(), nodeResults);
}
/**
* Computes the aggregate list of ConfigVerificationResultDTO based on all of the results added using the {link {@link #addNodeResults(NodeIdentifier, List)}} method
* @return the aggregate results of the config verification results from all nodes
*/
public List<ConfigVerificationResultDTO> computeAggregateResults() {
// For each node, build up a mapping of Step Name -> Results
final Map<String, List<ConfigVerificationResultDTO>> resultsByStepName = new HashMap<>();
for (final Map.Entry<String, List<ConfigVerificationResultDTO>> entry : verificationResultDtos.entrySet()) {
final String nodeId = entry.getKey();
final List<ConfigVerificationResultDTO> nodeResults = entry.getValue();
// If the result hasn't been set, the task is not yet complete, so we don't have to bother merging the results.
if (nodeResults == null) {
return null;
}
for (final ConfigVerificationResultDTO result : nodeResults) {
final String stepName = result.getVerificationStepName();
final List<ConfigVerificationResultDTO> resultList = resultsByStepName.computeIfAbsent(stepName, key -> new ArrayList<>());
// If skipped or unsuccessful, add the node's address to the explanation
if (!Outcome.SUCCESSFUL.name().equals(result.getOutcome())) {
result.setExplanation(nodeId + " - " + result.getExplanation());
}
resultList.add(result);
}
}
// Merge together all results for each step name
final List<ConfigVerificationResultDTO> aggregateResults = new ArrayList<>();
for (final Map.Entry<String, List<ConfigVerificationResultDTO>> entry : resultsByStepName.entrySet()) {
final String stepName = entry.getKey();
final List<ConfigVerificationResultDTO> resultList = entry.getValue();
final ConfigVerificationResultDTO firstResult = resultList.get(0); // This is safe because the list won't be added to the map unless it has at least 1 element.
String outcome = firstResult.getOutcome();
String explanation = firstResult.getExplanation();
for (final ConfigVerificationResultDTO result : resultList) {
// If any node indicates failure, the outcome is failure.
// Otherwise, if any node indicates that a step was skipped, the outcome is skipped.
// Otherwise, all nodes have reported the outcome is successful, so the outcome is successful.
if (Outcome.FAILED.name().equals(result.getOutcome())) {
outcome = result.getOutcome();
explanation = result.getExplanation();
} else if (Outcome.SKIPPED.name().equals(result.getOutcome()) && Outcome.SUCCESSFUL.name().equals(outcome)) {
outcome = result.getOutcome();
explanation = result.getExplanation();
}
}
final ConfigVerificationResultDTO resultDto = new ConfigVerificationResultDTO();
resultDto.setVerificationStepName(stepName);
resultDto.setOutcome(outcome);
resultDto.setExplanation(explanation);
aggregateResults.add(resultDto);
}
// Determine the ordering of the original steps.
final Map<String, Integer> stepOrders = new HashMap<>();
for (final List<ConfigVerificationResultDTO> resultDtos : verificationResultDtos.values()) {
for (final ConfigVerificationResultDTO resultDto : resultDtos) {
final String stepName = resultDto.getVerificationStepName();
stepOrders.putIfAbsent(stepName, stepOrders.size());
}
}
// Sort the results by ordering them based on the order of the original steps. This will retain the original ordering of the steps.
aggregateResults.sort(Comparator.comparing(dto -> stepOrders.get(dto.getVerificationStepName())));
return aggregateResults;
}
}

View File

@ -0,0 +1,73 @@
/*
* 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.cluster.coordination.http.endpoints;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.dto.VerifyControllerServiceConfigRequestDTO;
import org.apache.nifi.web.api.entity.VerifyControllerServiceConfigRequestEntity;
import java.net.URI;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
public class VerifyControllerServiceConfigEndpointMerger extends AbstractSingleEntityEndpoint<VerifyControllerServiceConfigRequestEntity> {
public static final Pattern VERIFY_PROCESSOR_CONFIG_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/[a-f0-9\\-]{36}/config/verification-requests(/[a-f0-9\\-]{36})?");
@Override
protected Class<VerifyControllerServiceConfigRequestEntity> getEntityClass() {
return VerifyControllerServiceConfigRequestEntity.class;
}
@Override
public boolean canHandle(final URI uri, final String method) {
return VERIFY_PROCESSOR_CONFIG_URI_PATTERN.matcher(uri.getPath()).matches();
}
@Override
protected void mergeResponses(final VerifyControllerServiceConfigRequestEntity clientEntity, final Map<NodeIdentifier, VerifyControllerServiceConfigRequestEntity> entityMap,
final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses) {
final VerifyControllerServiceConfigRequestDTO requestDto = clientEntity.getRequest();
final List<ConfigVerificationResultDTO> results = requestDto.getResults();
// If the result hasn't been set, the task is not yet complete, so we don't have to bother merging the results.
if (results == null) {
return;
}
// Aggregate the Config Verification Results across all nodes into a single List
final ConfigVerificationResultMerger resultMerger = new ConfigVerificationResultMerger();
for (final Map.Entry<NodeIdentifier, VerifyControllerServiceConfigRequestEntity> entry : entityMap.entrySet()) {
final NodeIdentifier nodeId = entry.getKey();
final VerifyControllerServiceConfigRequestEntity entity = entry.getValue();
final List<ConfigVerificationResultDTO> nodeResults = entity.getRequest().getResults();
resultMerger.addNodeResults(nodeId, nodeResults);
}
final List<ConfigVerificationResultDTO> aggregateResults = resultMerger.computeAggregateResults();
clientEntity.getRequest().setResults(aggregateResults);
}
}

View File

@ -0,0 +1,72 @@
/*
* 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.cluster.coordination.http.endpoints;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.dto.VerifyProcessorConfigRequestDTO;
import org.apache.nifi.web.api.entity.VerifyProcessorConfigRequestEntity;
import java.net.URI;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
public class VerifyProcessorConfigEndpointMerger extends AbstractSingleEntityEndpoint<VerifyProcessorConfigRequestEntity> {
public static final Pattern VERIFY_PROCESSOR_CONFIG_URI_PATTERN = Pattern.compile("/nifi-api/processors/[a-f0-9\\-]{36}/config/verification-requests(/[a-f0-9\\-]{36})?");
@Override
protected Class<VerifyProcessorConfigRequestEntity> getEntityClass() {
return VerifyProcessorConfigRequestEntity.class;
}
@Override
public boolean canHandle(final URI uri, final String method) {
return VERIFY_PROCESSOR_CONFIG_URI_PATTERN.matcher(uri.getPath()).matches();
}
@Override
protected void mergeResponses(final VerifyProcessorConfigRequestEntity clientEntity, final Map<NodeIdentifier, VerifyProcessorConfigRequestEntity> entityMap,
final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses) {
final VerifyProcessorConfigRequestDTO requestDto = clientEntity.getRequest();
final List<ConfigVerificationResultDTO> results = requestDto.getResults();
// If the result hasn't been set, the task is not yet complete, so we don't have to bother merging the results.
if (results == null) {
return;
}
// Aggregate the Config Verification Results across all nodes into a single List
final ConfigVerificationResultMerger resultMerger = new ConfigVerificationResultMerger();
for (final Map.Entry<NodeIdentifier, VerifyProcessorConfigRequestEntity> entry : entityMap.entrySet()) {
final NodeIdentifier nodeId = entry.getKey();
final VerifyProcessorConfigRequestEntity entity = entry.getValue();
final List<ConfigVerificationResultDTO> nodeResults = entity.getRequest().getResults();
resultMerger.addNodeResults(nodeId, nodeResults);
}
final List<ConfigVerificationResultDTO> aggregateResults = resultMerger.computeAggregateResults();
clientEntity.getRequest().setResults(aggregateResults);
}
}

View File

@ -0,0 +1,71 @@
/*
* 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.cluster.coordination.http.endpoints;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.dto.VerifyReportingTaskConfigRequestDTO;
import org.apache.nifi.web.api.entity.VerifyReportingTaskConfigRequestEntity;
import java.net.URI;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
public class VerifyReportingTaskConfigEndpointMerger extends AbstractSingleEntityEndpoint<VerifyReportingTaskConfigRequestEntity> {
public static final Pattern VERIFY_REPORTING_TASK_CONFIG_URI_PATTERN = Pattern.compile("/nifi-api/reporting-tasks/[a-f0-9\\-]{36}/config/verification-requests(/[a-f0-9\\-]{36})?");
@Override
protected Class<VerifyReportingTaskConfigRequestEntity> getEntityClass() {
return VerifyReportingTaskConfigRequestEntity.class;
}
@Override
public boolean canHandle(final URI uri, final String method) {
return VERIFY_REPORTING_TASK_CONFIG_URI_PATTERN.matcher(uri.getPath()).matches();
}
@Override
protected void mergeResponses(final VerifyReportingTaskConfigRequestEntity clientEntity, final Map<NodeIdentifier, VerifyReportingTaskConfigRequestEntity> entityMap,
final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses) {
final VerifyReportingTaskConfigRequestDTO requestDto = clientEntity.getRequest();
final List<ConfigVerificationResultDTO> results = requestDto.getResults();
// If the result hasn't been set, the task is not yet complete, so we don't have to bother merging the results.
if (results == null) {
return;
}
// Aggregate the Config Verification Results across all nodes into a single List
final ConfigVerificationResultMerger resultMerger = new ConfigVerificationResultMerger();
for (final Map.Entry<NodeIdentifier, VerifyReportingTaskConfigRequestEntity> entry : entityMap.entrySet()) {
final NodeIdentifier nodeId = entry.getKey();
final VerifyReportingTaskConfigRequestEntity entity = entry.getValue();
final List<ConfigVerificationResultDTO> nodeResults = entity.getRequest().getResults();
resultMerger.addNodeResults(nodeId, nodeResults);
}
final List<ConfigVerificationResultDTO> aggregateResults = resultMerger.computeAggregateResults();
clientEntity.getRequest().setResults(aggregateResults);
}
}

View File

@ -33,7 +33,10 @@ import org.apache.nifi.authorization.Resource;
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.bundle.Bundle;
import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
@ -55,6 +58,7 @@ import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.logging.LogLevel;
import org.apache.nifi.logging.LogRepositoryFactory;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.InstanceClassLoader;
import org.apache.nifi.nar.NarCloseable;
import org.apache.nifi.parameter.ParameterContext;
import org.apache.nifi.parameter.ParameterLookup;
@ -63,6 +67,7 @@ import org.apache.nifi.processor.ProcessSessionFactory;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.SimpleProcessLogger;
import org.apache.nifi.processor.VerifiableProcessor;
import org.apache.nifi.registry.ComponentVariableRegistry;
import org.apache.nifi.scheduling.ExecutionNode;
import org.apache.nifi.scheduling.SchedulingStrategy;
@ -1062,6 +1067,71 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
return nonLoopConnections;
}
@Override
public List<ConfigVerificationResult> verifyConfiguration(final ProcessContext context, final ComponentLog logger, final Map<String, String> attributes, final ExtensionManager extensionManager) {
final List<ConfigVerificationResult> results = new ArrayList<>();
try {
verifyCanPerformVerification();
final long startNanos = System.nanoTime();
// Call super's verifyConfig, which will perform component validation
results.addAll(super.verifyConfig(context.getProperties(), context.getAnnotationData(), getProcessGroup().getParameterContext()));
final long validationComplete = System.nanoTime();
// If any invalid outcomes from validation, we do not want to perform additional verification, because we only run additional verification when the component is valid.
// This is done in order to make it much simpler to develop these verifications, since the developer doesn't have to worry about whether or not the given values are valid.
if (!results.isEmpty() && results.stream().anyMatch(result -> result.getOutcome() == Outcome.FAILED)) {
return results;
}
final Processor processor = getProcessor();
if (processor instanceof VerifiableProcessor) {
LOG.debug("{} is a VerifiableProcessor. Will perform full verification of configuration.", this);
final VerifiableProcessor verifiable = (VerifiableProcessor) getProcessor();
// Check if the given configuration requires a different classloader than the current configuration
final boolean classpathDifferent = isClasspathDifferent(context.getProperties());
if (classpathDifferent) {
// Create a classloader for the given configuration and use that to verify the component's configuration
final Bundle bundle = extensionManager.getBundle(getBundleCoordinate());
final Set<URL> classpathUrls = getAdditionalClasspathResources(context.getProperties().keySet(), descriptor -> context.getProperty(descriptor).getValue());
final ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader();
try (final InstanceClassLoader detectedClassLoader = extensionManager.createInstanceClassLoader(getComponentType(), getIdentifier(), bundle, classpathUrls, false)) {
Thread.currentThread().setContextClassLoader(detectedClassLoader);
results.addAll(verifiable.verify(context, logger, attributes));
} finally {
Thread.currentThread().setContextClassLoader(currentClassLoader);
}
} else {
// Verify the configuration, using the component's classloader
try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, processor.getClass(), getIdentifier())) {
results.addAll(verifiable.verify(context, logger, attributes));
}
}
final long validationNanos = validationComplete - startNanos;
final long verificationNanos = System.nanoTime() - validationComplete;
LOG.debug("{} completed full configuration validation in {} plus {} for validation",
this, FormatUtils.formatNanos(verificationNanos, false), FormatUtils.formatNanos(validationNanos, false));
} else {
LOG.debug("{} is not a VerifiableProcessor, so will not perform full verification of configuration. Validation took {}", this,
FormatUtils.formatNanos(validationComplete - startNanos, false));
}
} catch (final Throwable t) {
LOG.error("Failed to perform verification of processor's configuration for {}", this, t);
results.add(new ConfigVerificationResult.Builder()
.outcome(Outcome.FAILED)
.verificationStepName("Perform Verification")
.explanation("Encountered unexpected failure when attempting to perform verification: " + t)
.build());
}
return results;
}
@Override
public Collection<ValidationResult> getValidationErrors() {
@ -1080,36 +1150,38 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
.forEach(results::add);
// Ensure that any relationships that don't have a connection defined are auto-terminated
for (final Relationship relationship : getUndefinedRelationships()) {
if (!isAutoTerminated(relationship)) {
final ValidationResult error = new ValidationResult.Builder()
.explanation("Relationship '" + relationship.getName()
+ "' is not connected to any component and is not auto-terminated")
.subject("Relationship " + relationship.getName()).valid(false).build();
results.add(error);
if (validationContext.isValidateConnections()) {
for (final Relationship relationship : getUndefinedRelationships()) {
if (!isAutoTerminated(relationship)) {
final ValidationResult error = new ValidationResult.Builder()
.explanation("Relationship '" + relationship.getName()
+ "' is not connected to any component and is not auto-terminated")
.subject("Relationship " + relationship.getName()).valid(false).build();
results.add(error);
}
}
}
// Ensure that the requirements of the InputRequirement are met.
switch (getInputRequirement()) {
case INPUT_ALLOWED:
break;
case INPUT_FORBIDDEN: {
final int incomingConnCount = getIncomingNonLoopConnections().size();
if (incomingConnCount != 0) {
results.add(new ValidationResult.Builder().explanation(
"Processor does not allow upstream connections but currently has " + incomingConnCount)
.subject("Upstream Connections").valid(false).build());
// Ensure that the requirements of the InputRequirement are met.
switch (getInputRequirement()) {
case INPUT_ALLOWED:
break;
case INPUT_FORBIDDEN: {
final int incomingConnCount = getIncomingNonLoopConnections().size();
if (incomingConnCount != 0) {
results.add(new ValidationResult.Builder().explanation(
"Processor does not allow upstream connections but currently has " + incomingConnCount)
.subject("Upstream Connections").valid(false).build());
}
break;
}
break;
}
case INPUT_REQUIRED: {
if (getIncomingNonLoopConnections().isEmpty()) {
results.add(new ValidationResult.Builder()
.explanation("Processor requires an upstream connection but currently has none")
.subject("Upstream Connections").valid(false).build());
case INPUT_REQUIRED: {
if (getIncomingNonLoopConnections().isEmpty()) {
results.add(new ValidationResult.Builder()
.explanation("Processor requires an upstream connection but currently has none")
.subject("Upstream Connections").valid(false).build());
}
break;
}
break;
}
}

View File

@ -17,7 +17,10 @@
package org.apache.nifi.controller.reporting;
import org.apache.nifi.annotation.configuration.DefaultSchedule;
import org.apache.nifi.bundle.Bundle;
import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.validation.ValidationStatus;
@ -35,10 +38,14 @@ import org.apache.nifi.controller.ValidationContextFactory;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.controller.service.StandardConfigurationContext;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.InstanceClassLoader;
import org.apache.nifi.nar.NarCloseable;
import org.apache.nifi.parameter.ParameterLookup;
import org.apache.nifi.registry.ComponentVariableRegistry;
import org.apache.nifi.reporting.ReportingTask;
import org.apache.nifi.reporting.VerifiableReportingTask;
import org.apache.nifi.scheduling.SchedulingStrategy;
import org.apache.nifi.util.CharacterFilterUtils;
import org.apache.nifi.util.FormatUtils;
@ -47,7 +54,9 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.net.URL;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
@ -315,4 +324,77 @@ public abstract class AbstractReportingTaskNode extends AbstractComponentNode im
public ParameterLookup getParameterLookup() {
return ParameterLookup.EMPTY;
}
@Override
public void verifyCanPerformVerification() {
if (isRunning()) {
throw new IllegalStateException("Cannot perform verification because Reporting Task is not fully stopped");
}
}
@Override
public List<ConfigVerificationResult> verifyConfiguration(final ConfigurationContext context, final ComponentLog logger, final ExtensionManager extensionManager) {
final List<ConfigVerificationResult> results = new ArrayList<>();
try {
verifyCanPerformVerification();
final long startNanos = System.nanoTime();
// Call super's verifyConfig, which will perform component validation
results.addAll(super.verifyConfig(context.getProperties(), context.getAnnotationData(), null));
final long validationComplete = System.nanoTime();
// If any invalid outcomes from validation, we do not want to perform additional verification, because we only run additional verification when the component is valid.
// This is done in order to make it much simpler to develop these verifications, since the developer doesn't have to worry about whether or not the given values are valid.
if (!results.isEmpty() && results.stream().anyMatch(result -> result.getOutcome() == Outcome.FAILED)) {
return results;
}
final ReportingTask reportingTask = getReportingTask();
if (reportingTask instanceof VerifiableReportingTask) {
logger.debug("{} is a VerifiableReportingTask. Will perform full verification of configuration.", this);
final VerifiableReportingTask verifiable = (VerifiableReportingTask) reportingTask;
// Check if the given configuration requires a different classloader than the current configuration
final boolean classpathDifferent = isClasspathDifferent(context.getProperties());
if (classpathDifferent) {
// Create a classloader for the given configuration and use that to verify the component's configuration
final Bundle bundle = extensionManager.getBundle(getBundleCoordinate());
final Set<URL> classpathUrls = getAdditionalClasspathResources(context.getProperties().keySet(), descriptor -> context.getProperty(descriptor).getValue());
final ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader();
try (final InstanceClassLoader detectedClassLoader = extensionManager.createInstanceClassLoader(getComponentType(), getIdentifier(), bundle, classpathUrls, false)) {
Thread.currentThread().setContextClassLoader(detectedClassLoader);
results.addAll(verifiable.verify(context, logger));
} finally {
Thread.currentThread().setContextClassLoader(currentClassLoader);
}
} else {
// Verify the configuration, using the component's classloader
try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, reportingTask.getClass(), getIdentifier())) {
results.addAll(verifiable.verify(context, logger));
}
}
final long validationNanos = validationComplete - startNanos;
final long verificationNanos = System.nanoTime() - validationComplete;
logger.debug("{} completed full configuration validation in {} plus {} for validation",
this, FormatUtils.formatNanos(verificationNanos, false), FormatUtils.formatNanos(validationNanos, false));
} else {
logger.debug("{} is not a VerifiableReportingTask, so will not perform full verification of configuration. Validation took {}", this,
FormatUtils.formatNanos(validationComplete - startNanos, false));
}
} catch (final Throwable t) {
logger.error("Failed to perform verification of Reporting Task's configuration for {}", this, t);
results.add(new ConfigVerificationResult.Builder()
.outcome(Outcome.FAILED)
.verificationStepName("Perform Verification")
.explanation("Encountered unexpected failure when attempting to perform verification: " + t)
.build());
}
return results;
}
}

View File

@ -27,9 +27,13 @@ import org.apache.nifi.components.resource.StandardResourceReferenceFactory;
import org.apache.nifi.controller.ComponentNode;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.controller.PropertyConfiguration;
import org.apache.nifi.controller.PropertyConfigurationMapper;
import org.apache.nifi.parameter.ParameterLookup;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.util.FormatUtils;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.Map;
@ -43,13 +47,27 @@ public class StandardConfigurationContext implements ConfigurationContext {
private final VariableRegistry variableRegistry;
private final String schedulingPeriod;
private final Long schedulingNanos;
private final Map<PropertyDescriptor, String> properties;
private final String annotationData;
public StandardConfigurationContext(final ComponentNode component, final ControllerServiceLookup serviceLookup, final String schedulingPeriod,
final VariableRegistry variableRegistry) {
this(component, serviceLookup, schedulingPeriod, variableRegistry, component.getEffectivePropertyValues(), component.getAnnotationData());
}
public StandardConfigurationContext(final ComponentNode component, final Map<String, String> propertyOverrides, final String annotationDataOverride, final ParameterLookup parameterLookup,
final ControllerServiceLookup serviceLookup, final String schedulingPeriod, final VariableRegistry variableRegistry) {
this(component, serviceLookup, schedulingPeriod, variableRegistry, resolvePropertyValues(component, parameterLookup, propertyOverrides), annotationDataOverride);
}
public StandardConfigurationContext(final ComponentNode component, final ControllerServiceLookup serviceLookup, final String schedulingPeriod,
final VariableRegistry variableRegistry, final Map<PropertyDescriptor, String> propertyValues, final String annotationData) {
this.component = component;
this.serviceLookup = serviceLookup;
this.schedulingPeriod = schedulingPeriod;
this.variableRegistry = variableRegistry;
this.properties = Collections.unmodifiableMap(propertyValues);
this.annotationData = annotationData;
if (schedulingPeriod == null) {
schedulingNanos = null;
@ -62,7 +80,7 @@ public class StandardConfigurationContext implements ConfigurationContext {
}
preparedQueries = new HashMap<>();
for (final Map.Entry<PropertyDescriptor, String> entry : component.getEffectivePropertyValues().entrySet()) {
for (final Map.Entry<PropertyDescriptor, String> entry : propertyValues.entrySet()) {
final PropertyDescriptor desc = entry.getKey();
String value = entry.getValue();
if (value == null) {
@ -74,9 +92,29 @@ public class StandardConfigurationContext implements ConfigurationContext {
}
}
private static Map<PropertyDescriptor, String> resolvePropertyValues(final ComponentNode component, final ParameterLookup parameterLookup, final Map<String, String> propertyOverrides) {
final Map<PropertyDescriptor, String> resolvedProperties = new LinkedHashMap<>(component.getEffectivePropertyValues());
final PropertyConfigurationMapper configurationMapper = new PropertyConfigurationMapper();
for (final Map.Entry<String, String> entry : propertyOverrides.entrySet()) {
final String propertyName = entry.getKey();
final String propertyValue = entry.getValue();
final PropertyDescriptor propertyDescriptor = component.getPropertyDescriptor(propertyName);
if (propertyValue == null) {
resolvedProperties.remove(propertyDescriptor);
} else {
final PropertyConfiguration configuration = configurationMapper.mapRawPropertyValuesToPropertyConfiguration(propertyDescriptor, propertyValue);
final String effectiveValue = configuration.getEffectiveValue(parameterLookup);
resolvedProperties.put(propertyDescriptor, effectiveValue);
}
}
return resolvedProperties;
}
@Override
public PropertyValue getProperty(final PropertyDescriptor property) {
final String configuredValue = component.getEffectivePropertyValue(property);
final String configuredValue = properties.get(property);
// We need to get the 'canonical representation' of the property descriptor from the component itself,
// since the supplied PropertyDescriptor may not have the proper default value.
@ -89,7 +127,12 @@ public class StandardConfigurationContext implements ConfigurationContext {
@Override
public Map<PropertyDescriptor, String> getProperties() {
return component.getEffectivePropertyValues();
return properties;
}
@Override
public String getAnnotationData() {
return annotationData;
}
@Override

View File

@ -25,7 +25,10 @@ import org.apache.nifi.authorization.Resource;
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.bundle.Bundle;
import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.validation.ValidationState;
@ -39,16 +42,19 @@ import org.apache.nifi.controller.LoggableComponent;
import org.apache.nifi.controller.ReloadComponent;
import org.apache.nifi.controller.TerminationAwareLogger;
import org.apache.nifi.controller.ValidationContextFactory;
import org.apache.nifi.controller.VerifiableControllerService;
import org.apache.nifi.controller.exception.ControllerServiceInstantiationException;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.InstanceClassLoader;
import org.apache.nifi.nar.NarCloseable;
import org.apache.nifi.parameter.ParameterContext;
import org.apache.nifi.parameter.ParameterLookup;
import org.apache.nifi.processor.SimpleProcessLogger;
import org.apache.nifi.registry.ComponentVariableRegistry;
import org.apache.nifi.util.CharacterFilterUtils;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.ReflectionUtils;
import org.apache.nifi.util.Tuple;
import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
@ -61,6 +67,7 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Optional;
import java.util.Set;
@ -382,6 +389,82 @@ public class StandardControllerServiceNode extends AbstractComponentNode impleme
return enabled;
}
@Override
public void verifyCanPerformVerification() {
if (getState() != ControllerServiceState.DISABLED) {
throw new IllegalStateException("Cannot perform verification because the Controller Service is not disabled");
}
}
@Override
public List<ConfigVerificationResult> verifyConfiguration(final ConfigurationContext context, final ComponentLog logger, final Map<String, String> variables,
final ExtensionManager extensionManager) {
final List<ConfigVerificationResult> results = new ArrayList<>();
try {
verifyCanPerformVerification();
final long startNanos = System.nanoTime();
// Call super's verifyConfig, which will perform component validation
results.addAll(super.verifyConfig(context.getProperties(), context.getAnnotationData(), getProcessGroup() == null ? null : getProcessGroup().getParameterContext()));
final long validationComplete = System.nanoTime();
// If any invalid outcomes from validation, we do not want to perform additional verification, because we only run additional verification when the component is valid.
// This is done in order to make it much simpler to develop these verifications, since the developer doesn't have to worry about whether or not the given values are valid.
if (!results.isEmpty() && results.stream().anyMatch(result -> result.getOutcome() == Outcome.FAILED)) {
return results;
}
final ControllerService controllerService = getControllerServiceImplementation();
if (controllerService instanceof VerifiableControllerService) {
LOG.debug("{} is a VerifiableControllerService. Will perform full verification of configuration.", this);
final VerifiableControllerService verifiable = (VerifiableControllerService) controllerService;
// Check if the given configuration requires a different classloader than the current configuration
final boolean classpathDifferent = isClasspathDifferent(context.getProperties());
if (classpathDifferent) {
// Create a classloader for the given configuration and use that to verify the component's configuration
final Bundle bundle = extensionManager.getBundle(getBundleCoordinate());
final Set<URL> classpathUrls = getAdditionalClasspathResources(context.getProperties().keySet(), descriptor -> context.getProperty(descriptor).getValue());
final ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader();
try (final InstanceClassLoader detectedClassLoader = extensionManager.createInstanceClassLoader(getComponentType(), getIdentifier(), bundle, classpathUrls, false)) {
Thread.currentThread().setContextClassLoader(detectedClassLoader);
results.addAll(verifiable.verify(context, logger, variables));
} finally {
Thread.currentThread().setContextClassLoader(currentClassLoader);
}
} else {
// Verify the configuration, using the component's classloader
try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, controllerService.getClass(), getIdentifier())) {
results.addAll(verifiable.verify(context, logger, variables));
}
}
final long validationNanos = validationComplete - startNanos;
final long verificationNanos = System.nanoTime() - validationComplete;
LOG.debug("{} completed full configuration validation in {} plus {} for validation",
this, FormatUtils.formatNanos(verificationNanos, false), FormatUtils.formatNanos(validationNanos, false));
} else {
LOG.debug("{} is not a VerifiableControllerService, so will not perform full verification of configuration. Validation took {}", this,
FormatUtils.formatNanos(validationComplete - startNanos, false));
}
} catch (final Throwable t) {
LOG.error("Failed to perform verification of Controller Service's configuration for {}", this, t);
results.add(new ConfigVerificationResult.Builder()
.outcome(Outcome.FAILED)
.verificationStepName("Perform Verification")
.explanation("Encountered unexpected failure when attempting to perform verification: " + t)
.build());
}
return results;
}
@Override
public boolean isValidationNecessary() {
switch (getState()) {

View File

@ -256,7 +256,7 @@ public class StandardStateManagerProvider implements StateManagerProvider {
provider.initialize(initContext);
}
final ValidationContext validationContext = new StandardValidationContext(null, propertyStringMap, null, null, null, variableRegistry, null);
final ValidationContext validationContext = new StandardValidationContext(null, propertyStringMap, null, null, null, variableRegistry, null, true);
final Collection<ValidationResult> results = provider.validate(validationContext);
final StringBuilder validationFailures = new StringBuilder();

View File

@ -0,0 +1,102 @@
/*
* 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.logging.repository;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.logging.LogLevel;
import org.apache.nifi.logging.LogObserver;
import org.apache.nifi.logging.LogRepository;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
public class NopLogRepository implements LogRepository {
private final ConcurrentMap<String, LogLevel> observationLevels = new ConcurrentHashMap<>();
private volatile ComponentLog logger;
@Override
public void addLogMessage(final LogLevel level, final String message) {
}
@Override
public void addLogMessage(final LogLevel level, final String message, final Throwable t) {
}
@Override
public void addLogMessage(final LogLevel level, final String messageFormat, final Object[] params) {
}
@Override
public void addLogMessage(final LogLevel level, final String messageFormat, final Object[] params, final Throwable t) {
}
@Override
public void addObserver(final String observerIdentifier, final LogLevel level, final LogObserver observer) {
}
@Override
public void setObservationLevel(final String observerIdentifier, final LogLevel level) {
observationLevels.put(observerIdentifier, level);
}
@Override
public LogLevel getObservationLevel(final String observerIdentifier) {
return observationLevels.get(observerIdentifier);
}
@Override
public LogObserver removeObserver(final String observerIdentifier) {
observationLevels.remove(observerIdentifier);
return null;
}
@Override
public void removeAllObservers() {
observationLevels.clear();
}
@Override
public void setLogger(final ComponentLog logger) {
this.logger = logger;
}
@Override
public ComponentLog getLogger() {
return logger;
}
@Override
public boolean isDebugEnabled() {
return false;
}
@Override
public boolean isInfoEnabled() {
return true;
}
@Override
public boolean isWarnEnabled() {
return true;
}
@Override
public boolean isErrorEnabled() {
return true;
}
}

View File

@ -37,8 +37,12 @@ public class SimpleProcessLogger implements ComponentLog {
private final Object component;
public SimpleProcessLogger(final String componentId, final Object component) {
this(component, LogRepositoryFactory.getRepository(componentId));
}
public SimpleProcessLogger(final Object component, final LogRepository logRepository) {
this.logger = LoggerFactory.getLogger(component.getClass());
this.logRepository = LogRepositoryFactory.getRepository(componentId);
this.logRepository = logRepository;
this.component = component;
}

View File

@ -27,13 +27,17 @@ import org.apache.nifi.components.resource.StandardResourceContext;
import org.apache.nifi.components.resource.StandardResourceReferenceFactory;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.connectable.Connection;
import org.apache.nifi.controller.ComponentNode;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.controller.NodeTypeProvider;
import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.PropertyConfiguration;
import org.apache.nifi.controller.PropertyConfigurationMapper;
import org.apache.nifi.controller.lifecycle.TaskTermination;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.encrypt.PropertyEncryptor;
import org.apache.nifi.parameter.ParameterLookup;
import org.apache.nifi.processor.exception.TerminatedTaskException;
import org.apache.nifi.scheduling.ExecutionNode;
import org.apache.nifi.util.Connectables;
@ -57,18 +61,36 @@ public class StandardProcessContext implements ProcessContext, ControllerService
private final TaskTermination taskTermination;
private final NodeTypeProvider nodeTypeProvider;
private final Map<PropertyDescriptor, String> properties;
private final String annotationData;
public StandardProcessContext(final ProcessorNode processorNode, final ControllerServiceProvider controllerServiceProvider, final PropertyEncryptor propertyEncryptor,
final StateManager stateManager, final TaskTermination taskTermination, final NodeTypeProvider nodeTypeProvider) {
this(processorNode, controllerServiceProvider, propertyEncryptor, stateManager, taskTermination, nodeTypeProvider,
processorNode.getEffectivePropertyValues(), processorNode.getAnnotationData());
}
public StandardProcessContext(final ProcessorNode processorNode, final Map<String, String> propertiesOverride, final String annotationDataOverride, final ParameterLookup parameterLookup,
final ControllerServiceProvider controllerServiceProvider, final PropertyEncryptor propertyEncryptor,
final StateManager stateManager, final TaskTermination taskTermination, final NodeTypeProvider nodeTypeProvider) {
this(processorNode, controllerServiceProvider, propertyEncryptor, stateManager, taskTermination, nodeTypeProvider,
resolvePropertyValues(processorNode, parameterLookup, propertiesOverride), annotationDataOverride);
}
public StandardProcessContext(final ProcessorNode processorNode, final ControllerServiceProvider controllerServiceProvider, final PropertyEncryptor propertyEncryptor,
final StateManager stateManager, final TaskTermination taskTermination, final NodeTypeProvider nodeTypeProvider,
final Map<PropertyDescriptor, String> propertyValues, final String annotationData) {
this.procNode = processorNode;
this.controllerServiceProvider = controllerServiceProvider;
this.propertyEncryptor = propertyEncryptor;
this.stateManager = stateManager;
this.taskTermination = taskTermination;
this.nodeTypeProvider = nodeTypeProvider;
this.annotationData = annotationData;
properties = Collections.unmodifiableMap(processorNode.getEffectivePropertyValues());
properties = Collections.unmodifiableMap(propertyValues);
preparedQueries = new HashMap<>();
for (final Map.Entry<PropertyDescriptor, String> entry : properties.entrySet()) {
@ -85,6 +107,27 @@ public class StandardProcessContext implements ProcessContext, ControllerService
}
}
private static Map<PropertyDescriptor, String> resolvePropertyValues(final ComponentNode component, final ParameterLookup parameterLookup, final Map<String, String> propertyValues) {
final Map<PropertyDescriptor, String> resolvedProperties = new LinkedHashMap<>(component.getEffectivePropertyValues());
final PropertyConfigurationMapper configurationMapper = new PropertyConfigurationMapper();
for (final Map.Entry<String, String> entry : propertyValues.entrySet()) {
final String propertyName = entry.getKey();
final String propertyValue = entry.getValue();
final PropertyDescriptor propertyDescriptor = component.getPropertyDescriptor(propertyName);
if (propertyValue == null) {
resolvedProperties.remove(propertyDescriptor);
} else {
final PropertyConfiguration configuration = configurationMapper.mapRawPropertyValuesToPropertyConfiguration(propertyDescriptor, propertyValue);
final String effectiveValue = configuration.getEffectiveValue(parameterLookup);
resolvedProperties.put(propertyDescriptor, effectiveValue);
}
}
return resolvedProperties;
}
private void verifyTaskActive() {
if (taskTermination.isTerminated()) {
throw new TerminatedTaskException();
@ -163,7 +206,7 @@ public class StandardProcessContext implements ProcessContext, ControllerService
@Override
public String getAnnotationData() {
verifyTaskActive();
return procNode.getAnnotationData();
return annotationData;
}
@Override

View File

@ -64,7 +64,7 @@ public class StandardValidationContext extends AbstractValidationContext impleme
private final String componentId;
private final ParameterContext parameterContext;
private final AtomicReference<Map<PropertyDescriptor, String>> effectiveValuesRef = new AtomicReference<>();
private final boolean validateConnections;
public StandardValidationContext(
final ControllerServiceProvider controllerServiceProvider,
@ -73,7 +73,8 @@ public class StandardValidationContext extends AbstractValidationContext impleme
final String groupId,
final String componentId,
final VariableRegistry variableRegistry,
final ParameterContext parameterContext) {
final ParameterContext parameterContext,
final boolean validateConnections) {
super(parameterContext, properties);
this.controllerServiceProvider = controllerServiceProvider;
@ -83,6 +84,7 @@ public class StandardValidationContext extends AbstractValidationContext impleme
this.groupId = groupId;
this.componentId = componentId;
this.parameterContext = parameterContext;
this.validateConnections = validateConnections;
preparedQueries = new HashMap<>(properties.size());
for (final Map.Entry<PropertyDescriptor, PropertyConfiguration> entry : properties.entrySet()) {
@ -121,7 +123,7 @@ public class StandardValidationContext extends AbstractValidationContext impleme
final ProcessGroup serviceGroup = serviceNode.getProcessGroup();
final String serviceGroupId = serviceGroup == null ? null : serviceGroup.getIdentifier();
return new StandardValidationContext(controllerServiceProvider, serviceNode.getProperties(), serviceNode.getAnnotationData(), serviceGroupId,
serviceNode.getIdentifier(), variableRegistry, serviceNode.getProcessGroup().getParameterContext());
serviceNode.getIdentifier(), variableRegistry, serviceNode.getProcessGroup().getParameterContext(), validateConnections);
}
@Override
@ -241,6 +243,10 @@ public class StandardValidationContext extends AbstractValidationContext impleme
return value != null;
}
@Override
public boolean isValidateConnections() {
return validateConnections;
}
@Override
public String toString() {

View File

@ -38,7 +38,7 @@ public class StandardValidationContextFactory implements ValidationContextFactor
@Override
public ValidationContext newValidationContext(final Map<PropertyDescriptor, PropertyConfiguration> properties, final String annotationData, final String groupId, final String componentId,
final ParameterContext parameterContext) {
return new StandardValidationContext(serviceProvider, properties, annotationData, groupId, componentId, variableRegistry, parameterContext);
final ParameterContext parameterContext, final boolean validateConnections) {
return new StandardValidationContext(serviceProvider, properties, annotationData, groupId, componentId, variableRegistry, parameterContext, validateConnections);
}
}

View File

@ -17,9 +17,12 @@
package org.apache.nifi.controller;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.attribute.expression.language.Query;
import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
import org.apache.nifi.bundle.Bundle;
import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
@ -52,6 +55,7 @@ import org.apache.nifi.parameter.ParameterTokenList;
import org.apache.nifi.parameter.ParameterUpdate;
import org.apache.nifi.registry.ComponentVariableRegistry;
import org.apache.nifi.util.CharacterFilterUtils;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -60,6 +64,7 @@ import java.net.URL;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.LinkedHashSet;
import java.util.List;
@ -78,6 +83,7 @@ import java.util.function.Function;
import java.util.stream.Collectors;
public abstract class AbstractComponentNode implements ComponentNode {
private static final String PERFORM_VALIDATION_STEP_NAME = "Perform Validation";
private static final Logger logger = LoggerFactory.getLogger(AbstractComponentNode.class);
private final String id;
@ -163,14 +169,17 @@ public abstract class AbstractComponentNode implements ComponentNode {
return getAdditionalClasspathResources((Collection<PropertyDescriptor>) propertyDescriptors);
}
private Set<URL> getAdditionalClasspathResources(final Collection<PropertyDescriptor> propertyDescriptors) {
protected Set<URL> getAdditionalClasspathResources(final Collection<PropertyDescriptor> propertyDescriptors) {
return getAdditionalClasspathResources(propertyDescriptors, this::getEffectivePropertyValue);
}
protected Set<URL> getAdditionalClasspathResources(final Collection<PropertyDescriptor> propertyDescriptors, final Function<PropertyDescriptor, String> effectiveValueLookup) {
final Set<URL> additionalUrls = new LinkedHashSet<>();
final ResourceReferenceFactory resourceReferenceFactory = new StandardResourceReferenceFactory();
for (final PropertyDescriptor descriptor : propertyDescriptors) {
if (descriptor.isDynamicClasspathModifier()) {
final PropertyConfiguration propertyConfiguration = getProperty(descriptor);
final String value = propertyConfiguration == null ? null : propertyConfiguration.getEffectiveValue(getParameterContext());
final String value = effectiveValueLookup.apply(descriptor);
if (!StringUtils.isEmpty(value)) {
final ResourceContext resourceContext = new StandardResourceContext(resourceReferenceFactory, descriptor);
@ -184,6 +193,36 @@ public abstract class AbstractComponentNode implements ComponentNode {
return additionalUrls;
}
/**
* Determines if the given set of properties will result in a different classpath than the currently configured set of properties
* @param properties the properties to analyze
* @return <code>true</code> if the given set of properties will require a different classpath (and therefore a different classloader) than the currently
* configured set of properties
*/
protected boolean isClasspathDifferent(final Map<PropertyDescriptor, String> properties) {
// If any property in the given map modifies classpath and is different than the currently configured value,
// the given properties will require a different classpath.
for (final Map.Entry<PropertyDescriptor, String> entry : properties.entrySet()) {
final PropertyDescriptor descriptor = entry.getKey();
final String value = entry.getValue();
final String currentlyConfiguredValue = getEffectivePropertyValue(descriptor);
if (descriptor.isDynamicClasspathModifier() && !Objects.equals(value, currentlyConfiguredValue)) {
return true;
}
}
// If any property in the currently configured properties modifies classpath and is not in the given set of properties,
// the given properties will require a different classpath.
for (final Map.Entry<PropertyDescriptor, PropertyConfiguration> entry : getProperties().entrySet()) {
final PropertyDescriptor descriptor = entry.getKey();
if (descriptor.isDynamicClasspathModifier() && !properties.containsKey(descriptor)) {
return true;
}
}
return false;
}
@Override
public void setProperties(final Map<String, String> properties, final boolean allowRemovalOfRequiredProperties) {
@ -195,10 +234,8 @@ public abstract class AbstractComponentNode implements ComponentNode {
try {
verifyCanUpdateProperties(properties);
// Keep track of counts of each parameter reference. This way, when we complete the updates to property values, we can
// update our counts easily.
final ParameterParser elAgnosticParameterParser = new ExpressionLanguageAgnosticParameterParser();
final ParameterParser elAwareParameterParser = new ExpressionLanguageAwareParameterParser();
final PropertyConfigurationMapper configurationMapper = new PropertyConfigurationMapper();
final Map<String, PropertyConfiguration> configurationMap = configurationMapper.mapRawPropertyValuesToPropertyConfiguration(this, properties);
try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, getComponent().getClass(), id)) {
boolean classpathChanged = false;
@ -219,27 +256,17 @@ public abstract class AbstractComponentNode implements ComponentNode {
if (entry.getKey() != null && entry.getValue() == null) {
removeProperty(entry.getKey(), allowRemovalOfRequiredProperties);
} else if (entry.getKey() != null) {
final String updatedValue = CharacterFilterUtils.filterInvalidXmlCharacters(entry.getValue());
// Use the EL-Agnostic Parameter Parser to gather the list of referenced Parameters. We do this because we want to to keep track of which parameters
// are referenced, regardless of whether or not they are referenced from within an EL Expression. However, we also will need to derive a different ParameterTokenList
// that we can provide to the PropertyConfiguration, so that when compiling the Expression Language Expressions, we are able to keep the Parameter Reference within
// the Expression's text.
final ParameterTokenList updatedValueReferences = elAgnosticParameterParser.parseTokens(updatedValue);
final List<ParameterReference> parameterReferences = updatedValueReferences.toReferenceList();
final PropertyConfiguration propertyConfiguration = configurationMap.get(entry.getKey());
final List<ParameterReference> parameterReferences = propertyConfiguration.getParameterReferences();
for (final ParameterReference reference : parameterReferences) {
// increment count in map for this parameter
parameterReferenceCounts.merge(reference.getParameterName(), 1, (a, b) -> a == -1 ? null : a + b);
}
final PropertyConfiguration propertyConfiguration;
final boolean supportsEL = getPropertyDescriptor(entry.getKey()).isExpressionLanguageSupported();
if (supportsEL) {
propertyConfiguration = new PropertyConfiguration(updatedValue, elAwareParameterParser.parseTokens(updatedValue), parameterReferences);
} else {
propertyConfiguration = new PropertyConfiguration(updatedValue, updatedValueReferences, parameterReferences);
}
setProperty(entry.getKey(), propertyConfiguration, this.properties::get);
}
}
@ -324,6 +351,72 @@ public abstract class AbstractComponentNode implements ComponentNode {
}
}
protected List<ConfigVerificationResult> verifyConfig(final Map<PropertyDescriptor, String> propertyValues, final String annotationData, final ParameterContext parameterContext) {
final List<ConfigVerificationResult> results = new ArrayList<>();
try {
final long startNanos = System.nanoTime();
final Map<PropertyDescriptor, PropertyConfiguration> descriptorToConfigMap = new LinkedHashMap<>();
for (final Map.Entry<PropertyDescriptor, String> entry : propertyValues.entrySet()) {
final PropertyDescriptor descriptor = entry.getKey();
final String rawValue = entry.getValue();
final String propertyValue = rawValue == null ? descriptor.getDefaultValue() : rawValue;
final PropertyConfiguration propertyConfiguration = new PropertyConfiguration(propertyValue, null, Collections.emptyList());
descriptorToConfigMap.put(descriptor, propertyConfiguration);
}
final ValidationContext validationContext = getValidationContextFactory().newValidationContext(descriptorToConfigMap, annotationData,
getProcessGroupIdentifier(), getIdentifier(), parameterContext, false);
final ValidationState validationState = performValidation(validationContext);
final ValidationStatus validationStatus = validationState.getStatus();
if (validationStatus == ValidationStatus.INVALID) {
for (final ValidationResult result : validationState.getValidationErrors()) {
if (result.isValid()) {
continue;
}
results.add(new ConfigVerificationResult.Builder()
.verificationStepName(PERFORM_VALIDATION_STEP_NAME)
.outcome(Outcome.FAILED)
.explanation("Component is invalid: " + result.toString())
.build());
}
if (results.isEmpty()) {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName(PERFORM_VALIDATION_STEP_NAME)
.outcome(Outcome.FAILED)
.explanation("Component is invalid but provided no Validation Results to indicate why")
.build());
}
logger.debug("{} is not valid with the given configuration. Will not attempt to perform any additional verification of configuration. Validation took {}. Reason not valid: {}",
this, results, FormatUtils.formatNanos(System.nanoTime() - startNanos, false));
return results;
}
results.add(new ConfigVerificationResult.Builder()
.verificationStepName(PERFORM_VALIDATION_STEP_NAME)
.outcome(Outcome.SUCCESSFUL)
.explanation("Component Validation passed")
.build());
} catch (final Throwable t) {
logger.error("Failed to perform verification of component's configuration for {}", this, t);
results.add(new ConfigVerificationResult.Builder()
.verificationStepName(PERFORM_VALIDATION_STEP_NAME)
.outcome(Outcome.FAILED)
.explanation("Encountered unexpected failure when attempting to perform verification: " + t)
.build());
}
return results;
}
@Override
public Set<String> getReferencedParameterNames() {
return Collections.unmodifiableSet(parameterReferenceCounts.keySet());
@ -334,6 +427,19 @@ public abstract class AbstractComponentNode implements ComponentNode {
return !parameterReferenceCounts.isEmpty();
}
@Override
public Set<String> getReferencedAttributeNames() {
final Set<String> referencedAttributes = new HashSet<>();
for (final PropertyDescriptor descriptor : getPropertyDescriptors()) {
final String effectiveValue = getEffectivePropertyValue(descriptor);
final Set<String> attributes = Query.prepareWithParametersPreEvaluated(effectiveValue).getExplicitlyReferencedAttributes();
referencedAttributes.addAll(attributes);
}
return referencedAttributes;
}
// Keep setProperty/removeProperty private so that all calls go through setProperties
private void setProperty(final String name, final PropertyConfiguration propertyConfiguration, final Function<PropertyDescriptor, PropertyConfiguration> valueToCompareFunction) {
if (name == null || propertyConfiguration == null || propertyConfiguration.getRawValue() == null) {
@ -563,7 +669,7 @@ public abstract class AbstractComponentNode implements ComponentNode {
@Override
public ValidationState performValidation(final Map<PropertyDescriptor, PropertyConfiguration> properties, final String annotationData, final ParameterContext parameterContext) {
final ValidationContext validationContext = validationContextFactory.newValidationContext(properties, annotationData, getProcessGroupIdentifier(), getIdentifier(), parameterContext);
final ValidationContext validationContext = validationContextFactory.newValidationContext(properties, annotationData, getProcessGroupIdentifier(), getIdentifier(), parameterContext, true);
return performValidation(validationContext);
}
@ -1090,7 +1196,7 @@ public abstract class AbstractComponentNode implements ComponentNode {
return context;
}
context = getValidationContextFactory().newValidationContext(getProperties(), getAnnotationData(), getProcessGroupIdentifier(), getIdentifier(), getParameterContext());
context = getValidationContextFactory().newValidationContext(getProperties(), getAnnotationData(), getProcessGroupIdentifier(), getIdentifier(), getParameterContext(), true);
this.validationContext = context;
logger.debug("Updating validation context to {}", context);

View File

@ -78,6 +78,11 @@ public interface ComponentNode extends ComponentAuthorizable {
*/
void onParametersModified(Map<String, ParameterUpdate> parameterUpdates);
/**
* @return the Set of all attributes that are explicitly referenced by Expression Language used in the property values
*/
Set<String> getReferencedAttributeNames();
/**
* <p>
* Pause triggering asynchronous validation to occur when the component is updated. Often times, it is necessary

View File

@ -24,8 +24,10 @@ import org.apache.nifi.controller.scheduling.LifecycleState;
import org.apache.nifi.controller.scheduling.SchedulingAgent;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.logging.LogLevel;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.processor.Relationship;
@ -137,6 +139,14 @@ public abstract class ProcessorNode extends AbstractComponentNode implements Con
*/
public abstract void verifyCanStart(Set<ControllerServiceNode> ignoredReferences);
public void verifyCanPerformVerification() {
if (isRunning()) {
throw new IllegalStateException("Cannot perform verification because the Processor is not stopped");
}
}
public abstract List<ConfigVerificationResult> verifyConfiguration(ProcessContext processContext, ComponentLog logger, Map<String, String> attributes, ExtensionManager extensionManager);
public abstract void verifyCanTerminate();
/**

View File

@ -0,0 +1,70 @@
/*
* 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.controller;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.parameter.ExpressionLanguageAgnosticParameterParser;
import org.apache.nifi.parameter.ExpressionLanguageAwareParameterParser;
import org.apache.nifi.parameter.ParameterParser;
import org.apache.nifi.parameter.ParameterReference;
import org.apache.nifi.parameter.ParameterTokenList;
import org.apache.nifi.util.CharacterFilterUtils;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
public class PropertyConfigurationMapper {
private final ParameterParser elAgnosticParameterParser = new ExpressionLanguageAgnosticParameterParser();
private final ParameterParser elAwareParameterParser = new ExpressionLanguageAwareParameterParser();
public Map<String, PropertyConfiguration> mapRawPropertyValuesToPropertyConfiguration(final ComponentNode componentNode, final Map<String, String> rawPropertyValues) {
final Map<String, PropertyConfiguration> configurationMap = new LinkedHashMap<>();
for (final Map.Entry<String, String> entry : rawPropertyValues.entrySet()) {
final String propertyName = entry.getKey();
final String propertyValue = entry.getValue();
final PropertyDescriptor propertyDescriptor = componentNode.getPropertyDescriptor(propertyName);
final PropertyConfiguration configuration = mapRawPropertyValuesToPropertyConfiguration(propertyDescriptor, propertyValue);
configurationMap.put(propertyName, configuration);
}
return configurationMap;
}
public PropertyConfiguration mapRawPropertyValuesToPropertyConfiguration(final PropertyDescriptor propertyDescriptor, final String propertyValue) {
final String updatedValue = CharacterFilterUtils.filterInvalidXmlCharacters(propertyValue);
// Use the EL-Agnostic Parameter Parser to gather the list of referenced Parameters. We do this because we want to to keep track of which parameters
// are referenced, regardless of whether or not they are referenced from within an EL Expression. However, we also will need to derive a different ParameterTokenList
// that we can provide to the PropertyConfiguration, so that when compiling the Expression Language Expressions, we are able to keep the Parameter Reference within
// the Expression's text.
final ParameterTokenList updatedValueReferences = elAgnosticParameterParser.parseTokens(updatedValue);
final List<ParameterReference> parameterReferences = updatedValueReferences.toReferenceList();
final PropertyConfiguration propertyConfiguration;
final boolean supportsEL = propertyDescriptor.isExpressionLanguageSupported();
if (supportsEL) {
return new PropertyConfiguration(updatedValue, elAwareParameterParser.parseTokens(updatedValue), parameterReferences);
} else {
return new PropertyConfiguration(updatedValue, updatedValueReferences, parameterReferences);
}
}
}

View File

@ -17,10 +17,14 @@
package org.apache.nifi.controller;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.reporting.ReportingContext;
import org.apache.nifi.reporting.ReportingTask;
import org.apache.nifi.scheduling.SchedulingStrategy;
import java.util.List;
import java.util.Set;
import java.util.concurrent.TimeUnit;
@ -67,7 +71,7 @@ public interface ReportingTaskNode extends ComponentNode {
* has no active threads, only that it is not currently scheduled to be
* given any more threads. To determine whether or not the
* <code>ReportingTask</code> has any active threads, see
* {@link ProcessScheduler#getActiveThreadCount(ReportingTask)}
* {@link ProcessScheduler#getActiveThreadCount(Object)}
*/
ScheduledState getScheduledState();
@ -102,4 +106,23 @@ public interface ReportingTaskNode extends ComponentNode {
void verifyCanUpdate();
void verifyCanClearState();
/**
* Verifies that the Reporting Task is in a state in which it can verify a configuration by calling
* {@link #verifyConfiguration(ConfigurationContext, ComponentLog, ExtensionManager)}.
*
* @throws IllegalStateException if not in a state in which configuration can be verified
*/
void verifyCanPerformVerification();
/**
* Verifies that the given configuration is valid for the Reporting Task
*
* @param context the configuration to verify
* @param logger a logger that can be used when performing verification
* @param extensionManager extension manager that is used for obtaining appropriate NAR ClassLoaders
* @return a list of results indicating whether or not the given configuration is valid
*/
List<ConfigVerificationResult> verifyConfiguration(ConfigurationContext context, ComponentLog logger, ExtensionManager extensionManager);
}

View File

@ -24,6 +24,7 @@ import java.util.Map;
public interface ValidationContextFactory {
ValidationContext newValidationContext(Map<PropertyDescriptor, PropertyConfiguration> properties, String annotationData, String groupId, String componentId, ParameterContext parameterContext);
ValidationContext newValidationContext(Map<PropertyDescriptor, PropertyConfiguration> properties, String annotationData, String groupId, String componentId, ParameterContext parameterContext,
boolean validateConnections);
}

View File

@ -19,11 +19,16 @@ package org.apache.nifi.controller.service;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.VersionedComponent;
import org.apache.nifi.controller.ComponentNode;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.LoggableComponent;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.components.ConfigVerificationResult;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ScheduledExecutorService;
@ -191,6 +196,25 @@ public interface ControllerServiceNode extends ComponentNode, VersionedComponent
*/
boolean awaitEnabled(long timePeriod, TimeUnit timeUnit) throws InterruptedException;
/**
* Verifies that the Controller Service is in a state in which it can verify a configuration by calling
* {@link #verifyConfiguration(ConfigurationContext, ComponentLog, Map, ExtensionManager)}.
*
* @throws IllegalStateException if not in a state in which configuration can be verified
*/
void verifyCanPerformVerification();
/**
* Verifies that the given configuration is valid for the Controller Service
*
* @param context the configuration to verify
* @param logger a logger that can be used when performing verification
* @param variables variables that can be used to resolve property values via Expression Language
* @param extensionManager extension manager that is used for obtaining appropriate NAR ClassLoaders
* @return a list of results indicating whether or not the given configuration is valid
*/
List<ConfigVerificationResult> verifyConfiguration(ConfigurationContext context, ComponentLog logger, Map<String, String> variables, ExtensionManager extensionManager);
/**
* Sets a new proxy and implementation for this node.
*

View File

@ -238,6 +238,7 @@ public class TemplateUtils {
processorConfig.setCustomUiUrl(null);
processorConfig.setDefaultConcurrentTasks(null);
processorConfig.setDefaultSchedulingPeriod(null);
processorConfig.setReferencedAttributes(null);
processorConfig.setAutoTerminatedRelationships(null);
}
@ -306,6 +307,7 @@ public class TemplateUtils {
serviceDTO.setCustomUiUrl(null);
serviceDTO.setValidationErrors(null);
serviceDTO.setValidationStatus(null);
serviceDTO.setReferencedAttributes(null);
}
}

View File

@ -18,7 +18,6 @@ package org.apache.nifi.controller.reporting;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.documentation.DeprecationNotice;
import org.apache.nifi.parameter.ParameterLookup;
import org.apache.nifi.authorization.Resource;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.authorization.resource.ResourceFactory;
@ -32,12 +31,16 @@ import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.ValidationContextFactory;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.parameter.ParameterContext;
import org.apache.nifi.parameter.ParameterLookup;
import org.apache.nifi.registry.ComponentVariableRegistry;
import org.apache.nifi.reporting.ReportingContext;
import org.apache.nifi.reporting.ReportingTask;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class StandardReportingTaskNode extends AbstractReportingTaskNode implements ReportingTaskNode {
private static final Logger logger = LoggerFactory.getLogger(StandardReportingTaskNode.class);
private final FlowController flowController;
public StandardReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id, final FlowController controller,

View File

@ -562,7 +562,7 @@ public class StandardProcessorNodeIT {
return new ValidationContextFactory() {
@Override
public ValidationContext newValidationContext(Map<PropertyDescriptor, PropertyConfiguration> properties, String annotationData, String groupId, String componentId,
ParameterContext context) {
ParameterContext context, boolean validateConnections) {
return new ValidationContext() {
@Override

View File

@ -41,6 +41,11 @@ public class MockConfigurationContext implements ConfigurationContext {
return Collections.emptyMap();
}
@Override
public String getAnnotationData() {
return null;
}
@Override
public String getSchedulingPeriod() {
return "0 secs";

View File

@ -45,7 +45,21 @@ public interface ExtensionManager {
* @param additionalUrls additional URLs to add to the instance class loader
* @return the ClassLoader for the given instance of the given type, or null if the type is not a detected extension type
*/
InstanceClassLoader createInstanceClassLoader(String classType, String instanceIdentifier, Bundle bundle, Set<URL> additionalUrls);
default InstanceClassLoader createInstanceClassLoader(String classType, String instanceIdentifier, Bundle bundle, Set<URL> additionalUrls) {
return createInstanceClassLoader(classType, instanceIdentifier, bundle, additionalUrls, true);
}
/**
* Creates the ClassLoader for the instance of the given type.
*
* @param classType the type of class to create the ClassLoader for
* @param instanceIdentifier the identifier of the specific instance of the classType to look up the ClassLoader for
* @param bundle the bundle where the classType exists
* @param additionalUrls additional URLs to add to the instance class loader
* @param registerClassLoader whether or not to register the class loader as the new classloader for the component with the given ID
* @return the ClassLoader for the given instance of the given type, or null if the type is not a detected extension type
*/
InstanceClassLoader createInstanceClassLoader(String classType, String instanceIdentifier, Bundle bundle, Set<URL> additionalUrls, boolean registerClassLoader);
/**
* Retrieves the InstanceClassLoader for the component with the given identifier.

View File

@ -356,7 +356,7 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering
}
@Override
public InstanceClassLoader createInstanceClassLoader(final String classType, final String instanceIdentifier, final Bundle bundle, final Set<URL> additionalUrls) {
public InstanceClassLoader createInstanceClassLoader(final String classType, final String instanceIdentifier, final Bundle bundle, final Set<URL> additionalUrls, final boolean register) {
if (StringUtils.isEmpty(classType)) {
throw new IllegalArgumentException("Class-Type is required");
}
@ -427,7 +427,10 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering
}
}
instanceClassloaderLookup.put(instanceIdentifier, instanceClassLoader);
if (register) {
instanceClassloaderLookup.put(instanceIdentifier, instanceClassLoader);
}
return instanceClassLoader;
}

View File

@ -41,6 +41,7 @@ import org.apache.nifi.web.api.dto.BundleDTO;
import org.apache.nifi.web.api.dto.ClusterDTO;
import org.apache.nifi.web.api.dto.ComponentHistoryDTO;
import org.apache.nifi.web.api.dto.ComponentStateDTO;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.dto.ConnectionDTO;
import org.apache.nifi.web.api.dto.ControllerConfigurationDTO;
import org.apache.nifi.web.api.dto.ControllerDTO;
@ -58,6 +59,7 @@ import org.apache.nifi.web.api.dto.NodeDTO;
import org.apache.nifi.web.api.dto.ParameterContextDTO;
import org.apache.nifi.web.api.dto.PortDTO;
import org.apache.nifi.web.api.dto.ProcessGroupDTO;
import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
import org.apache.nifi.web.api.dto.ProcessorDTO;
import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
import org.apache.nifi.web.api.dto.RegistryDTO;
@ -633,6 +635,15 @@ public interface NiFiServiceFacade {
*/
ProcessorEntity updateProcessor(Revision revision, ProcessorDTO processorDTO);
/**
* Performs verification of the given Processor Configuration for the Processor with the given ID
* @param processorId the id of the processor
* @param processorConfig the configuration to verify
* @param attributes a map of values that can be used for resolving FlowFile attributes for Expression Language
* @return verification results
*/
List<ConfigVerificationResultDTO> verifyProcessorConfiguration(String processorId, ProcessorConfigDTO processorConfig, Map<String, String> attributes);
/**
* Verifies the specified processor can be removed.
*
@ -1618,6 +1629,25 @@ public interface NiFiServiceFacade {
*/
void verifyCanUpdate(String groupId, VersionedFlowSnapshot proposedFlow, boolean verifyConnectionRemoval, boolean verifyNotDirty);
/**
* Verifies that the Processor with the given identifier is in a state where its configuration can be verified
* @param processorId the ID of the processor
*/
void verifyCanVerifyProcessorConfig(String processorId);
/**
* Verifies that the Controller Service with the given identifier is in a state where its configuration can be verified
* @param controllerServiceId the ID of the service
*/
void verifyCanVerifyControllerServiceConfig(String controllerServiceId);
/**
* Verifies that the Reporting Task with the given identifier is in a state where its configuration can be verified
* @param reportingTaskId the ID of the service
*/
void verifyCanVerifyReportingTaskConfig(String reportingTaskId);
/**
* Verifies that the Process Group with the given identifier can be saved to the flow registry
*
@ -2013,6 +2043,16 @@ public interface NiFiServiceFacade {
*/
ControllerServiceEntity updateControllerService(Revision revision, ControllerServiceDTO controllerServiceDTO);
/**
* Performs verification of the given Configuration for the Controller Service with the given ID
* @param controllerServiceId the id of the controller service
* @param controllerService the configuration to verify
* @param variables a map of values that can be used for resolving FlowFile attributes for Expression Language
* @return verification results
*/
List<ConfigVerificationResultDTO> verifyControllerServiceConfiguration(String controllerServiceId, ControllerServiceDTO controllerService, Map<String, String> variables);
/**
* Deletes the specified label.
*
@ -2098,6 +2138,14 @@ public interface NiFiServiceFacade {
*/
ReportingTaskEntity updateReportingTask(Revision revision, ReportingTaskDTO reportingTaskDTO);
/**
* Performs verification of the given Configuration for the Reporting Task with the given ID
* @param reportingTaskId the id of the reporting task
* @param reportingTask the configuration to verify
* @return verification results
*/
List<ConfigVerificationResultDTO> verifyReportingTaskConfiguration(String reportingTaskId, ReportingTaskDTO reportingTask);
/**
* Deletes the specified reporting task.
*

View File

@ -173,6 +173,7 @@ import org.apache.nifi.web.api.dto.ComponentReferenceDTO;
import org.apache.nifi.web.api.dto.ComponentRestrictionPermissionDTO;
import org.apache.nifi.web.api.dto.ComponentStateDTO;
import org.apache.nifi.web.api.dto.ComponentValidationResultDTO;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.dto.ConnectionDTO;
import org.apache.nifi.web.api.dto.ControllerConfigurationDTO;
import org.apache.nifi.web.api.dto.ControllerDTO;
@ -546,6 +547,21 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
}
}
@Override
public void verifyCanVerifyProcessorConfig(final String processorId) {
processorDAO.verifyConfigVerification(processorId);
}
@Override
public void verifyCanVerifyControllerServiceConfig(final String controllerServiceId) {
controllerServiceDAO.verifyConfigVerification(controllerServiceId);
}
@Override
public void verifyCanVerifyReportingTaskConfig(final String reportingTaskId) {
reportingTaskDAO.verifyConfigVerification(reportingTaskId);
}
@Override
public void verifyDeleteProcessor(final String processorId) {
processorDAO.verifyDelete(processorId);
@ -748,6 +764,11 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
return entityFactory.createProcessorEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), permissions, operatePermissions, status, bulletinEntities);
}
@Override
public List<ConfigVerificationResultDTO> verifyProcessorConfiguration(final String processorId, final ProcessorConfigDTO processorConfig, final Map<String, String> attributes) {
return processorDAO.verifyProcessorConfiguration(processorId, processorConfig, attributes);
}
private void awaitValidationCompletion(final ComponentNode component) {
component.getValidationStatus(VALIDATION_WAIT_MILLIS, TimeUnit.MILLISECONDS);
}
@ -2719,6 +2740,10 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
return entityFactory.createControllerServiceEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), permissions, operatePermissions, bulletinEntities);
}
@Override
public List<ConfigVerificationResultDTO> verifyControllerServiceConfiguration(final String controllerServiceId, final ControllerServiceDTO controllerService, final Map<String, String> variables) {
return controllerServiceDAO.verifyConfiguration(controllerServiceId, controllerService, variables);
}
@Override
public ControllerServiceReferencingComponentsEntity updateControllerServiceReferencingComponents(
@ -3098,6 +3123,11 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
return entityFactory.createReportingTaskEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), permissions, operatePermissions, bulletinEntities);
}
@Override
public List<ConfigVerificationResultDTO> verifyReportingTaskConfiguration(final String reportingTaskId, final ReportingTaskDTO reportingTask) {
return reportingTaskDAO.verifyConfiguration(reportingTaskId, reportingTask);
}
@Override
public ReportingTaskEntity deleteReportingTask(final Revision revision, final String reportingTaskId) {
final ReportingTaskNode reportingTask = reportingTaskDAO.getReportingTask(reportingTaskId);

View File

@ -30,25 +30,36 @@ import org.apache.nifi.authorization.ComponentAuthorizable;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.authorization.resource.OperationAuthorizable;
import org.apache.nifi.authorization.user.NiFiUser;
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;
import org.apache.nifi.ui.extension.UiExtensionMapping;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.ResourceNotFoundException;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UiExtensionType;
import org.apache.nifi.web.api.concurrent.AsyncRequestManager;
import org.apache.nifi.web.api.concurrent.AsynchronousWebRequest;
import org.apache.nifi.web.api.concurrent.RequestManager;
import org.apache.nifi.web.api.concurrent.StandardAsynchronousWebRequest;
import org.apache.nifi.web.api.concurrent.StandardUpdateStep;
import org.apache.nifi.web.api.concurrent.UpdateStep;
import org.apache.nifi.web.api.dto.BundleDTO;
import org.apache.nifi.web.api.dto.ComponentStateDTO;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
import org.apache.nifi.web.api.dto.RevisionDTO;
import org.apache.nifi.web.api.dto.VerifyControllerServiceConfigRequestDTO;
import org.apache.nifi.web.api.entity.ComponentStateEntity;
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity;
import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity;
import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
import org.apache.nifi.web.api.entity.UpdateControllerServiceReferenceRequestEntity;
import org.apache.nifi.web.api.entity.VerifyControllerServiceConfigRequestEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import org.slf4j.Logger;
@ -70,10 +81,13 @@ 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.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
import java.util.stream.Collectors;
/**
@ -87,6 +101,9 @@ import java.util.stream.Collectors;
public class ControllerServiceResource extends ApplicationResource {
private static final Logger logger = LoggerFactory.getLogger(ControllerServiceResource.class);
private static final String VERIFICATION_REQUEST_TYPE = "verification-request";
private RequestManager<VerifyControllerServiceConfigRequestEntity, List<ConfigVerificationResultDTO>> updateRequestManager =
new AsyncRequestManager<>(100, TimeUnit.MINUTES.toMillis(1L), "Verify Controller Service Config Thread");
private NiFiServiceFacade serviceFacade;
private Authorizer authorizer;
@ -824,6 +841,228 @@ public class ControllerServiceResource extends ApplicationResource {
);
}
@POST
@Consumes(MediaType.APPLICATION_JSON)
@Produces(MediaType.APPLICATION_JSON)
@Path("/{id}/config/verification-requests")
@ApiOperation(
value = "Performs verification of the Controller Service's configuration",
response = VerifyControllerServiceConfigRequestEntity.class,
notes = "This will initiate the process of verifying a given Controller Service configuration. This may be a long-running task. As a result, this endpoint will immediately return a " +
"ControllerServiceConfigVerificationRequestEntity, and the process of performing the verification will occur asynchronously in the background. " +
"The client may then periodically poll the status of the request by " +
"issuing a GET request to /controller-services/{serviceId}/verification-requests/{requestId}. Once the request is completed, the client is expected to issue a DELETE request to " +
"/controller-services/{serviceId}/verification-requests/{requestId}.",
authorizations = {
@Authorization(value = "Read - /controller-services/{uuid}")
}
)
@ApiResponses(
value = {
@ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
@ApiResponse(code = 401, message = "Client could not be authenticated."),
@ApiResponse(code = 403, message = "Client is not authorized to make this request."),
@ApiResponse(code = 404, message = "The specified resource could not be found."),
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
}
)
public Response submitConfigVerificationRequest(
@ApiParam(value = "The controller service id.", required = true) @PathParam("id") final String controllerServiceId,
@ApiParam(value = "The controller service configuration verification request.", required = true) final VerifyControllerServiceConfigRequestEntity controllerServiceConfigRequest) {
if (controllerServiceConfigRequest == null) {
throw new IllegalArgumentException("Controller Service's configuration must be specified");
}
final VerifyControllerServiceConfigRequestDTO requestDto = controllerServiceConfigRequest.getRequest();
if (requestDto == null || requestDto.getControllerService() == null) {
throw new IllegalArgumentException("Controller Service must be specified");
}
if (requestDto.getControllerServiceId() == null) {
throw new IllegalArgumentException("Controller Service's identifier must be specified in the request");
}
if (!requestDto.getControllerServiceId().equals(controllerServiceId)) {
throw new IllegalArgumentException("Controller Service's identifier in the request must match the identifier provided in the URL");
}
if (isReplicateRequest()) {
return replicate(HttpMethod.POST, controllerServiceConfigRequest);
}
final NiFiUser user = NiFiUserUtils.getNiFiUser();
return withWriteLock(
serviceFacade,
controllerServiceConfigRequest,
lookup -> {
final ComponentAuthorizable controllerService = lookup.getControllerService(controllerServiceId);
controllerService.getAuthorizable().authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
},
() -> {
serviceFacade.verifyCanVerifyControllerServiceConfig(controllerServiceId);
},
entity -> performAsyncConfigVerification(entity, user)
);
}
@GET
@Consumes(MediaType.WILDCARD)
@Produces(MediaType.APPLICATION_JSON)
@Path("{id}/config/verification-requests/{requestId}")
@ApiOperation(
value = "Returns the Verification Request with the given ID",
response = VerifyControllerServiceConfigRequestEntity.class,
notes = "Returns the Verification Request with the given ID. Once an Verification Request has been created, "
+ "that request can subsequently be retrieved via this endpoint, and the request that is fetched will contain the updated state, such as percent complete, the "
+ "current state of the request, and any failures. ",
authorizations = {
@Authorization(value = "Only the user that submitted the request can get it")
})
@ApiResponses(value = {
@ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
@ApiResponse(code = 401, message = "Client could not be authenticated."),
@ApiResponse(code = 403, message = "Client is not authorized to make this request."),
@ApiResponse(code = 404, message = "The specified resource could not be found."),
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
})
public Response getVerificationRequest(
@ApiParam("The ID of the Controller Service") @PathParam("id") final String controllerServiceId,
@ApiParam("The ID of the Verification Request") @PathParam("requestId") final String requestId) {
if (isReplicateRequest()) {
return replicate(HttpMethod.GET);
}
final NiFiUser user = NiFiUserUtils.getNiFiUser();
// request manager will ensure that the current is the user that submitted this request
final AsynchronousWebRequest<VerifyControllerServiceConfigRequestEntity, List<ConfigVerificationResultDTO>> asyncRequest =
updateRequestManager.getRequest(VERIFICATION_REQUEST_TYPE, requestId, user);
final VerifyControllerServiceConfigRequestEntity updateRequestEntity = createVerifyControllerServiceConfigRequestEntity(asyncRequest, requestId);
return generateOkResponse(updateRequestEntity).build();
}
@DELETE
@Consumes(MediaType.WILDCARD)
@Produces(MediaType.APPLICATION_JSON)
@Path("{id}/config/verification-requests/{requestId}")
@ApiOperation(
value = "Deletes the Verification Request with the given ID",
response = VerifyControllerServiceConfigRequestEntity.class,
notes = "Deletes the Verification Request with the given ID. After a request is created, it is expected "
+ "that the client will properly clean up the request by DELETE'ing it, once the Verification process has completed. If the request is deleted before the request "
+ "completes, then the Verification request will finish the step that it is currently performing and then will cancel any subsequent steps.",
authorizations = {
@Authorization(value = "Only the user that submitted the request can remove it")
})
@ApiResponses(value = {
@ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
@ApiResponse(code = 401, message = "Client could not be authenticated."),
@ApiResponse(code = 403, message = "Client is not authorized to make this request."),
@ApiResponse(code = 404, message = "The specified resource could not be found."),
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
})
public Response deleteValidationRequest(
@ApiParam("The ID of the Controller Service") @PathParam("id") final String controllerServiceId,
@ApiParam("The ID of the Verification Request") @PathParam("requestId") final String requestId) {
if (isReplicateRequest()) {
return replicate(HttpMethod.DELETE);
}
final NiFiUser user = NiFiUserUtils.getNiFiUser();
final boolean twoPhaseRequest = isTwoPhaseRequest(httpServletRequest);
final boolean executionPhase = isExecutionPhase(httpServletRequest);
// If this is a standalone node, or if this is the execution phase of the request, perform the actual request.
if (!twoPhaseRequest || executionPhase) {
// request manager will ensure that the current is the user that submitted this request
final AsynchronousWebRequest<VerifyControllerServiceConfigRequestEntity, List<ConfigVerificationResultDTO>> asyncRequest =
updateRequestManager.removeRequest(VERIFICATION_REQUEST_TYPE, requestId, user);
if (asyncRequest == null) {
throw new ResourceNotFoundException("Could not find request of type " + VERIFICATION_REQUEST_TYPE + " with ID " + requestId);
}
if (!asyncRequest.isComplete()) {
asyncRequest.cancel();
}
final VerifyControllerServiceConfigRequestEntity updateRequestEntity = createVerifyControllerServiceConfigRequestEntity(asyncRequest, requestId);
return generateOkResponse(updateRequestEntity).build();
}
if (isValidationPhase(httpServletRequest)) {
// Perform authorization by attempting to get the request
updateRequestManager.getRequest(VERIFICATION_REQUEST_TYPE, requestId, user);
return generateContinueResponse().build();
} else if (isCancellationPhase(httpServletRequest)) {
return generateOkResponse().build();
} else {
throw new IllegalStateException("This request does not appear to be part of the two phase commit.");
}
}
public Response performAsyncConfigVerification(final VerifyControllerServiceConfigRequestEntity configRequest, final NiFiUser user) {
// Create an asynchronous request that will occur in the background, because this request may take an indeterminate amount of time.
final String requestId = generateUuid();
final VerifyControllerServiceConfigRequestDTO requestDto = configRequest.getRequest();
final String serviceId = requestDto.getControllerServiceId();
final List<UpdateStep> updateSteps = Collections.singletonList(new StandardUpdateStep("Verify Controller Service Configuration"));
final AsynchronousWebRequest<VerifyControllerServiceConfigRequestEntity, List<ConfigVerificationResultDTO>> request =
new StandardAsynchronousWebRequest<>(requestId, configRequest, serviceId, user, updateSteps);
// Submit the request to be performed in the background
final Consumer<AsynchronousWebRequest<VerifyControllerServiceConfigRequestEntity, List<ConfigVerificationResultDTO>>> updateTask = asyncRequest -> {
try {
final Map<String, String> attributes = requestDto.getAttributes() == null ? Collections.emptyMap() : requestDto.getAttributes();
final List<ConfigVerificationResultDTO> results = serviceFacade.verifyControllerServiceConfiguration(serviceId, requestDto.getControllerService(), attributes);
asyncRequest.markStepComplete(results);
} catch (final Exception e) {
logger.error("Failed to verify Controller Service configuration", e);
asyncRequest.fail("Failed to verify Controller Service configuration due to " + e);
}
};
updateRequestManager.submitRequest(VERIFICATION_REQUEST_TYPE, requestId, request, updateTask);
// Generate the response
final VerifyControllerServiceConfigRequestEntity resultsEntity = createVerifyControllerServiceConfigRequestEntity(request, requestId);
return generateOkResponse(resultsEntity).build();
}
private VerifyControllerServiceConfigRequestEntity createVerifyControllerServiceConfigRequestEntity(
final AsynchronousWebRequest<VerifyControllerServiceConfigRequestEntity, List<ConfigVerificationResultDTO>> asyncRequest, final String requestId) {
final VerifyControllerServiceConfigRequestDTO requestDto = asyncRequest.getRequest().getRequest();
final List<ConfigVerificationResultDTO> resultsList = asyncRequest.getResults();
final VerifyControllerServiceConfigRequestDTO dto = new VerifyControllerServiceConfigRequestDTO();
dto.setControllerServiceId(requestDto.getControllerServiceId());
dto.setControllerService(requestDto.getControllerService());
dto.setResults(resultsList);
dto.setComplete(resultsList != null);
dto.setFailureReason(asyncRequest.getFailureReason());
dto.setLastUpdated(asyncRequest.getLastUpdated());
dto.setPercentCompleted(asyncRequest.getPercentComplete());
dto.setRequestId(requestId);
dto.setState(asyncRequest.getState());
dto.setUri(generateResourceUri("controller-services", requestDto.getControllerServiceId(), "config", "verification-requests", requestId));
final VerifyControllerServiceConfigRequestEntity entity = new VerifyControllerServiceConfigRequestEntity();
entity.setRequest(dto);
return entity;
}
private ControllerServiceDTO createDTOWithDesiredRunStatus(final String id, final String runStatus) {
final ControllerServiceDTO dto = new ControllerServiceDTO();
dto.setId(id);

View File

@ -311,7 +311,7 @@ public class ParameterContextResource extends ApplicationResource {
value = "Initiate the Update Request of a Parameter Context",
response = ParameterContextUpdateRequestEntity.class,
notes = "This will initiate the process of updating a Parameter Context. Changing the value of a Parameter may require that one or more components be stopped and " +
"restarted, so this acttion may take significantly more time than many other REST API actions. As a result, this endpoint will immediately return a ParameterContextUpdateRequestEntity, " +
"restarted, so this action may take significantly more time than many other REST API actions. As a result, this endpoint will immediately return a ParameterContextUpdateRequestEntity, " +
"and the process of updating the necessary components will occur asynchronously in the background. The client may then periodically poll the status of the request by " +
"issuing a GET request to /parameter-contexts/update-requests/{requestId}. Once the request is completed, the client is expected to issue a DELETE request to " +
"/parameter-contexts/update-requests/{requestId}.",

View File

@ -37,21 +37,32 @@ import org.apache.nifi.ui.extension.UiExtensionMapping;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UiExtensionType;
import org.apache.nifi.web.api.concurrent.AsyncRequestManager;
import org.apache.nifi.web.api.concurrent.AsynchronousWebRequest;
import org.apache.nifi.web.api.concurrent.RequestManager;
import org.apache.nifi.web.api.concurrent.StandardAsynchronousWebRequest;
import org.apache.nifi.web.api.concurrent.StandardUpdateStep;
import org.apache.nifi.web.api.concurrent.UpdateStep;
import org.apache.nifi.web.api.dto.BundleDTO;
import org.apache.nifi.web.api.dto.ComponentStateDTO;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.dto.PositionDTO;
import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
import org.apache.nifi.web.api.dto.ProcessorDTO;
import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
import org.apache.nifi.web.api.dto.VerifyProcessorConfigRequestDTO;
import org.apache.nifi.web.api.entity.ComponentStateEntity;
import org.apache.nifi.web.api.entity.ProcessorDiagnosticsEntity;
import org.apache.nifi.web.api.entity.ProcessorEntity;
import org.apache.nifi.web.api.entity.ProcessorRunStatusEntity;
import org.apache.nifi.web.api.entity.ProcessorsRunStatusDetailsEntity;
import org.apache.nifi.web.api.entity.RunStatusDetailsRequestEntity;
import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
import org.apache.nifi.web.api.entity.RunStatusDetailsRequestEntity;
import org.apache.nifi.web.api.entity.VerifyProcessorConfigRequestEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.servlet.ServletContext;
import javax.servlet.http.HttpServletRequest;
@ -69,8 +80,12 @@ 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.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
/**
* RESTful endpoint for managing a Processor.
@ -81,6 +96,12 @@ import java.util.Set;
description = "Endpoint for managing a Processor."
)
public class ProcessorResource extends ApplicationResource {
private static final Logger logger = LoggerFactory.getLogger(ProcessorResource.class);
private static final String VERIFICATION_REQUEST_TYPE = "verification-request";
private RequestManager<VerifyProcessorConfigRequestEntity, List<ConfigVerificationResultDTO>> updateRequestManager =
new AsyncRequestManager<>(100, TimeUnit.MINUTES.toMillis(1L), "Verify Processor Config Thread");
private NiFiServiceFacade serviceFacade;
private Authorizer authorizer;
@ -525,6 +546,168 @@ public class ProcessorResource extends ApplicationResource {
);
}
@POST
@Consumes(MediaType.APPLICATION_JSON)
@Produces(MediaType.APPLICATION_JSON)
@Path("/{id}/config/verification-requests")
@ApiOperation(
value = "Performs verification of the Processor's configuration",
response = VerifyProcessorConfigRequestEntity.class,
notes = "This will initiate the process of verifying a given Processor configuration. This may be a long-running task. As a result, this endpoint will immediately return a " +
"ProcessorConfigVerificationRequestEntity, and the process of performing the verification will occur asynchronously in the background. " +
"The client may then periodically poll the status of the request by " +
"issuing a GET request to /processors/{processorId}/verification-requests/{requestId}. Once the request is completed, the client is expected to issue a DELETE request to " +
"/processors/{processorId}/verification-requests/{requestId}.",
authorizations = {
@Authorization(value = "Read - /processors/{uuid}")
}
)
@ApiResponses(
value = {
@ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
@ApiResponse(code = 401, message = "Client could not be authenticated."),
@ApiResponse(code = 403, message = "Client is not authorized to make this request."),
@ApiResponse(code = 404, message = "The specified resource could not be found."),
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
}
)
public Response submitProcessorVerificationRequest(
@ApiParam(value = "The processor id.", required = true) @PathParam("id") final String processorId,
@ApiParam(value = "The processor configuration verification request.", required = true) final VerifyProcessorConfigRequestEntity processorConfigRequest) {
if (processorConfigRequest == null) {
throw new IllegalArgumentException("Processor's configuration must be specified");
}
final VerifyProcessorConfigRequestDTO requestDto = processorConfigRequest.getRequest();
if (requestDto == null || requestDto.getProcessorConfig() == null) {
throw new IllegalArgumentException("Processor's configuration must be specified");
}
if (requestDto.getProcessorId() == null) {
throw new IllegalArgumentException("Processor's identifier must be specified in the request");
}
if (!requestDto.getProcessorId().equals(processorId)) {
throw new IllegalArgumentException("Processor's identifier in the request must match the identifier provided in the URL");
}
if (isReplicateRequest()) {
return replicate(HttpMethod.POST, processorConfigRequest);
}
final NiFiUser user = NiFiUserUtils.getNiFiUser();
return withWriteLock(
serviceFacade,
processorConfigRequest,
lookup -> {
final ComponentAuthorizable processor = lookup.getProcessor(processorId);
processor.getAuthorizable().authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
},
() -> {
serviceFacade.verifyCanVerifyProcessorConfig(processorId);
},
entity -> performAsyncConfigVerification(entity, user)
);
}
@GET
@Consumes(MediaType.WILDCARD)
@Produces(MediaType.APPLICATION_JSON)
@Path("{id}/config/verification-requests/{requestId}")
@ApiOperation(
value = "Returns the Verification Request with the given ID",
response = VerifyProcessorConfigRequestEntity.class,
notes = "Returns the Verification Request with the given ID. Once an Verification Request has been created, "
+ "that request can subsequently be retrieved via this endpoint, and the request that is fetched will contain the updated state, such as percent complete, the "
+ "current state of the request, and any failures. ",
authorizations = {
@Authorization(value = "Only the user that submitted the request can get it")
})
@ApiResponses(value = {
@ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
@ApiResponse(code = 401, message = "Client could not be authenticated."),
@ApiResponse(code = 403, message = "Client is not authorized to make this request."),
@ApiResponse(code = 404, message = "The specified resource could not be found."),
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
})
public Response getVerificationRequest(
@ApiParam("The ID of the Processor") @PathParam("id") final String processorId,
@ApiParam("The ID of the Verification Request") @PathParam("requestId") final String requestId) {
if (isReplicateRequest()) {
return replicate(HttpMethod.GET);
}
final NiFiUser user = NiFiUserUtils.getNiFiUser();
// request manager will ensure that the current is the user that submitted this request
final AsynchronousWebRequest<VerifyProcessorConfigRequestEntity, List<ConfigVerificationResultDTO>> asyncRequest = updateRequestManager.getRequest(VERIFICATION_REQUEST_TYPE, requestId, user);
final VerifyProcessorConfigRequestEntity updateRequestEntity = createVerifyProcessorConfigRequestEntity(asyncRequest, requestId);
return generateOkResponse(updateRequestEntity).build();
}
@DELETE
@Consumes(MediaType.WILDCARD)
@Produces(MediaType.APPLICATION_JSON)
@Path("{id}/config/verification-requests/{requestId}")
@ApiOperation(
value = "Deletes the Verification Request with the given ID",
response = VerifyProcessorConfigRequestEntity.class,
notes = "Deletes the Verification Request with the given ID. After a request is created, it is expected "
+ "that the client will properly clean up the request by DELETE'ing it, once the Verification process has completed. If the request is deleted before the request "
+ "completes, then the Verification request will finish the step that it is currently performing and then will cancel any subsequent steps.",
authorizations = {
@Authorization(value = "Only the user that submitted the request can remove it")
})
@ApiResponses(value = {
@ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
@ApiResponse(code = 401, message = "Client could not be authenticated."),
@ApiResponse(code = 403, message = "Client is not authorized to make this request."),
@ApiResponse(code = 404, message = "The specified resource could not be found."),
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
})
public Response deleteVerificationRequest(
@ApiParam("The ID of the Processor") @PathParam("id") final String processorId,
@ApiParam("The ID of the Verification Request") @PathParam("requestId") final String requestId) {
if (isReplicateRequest()) {
return replicate(HttpMethod.DELETE);
}
final NiFiUser user = NiFiUserUtils.getNiFiUser();
final boolean twoPhaseRequest = isTwoPhaseRequest(httpServletRequest);
final boolean executionPhase = isExecutionPhase(httpServletRequest);
// If this is a standalone node, or if this is the execution phase of the request, perform the actual request.
if (!twoPhaseRequest || executionPhase) {
// request manager will ensure that the current is the user that submitted this request
final AsynchronousWebRequest<VerifyProcessorConfigRequestEntity, List<ConfigVerificationResultDTO>> asyncRequest =
updateRequestManager.removeRequest(VERIFICATION_REQUEST_TYPE, requestId, user);
if (!asyncRequest.isComplete()) {
asyncRequest.cancel();
}
final VerifyProcessorConfigRequestEntity updateRequestEntity = createVerifyProcessorConfigRequestEntity(asyncRequest, requestId);
return generateOkResponse(updateRequestEntity).build();
}
if (isValidationPhase(httpServletRequest)) {
// Perform authorization by attempting to get the request
updateRequestManager.getRequest(VERIFICATION_REQUEST_TYPE, requestId, user);
return generateContinueResponse().build();
} else if (isCancellationPhase(httpServletRequest)) {
return generateOkResponse().build();
} else {
throw new IllegalStateException("This request does not appear to be part of the two phase commit.");
}
}
/**
* Updates the specified processor with the specified values.
*
@ -532,7 +715,6 @@ public class ProcessorResource extends ApplicationResource {
* @param id The id of the processor to update.
* @param requestProcessorEntity A processorEntity.
* @return A processorEntity.
* @throws InterruptedException if interrupted
*/
@PUT
@Consumes(MediaType.APPLICATION_JSON)
@ -725,7 +907,6 @@ public class ProcessorResource extends ApplicationResource {
* @param id The id of the processor to update.
* @param requestRunStatus A processorEntity.
* @return A processorEntity.
* @throws InterruptedException if interrupted
*/
@PUT
@Consumes(MediaType.APPLICATION_JSON)
@ -805,6 +986,61 @@ public class ProcessorResource extends ApplicationResource {
return dto;
}
public Response performAsyncConfigVerification(final VerifyProcessorConfigRequestEntity processorConfigRequest, final NiFiUser user) {
// Create an asynchronous request that will occur in the background, because this request may take an indeterminate amount of time.
final String requestId = generateUuid();
logger.debug("Generated Config Verification Request with ID {} for Processor {}", requestId, processorConfigRequest.getRequest().getProcessorId());
final VerifyProcessorConfigRequestDTO requestDto = processorConfigRequest.getRequest();
final String processorId = requestDto.getProcessorId();
final List<UpdateStep> updateSteps = Collections.singletonList(new StandardUpdateStep("Verify Processor Configuration"));
final AsynchronousWebRequest<VerifyProcessorConfigRequestEntity, List<ConfigVerificationResultDTO>> request =
new StandardAsynchronousWebRequest<>(requestId, processorConfigRequest, processorId, user, updateSteps);
// Submit the request to be performed in the background
final Consumer<AsynchronousWebRequest<VerifyProcessorConfigRequestEntity, List<ConfigVerificationResultDTO>>> updateTask = asyncRequest -> {
try {
final Map<String, String> attributes = requestDto.getAttributes() == null ? Collections.emptyMap() : requestDto.getAttributes();
final List<ConfigVerificationResultDTO> results = serviceFacade.verifyProcessorConfiguration(processorId, requestDto.getProcessorConfig(), attributes);
asyncRequest.markStepComplete(results);
} catch (final Exception e) {
logger.error("Failed to verify Processor configuration", e);
asyncRequest.fail("Failed to verify Processor configuration due to " + e);
}
};
updateRequestManager.submitRequest(VERIFICATION_REQUEST_TYPE, requestId, request, updateTask);
// Generate the response
final VerifyProcessorConfigRequestEntity resultsEntity = createVerifyProcessorConfigRequestEntity(request, requestId);
return generateOkResponse(resultsEntity).build();
}
private VerifyProcessorConfigRequestEntity createVerifyProcessorConfigRequestEntity(
final AsynchronousWebRequest<VerifyProcessorConfigRequestEntity, List<ConfigVerificationResultDTO>> asyncRequest,
final String requestId) {
final VerifyProcessorConfigRequestDTO requestDto = asyncRequest.getRequest().getRequest();
final List<ConfigVerificationResultDTO> resultsList = asyncRequest.getResults();
final VerifyProcessorConfigRequestDTO dto = new VerifyProcessorConfigRequestDTO();
dto.setProcessorId(requestDto.getProcessorId());
dto.setProcessorConfig(requestDto.getProcessorConfig());
dto.setResults(resultsList);
dto.setComplete(resultsList != null);
dto.setFailureReason(asyncRequest.getFailureReason());
dto.setLastUpdated(asyncRequest.getLastUpdated());
dto.setPercentCompleted(asyncRequest.getPercentComplete());
dto.setRequestId(requestId);
dto.setState(asyncRequest.getState());
dto.setUri(generateResourceUri("processors", requestDto.getProcessorId(), "config", "verification-requests", requestId));
final VerifyProcessorConfigRequestEntity entity = new VerifyProcessorConfigRequestEntity();
entity.setRequest(dto);
return entity;
}
// setters
public void setServiceFacade(NiFiServiceFacade serviceFacade) {

View File

@ -29,22 +29,35 @@ import org.apache.nifi.authorization.ComponentAuthorizable;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.authorization.resource.OperationAuthorizable;
import org.apache.nifi.authorization.user.NiFiUser;
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;
import org.apache.nifi.web.ResourceNotFoundException;
import org.apache.nifi.web.Revision;
import org.apache.nifi.web.UiExtensionType;
import org.apache.nifi.web.api.concurrent.AsyncRequestManager;
import org.apache.nifi.web.api.concurrent.AsynchronousWebRequest;
import org.apache.nifi.web.api.concurrent.RequestManager;
import org.apache.nifi.web.api.concurrent.StandardAsynchronousWebRequest;
import org.apache.nifi.web.api.concurrent.StandardUpdateStep;
import org.apache.nifi.web.api.concurrent.UpdateStep;
import org.apache.nifi.web.api.dto.BundleDTO;
import org.apache.nifi.web.api.dto.ComponentStateDTO;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
import org.apache.nifi.web.api.dto.ReportingTaskDTO;
import org.apache.nifi.web.api.dto.VerifyReportingTaskConfigRequestDTO;
import org.apache.nifi.web.api.entity.ComponentStateEntity;
import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
import org.apache.nifi.web.api.entity.ReportingTaskRunStatusEntity;
import org.apache.nifi.web.api.entity.VerifyReportingTaskConfigRequestEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.servlet.ServletContext;
import javax.servlet.http.HttpServletRequest;
@ -62,8 +75,11 @@ 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.Collections;
import java.util.List;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
/**
* RESTful endpoint for managing a Reporting Task.
@ -75,6 +91,11 @@ import java.util.Set;
)
public class ReportingTaskResource extends ApplicationResource {
private static final Logger logger = LoggerFactory.getLogger(ReportingTaskResource.class);
private static final String VERIFICATION_REQUEST_TYPE = "verification-request";
private RequestManager<VerifyReportingTaskConfigRequestEntity, List<ConfigVerificationResultDTO>> updateRequestManager =
new AsyncRequestManager<>(100, TimeUnit.MINUTES.toMillis(1L), "Verify Reporting Task Config Thread");
private NiFiServiceFacade serviceFacade;
private Authorizer authorizer;
@ -625,6 +646,228 @@ public class ReportingTaskResource extends ApplicationResource {
);
}
@POST
@Consumes(MediaType.APPLICATION_JSON)
@Produces(MediaType.APPLICATION_JSON)
@Path("/{id}/config/verification-requests")
@ApiOperation(
value = "Performs verification of the Reporting Task's configuration",
response = VerifyReportingTaskConfigRequestEntity.class,
notes = "This will initiate the process of verifying a given Reporting Task configuration. This may be a long-running task. As a result, this endpoint will immediately return a " +
"ReportingTaskConfigVerificationRequestEntity, and the process of performing the verification will occur asynchronously in the background. " +
"The client may then periodically poll the status of the request by " +
"issuing a GET request to /reporting-tasks/{serviceId}/verification-requests/{requestId}. Once the request is completed, the client is expected to issue a DELETE request to " +
"/reporting-tasks/{serviceId}/verification-requests/{requestId}.",
authorizations = {
@Authorization(value = "Read - /reporting-tasks/{uuid}")
}
)
@ApiResponses(
value = {
@ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
@ApiResponse(code = 401, message = "Client could not be authenticated."),
@ApiResponse(code = 403, message = "Client is not authorized to make this request."),
@ApiResponse(code = 404, message = "The specified resource could not be found."),
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
}
)
public Response submitConfigVerificationRequest(
@ApiParam(value = "The reporting task id.", required = true) @PathParam("id") final String reportingTaskId,
@ApiParam(value = "The reporting task configuration verification request.", required = true) final VerifyReportingTaskConfigRequestEntity reportingTaskConfigRequest) {
if (reportingTaskConfigRequest == null) {
throw new IllegalArgumentException("Reporting Task's configuration must be specified");
}
final VerifyReportingTaskConfigRequestDTO requestDto = reportingTaskConfigRequest.getRequest();
if (requestDto == null || requestDto.getReportingTask() == null) {
throw new IllegalArgumentException("Reporting Task must be specified");
}
if (requestDto.getReportingTaskId() == null) {
throw new IllegalArgumentException("Reporting Task's identifier must be specified in the request");
}
if (!requestDto.getReportingTaskId().equals(reportingTaskId)) {
throw new IllegalArgumentException("Reporting Task's identifier in the request must match the identifier provided in the URL");
}
if (isReplicateRequest()) {
return replicate(HttpMethod.POST, reportingTaskConfigRequest);
}
final NiFiUser user = NiFiUserUtils.getNiFiUser();
return withWriteLock(
serviceFacade,
reportingTaskConfigRequest,
lookup -> {
final ComponentAuthorizable reportingTask = lookup.getReportingTask(reportingTaskId);
reportingTask.getAuthorizable().authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
},
() -> {
serviceFacade.verifyCanVerifyReportingTaskConfig(reportingTaskId);
},
entity -> performAsyncConfigVerification(entity, user)
);
}
@GET
@Consumes(MediaType.WILDCARD)
@Produces(MediaType.APPLICATION_JSON)
@Path("{id}/config/verification-requests/{requestId}")
@ApiOperation(
value = "Returns the Verification Request with the given ID",
response = VerifyReportingTaskConfigRequestEntity.class,
notes = "Returns the Verification Request with the given ID. Once an Verification Request has been created, "
+ "that request can subsequently be retrieved via this endpoint, and the request that is fetched will contain the updated state, such as percent complete, the "
+ "current state of the request, and any failures. ",
authorizations = {
@Authorization(value = "Only the user that submitted the request can get it")
})
@ApiResponses(value = {
@ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
@ApiResponse(code = 401, message = "Client could not be authenticated."),
@ApiResponse(code = 403, message = "Client is not authorized to make this request."),
@ApiResponse(code = 404, message = "The specified resource could not be found."),
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
})
public Response getVerificationRequest(
@ApiParam("The ID of the Reporting Task") @PathParam("id") final String reportingTaskId,
@ApiParam("The ID of the Verification Request") @PathParam("requestId") final String requestId) {
if (isReplicateRequest()) {
return replicate(HttpMethod.GET);
}
final NiFiUser user = NiFiUserUtils.getNiFiUser();
// request manager will ensure that the current is the user that submitted this request
final AsynchronousWebRequest<VerifyReportingTaskConfigRequestEntity, List<ConfigVerificationResultDTO>> asyncRequest =
updateRequestManager.getRequest(VERIFICATION_REQUEST_TYPE, requestId, user);
final VerifyReportingTaskConfigRequestEntity updateRequestEntity = createVerifyReportingTaskConfigRequestEntity(asyncRequest, requestId);
return generateOkResponse(updateRequestEntity).build();
}
@DELETE
@Consumes(MediaType.WILDCARD)
@Produces(MediaType.APPLICATION_JSON)
@Path("{id}/config/verification-requests/{requestId}")
@ApiOperation(
value = "Deletes the Verification Request with the given ID",
response = VerifyReportingTaskConfigRequestEntity.class,
notes = "Deletes the Verification Request with the given ID. After a request is created, it is expected "
+ "that the client will properly clean up the request by DELETE'ing it, once the Verification process has completed. If the request is deleted before the request "
+ "completes, then the Verification request will finish the step that it is currently performing and then will cancel any subsequent steps.",
authorizations = {
@Authorization(value = "Only the user that submitted the request can remove it")
})
@ApiResponses(value = {
@ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
@ApiResponse(code = 401, message = "Client could not be authenticated."),
@ApiResponse(code = 403, message = "Client is not authorized to make this request."),
@ApiResponse(code = 404, message = "The specified resource could not be found."),
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
})
public Response deleteValidationRequest(
@ApiParam("The ID of the Reporting Task") @PathParam("id") final String reportingTaskId,
@ApiParam("The ID of the Verification Request") @PathParam("requestId") final String requestId) {
if (isReplicateRequest()) {
return replicate(HttpMethod.DELETE);
}
final NiFiUser user = NiFiUserUtils.getNiFiUser();
final boolean twoPhaseRequest = isTwoPhaseRequest(httpServletRequest);
final boolean executionPhase = isExecutionPhase(httpServletRequest);
// If this is a standalone node, or if this is the execution phase of the request, perform the actual request.
if (!twoPhaseRequest || executionPhase) {
// request manager will ensure that the current is the user that submitted this request
final AsynchronousWebRequest<VerifyReportingTaskConfigRequestEntity, List<ConfigVerificationResultDTO>> asyncRequest =
updateRequestManager.removeRequest(VERIFICATION_REQUEST_TYPE, requestId, user);
if (asyncRequest == null) {
throw new ResourceNotFoundException("Could not find request of type " + VERIFICATION_REQUEST_TYPE + " with ID " + requestId);
}
if (!asyncRequest.isComplete()) {
asyncRequest.cancel();
}
final VerifyReportingTaskConfigRequestEntity updateRequestEntity = createVerifyReportingTaskConfigRequestEntity(asyncRequest, requestId);
return generateOkResponse(updateRequestEntity).build();
}
if (isValidationPhase(httpServletRequest)) {
// Perform authorization by attempting to get the request
updateRequestManager.getRequest(VERIFICATION_REQUEST_TYPE, requestId, user);
return generateContinueResponse().build();
} else if (isCancellationPhase(httpServletRequest)) {
return generateOkResponse().build();
} else {
throw new IllegalStateException("This request does not appear to be part of the two phase commit.");
}
}
public Response performAsyncConfigVerification(final VerifyReportingTaskConfigRequestEntity configRequest, final NiFiUser user) {
// Create an asynchronous request that will occur in the background, because this request may take an indeterminate amount of time.
final String requestId = generateUuid();
final VerifyReportingTaskConfigRequestDTO requestDto = configRequest.getRequest();
final String taskId = requestDto.getReportingTaskId();
final List<UpdateStep> updateSteps = Collections.singletonList(new StandardUpdateStep("Verify Reporting Task Configuration"));
final AsynchronousWebRequest<VerifyReportingTaskConfigRequestEntity, List<ConfigVerificationResultDTO>> request =
new StandardAsynchronousWebRequest<>(requestId, configRequest, taskId, user, updateSteps);
// Submit the request to be performed in the background
final Consumer<AsynchronousWebRequest<VerifyReportingTaskConfigRequestEntity, List<ConfigVerificationResultDTO>>> updateTask = asyncRequest -> {
try {
final List<ConfigVerificationResultDTO> results = serviceFacade.verifyReportingTaskConfiguration(taskId, requestDto.getReportingTask());
asyncRequest.markStepComplete(results);
} catch (final Exception e) {
logger.error("Failed to verify Reporting Task configuration", e);
asyncRequest.fail("Failed to verify Reporting Task configuration due to " + e);
}
};
updateRequestManager.submitRequest(VERIFICATION_REQUEST_TYPE, requestId, request, updateTask);
// Generate the response
final VerifyReportingTaskConfigRequestEntity resultsEntity = createVerifyReportingTaskConfigRequestEntity(request, requestId);
return generateOkResponse(resultsEntity).build();
}
private VerifyReportingTaskConfigRequestEntity createVerifyReportingTaskConfigRequestEntity(
final AsynchronousWebRequest<VerifyReportingTaskConfigRequestEntity, List<ConfigVerificationResultDTO>> asyncRequest, final String requestId) {
final VerifyReportingTaskConfigRequestDTO requestDto = asyncRequest.getRequest().getRequest();
final List<ConfigVerificationResultDTO> resultsList = asyncRequest.getResults();
final VerifyReportingTaskConfigRequestDTO dto = new VerifyReportingTaskConfigRequestDTO();
dto.setReportingTaskId(requestDto.getReportingTaskId());
dto.setReportingTask(requestDto.getReportingTask());
dto.setResults(resultsList);
dto.setComplete(resultsList != null);
dto.setFailureReason(asyncRequest.getFailureReason());
dto.setLastUpdated(asyncRequest.getLastUpdated());
dto.setPercentCompleted(asyncRequest.getPercentComplete());
dto.setRequestId(requestId);
dto.setState(asyncRequest.getState());
dto.setUri(generateResourceUri("reporting-tasks", requestDto.getReportingTaskId(), "config", "verification-requests", requestId));
final VerifyReportingTaskConfigRequestEntity entity = new VerifyReportingTaskConfigRequestEntity();
entity.setRequest(dto);
return entity;
}
private ReportingTaskDTO createDTOWithDesiredRunStatus(final String id, final String runStatus) {
final ReportingTaskDTO dto = new ReportingTaskDTO();
dto.setId(id);

View File

@ -1679,6 +1679,7 @@ public final class DtoFactory {
dto.getProperties().put(descriptor.getName(), propertyValue);
}
dto.setReferencedAttributes(controllerServiceNode.getReferencedAttributeNames());
dto.setValidationStatus(controllerServiceNode.getValidationStatus(1, TimeUnit.MILLISECONDS).name());
// add the validation errors
@ -4013,6 +4014,7 @@ public final class DtoFactory {
dto.setSchedulingStrategy(procNode.getSchedulingStrategy().name());
dto.setExecutionNode(procNode.getExecutionNode().name());
dto.setAnnotationData(procNode.getAnnotationData());
dto.setReferencedAttributes(procNode.getReferencedAttributeNames());
// set up the default values for concurrent tasks and scheduling period
final Map<String, String> defaultConcurrentTasks = new HashMap<>();
@ -4143,6 +4145,7 @@ public final class DtoFactory {
copy.setParentGroupId(original.getParentGroupId());
copy.setName(original.getName());
copy.setProperties(copy(original.getProperties()));
copy.setReferencedAttributes(new HashSet<>(original.getReferencedAttributes()));
copy.setReferencingComponents(copy(original.getReferencingComponents()));
copy.setState(original.getState());
copy.setType(original.getType());
@ -4255,6 +4258,7 @@ public final class DtoFactory {
copy.setBulletinLevel(original.getBulletinLevel());
copy.setDefaultConcurrentTasks(original.getDefaultConcurrentTasks());
copy.setDefaultSchedulingPeriod(original.getDefaultSchedulingPeriod());
copy.setReferencedAttributes(original.getReferencedAttributes());
copy.setLossTolerant(original.isLossTolerant());
return copy;

View File

@ -22,8 +22,11 @@ import org.apache.nifi.controller.ComponentNode;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
import java.util.List;
import java.util.Map;
import java.util.Set;
public interface ControllerServiceDAO {
@ -128,6 +131,21 @@ public interface ControllerServiceDAO {
*/
void verifyClearState(String controllerServiceId);
/**
* Verifies the controller service is in a state in which its configuration can be verified
* @param controllerServiceId the id of the Controller Service
*/
void verifyConfigVerification(String controllerServiceId);
/**
* Performs verification of the Configuration for the Controller Service with the given ID
* @param controllerServiceId the id of the controller service
* @param controllerService the configuration to verify
* @param variables a map of values that can be used for resolving FlowFile attributes for Expression Language
* @return verification results
*/
List<ConfigVerificationResultDTO> verifyConfiguration(String controllerServiceId, ControllerServiceDTO controllerService, Map<String, String> variables);
/**
* Clears the state of the specified controller service.
*

View File

@ -19,8 +19,12 @@ package org.apache.nifi.web.dao;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.components.state.StateMap;
import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
import org.apache.nifi.web.api.dto.ProcessorDTO;
import java.util.List;
import java.util.Map;
import java.util.Set;
public interface ProcessorDAO {
@ -71,6 +75,12 @@ public interface ProcessorDAO {
*/
void verifyUpdate(ProcessorDTO processorDTO);
/**
* Verifies the specified processor can have its configuration verified
* @param processorId the ID of the Processor
*/
void verifyConfigVerification(String processorId);
/**
* Verifies that the specified processor can be terminated at this time
*
@ -93,6 +103,15 @@ public interface ProcessorDAO {
*/
ProcessorNode updateProcessor(ProcessorDTO processorDTO);
/**
* Performs verification of the given Processor Configuration for the Processor with the given ID
* @param processorId the id of the processor
* @param processorConfig the configuration to verify
* @param attributes a map of values that can be used for resolving FlowFile attributes for Expression Language
* @return verification results
*/
List<ConfigVerificationResultDTO> verifyProcessorConfiguration(String processorId, ProcessorConfigDTO processorConfig, Map<String, String> attributes);
/**
* Verifies the specified processor can be removed.
*

View File

@ -19,8 +19,10 @@ package org.apache.nifi.web.dao;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.components.state.StateMap;
import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.dto.ReportingTaskDTO;
import java.util.List;
import java.util.Set;
public interface ReportingTaskDAO {
@ -78,6 +80,20 @@ public interface ReportingTaskDAO {
*/
void verifyUpdate(ReportingTaskDTO reportingTaskDTO);
/**
* Verifies the Reporting Task is in a state in which its configuration can be verified
* @param reportingTaskId the id of the Reporting Task
*/
void verifyConfigVerification(String reportingTaskId);
/**
* Performs verification of the Configuration for the Reporting Task with the given ID
* @param reportingTaskId the id of the Reporting Task
* @param reportingTask the configuration to verify
* @return verification results
*/
List<ConfigVerificationResultDTO> verifyConfiguration(String reportingTaskId, ReportingTaskDTO reportingTask);
/**
* Determines whether this reporting task can be removed.
*

View File

@ -21,6 +21,7 @@ import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.components.state.StateMap;
import org.apache.nifi.controller.ComponentNode;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.exception.ControllerServiceInstantiationException;
@ -29,12 +30,20 @@ import org.apache.nifi.controller.flow.FlowManager;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.controller.service.StandardConfigurationContext;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.logging.LogRepository;
import org.apache.nifi.logging.repository.NopLogRepository;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.parameter.ParameterLookup;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.processor.SimpleProcessLogger;
import org.apache.nifi.util.BundleUtils;
import org.apache.nifi.web.NiFiCoreException;
import org.apache.nifi.web.ResourceNotFoundException;
import org.apache.nifi.web.api.dto.BundleDTO;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
import org.apache.nifi.web.dao.ComponentStateDAO;
import org.apache.nifi.web.dao.ControllerServiceDAO;
@ -45,6 +54,7 @@ import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
public class StandardControllerServiceDAO extends ComponentDAO implements ControllerServiceDAO {
@ -384,6 +394,40 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
controllerService.verifyCanClearState();
}
@Override
public void verifyConfigVerification(final String controllerServiceId) {
final ControllerServiceNode controllerService = locateControllerService(controllerServiceId);
controllerService.verifyCanPerformVerification();
}
@Override
public List<ConfigVerificationResultDTO> verifyConfiguration(final String controllerServiceId, final ControllerServiceDTO controllerService, final Map<String, String> variables) {
final ControllerServiceNode serviceNode = locateControllerService(controllerServiceId);
final LogRepository logRepository = new NopLogRepository();
final ComponentLog configVerificationLog = new SimpleProcessLogger(serviceNode.getControllerServiceImplementation(), logRepository);
final ExtensionManager extensionManager = flowController.getExtensionManager();
final ParameterLookup parameterLookup = serviceNode.getProcessGroup() == null ? ParameterLookup.EMPTY : serviceNode.getProcessGroup().getParameterContext();
final ConfigurationContext configurationContext = new StandardConfigurationContext(serviceNode, controllerService.getProperties(), controllerService.getAnnotationData(),
parameterLookup, flowController.getControllerServiceProvider(), null, flowController.getVariableRegistry());
final List<ConfigVerificationResult> verificationResults = serviceNode.verifyConfiguration(configurationContext, configVerificationLog, variables, extensionManager);
final List<ConfigVerificationResultDTO> resultsDtos = verificationResults.stream()
.map(this::createConfigVerificationResultDto)
.collect(Collectors.toList());
return resultsDtos;
}
private ConfigVerificationResultDTO createConfigVerificationResultDto(final ConfigVerificationResult result) {
final ConfigVerificationResultDTO dto = new ConfigVerificationResultDTO();
dto.setExplanation(result.getExplanation());
dto.setOutcome(result.getOutcome().name());
dto.setVerificationStepName(result.getVerificationStepName());
return dto;
}
@Override
public void clearState(final String controllerServiceId) {
final ControllerServiceNode controllerService = locateControllerService(controllerServiceId);

View File

@ -29,10 +29,18 @@ import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.exception.ComponentLifeCycleException;
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
import org.apache.nifi.controller.exception.ValidationException;
import org.apache.nifi.documentation.init.NopStateManager;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.logging.LogLevel;
import org.apache.nifi.logging.LogRepository;
import org.apache.nifi.logging.repository.NopLogRepository;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.SimpleProcessLogger;
import org.apache.nifi.processor.StandardProcessContext;
import org.apache.nifi.scheduling.ExecutionNode;
import org.apache.nifi.scheduling.SchedulingStrategy;
import org.apache.nifi.util.BundleUtils;
@ -40,6 +48,7 @@ import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.web.NiFiCoreException;
import org.apache.nifi.web.ResourceNotFoundException;
import org.apache.nifi.web.api.dto.BundleDTO;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
import org.apache.nifi.web.api.dto.ProcessorDTO;
import org.apache.nifi.web.dao.ComponentStateDAO;
@ -58,6 +67,7 @@ import java.util.Set;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.regex.Matcher;
import java.util.stream.Collectors;
public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO {
@ -341,6 +351,11 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO {
processor.getProcessGroup().terminateProcessor(processor);
}
@Override
public void verifyConfigVerification(final String processorId) {
final ProcessorNode processor = locateProcessor(processorId);
processor.verifyCanPerformVerification();
}
@Override
public void verifyUpdate(final ProcessorDTO processorDTO) {
@ -426,6 +441,34 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO {
}
}
@Override
public List<ConfigVerificationResultDTO> verifyProcessorConfiguration(final String processorId, final ProcessorConfigDTO processorConfig, final Map<String, String> attributes) {
final ProcessorNode processor = locateProcessor(processorId);
final ProcessContext processContext = new StandardProcessContext(processor, processorConfig.getProperties(), processorConfig.getAnnotationData(),
processor.getProcessGroup().getParameterContext(), flowController.getControllerServiceProvider(), flowController.getEncryptor(),
new NopStateManager(), () -> false, flowController);
final LogRepository logRepository = new NopLogRepository();
final ComponentLog configVerificationLog = new SimpleProcessLogger(processor, logRepository);
final ExtensionManager extensionManager = flowController.getExtensionManager();
final List<ConfigVerificationResult> verificationResults = processor.verifyConfiguration(processContext, configVerificationLog, attributes, extensionManager);
final List<ConfigVerificationResultDTO> resultsDtos = verificationResults.stream()
.map(this::createConfigVerificationResultDto)
.collect(Collectors.toList());
return resultsDtos;
}
private ConfigVerificationResultDTO createConfigVerificationResultDto(final ConfigVerificationResult result) {
final ConfigVerificationResultDTO dto = new ConfigVerificationResultDTO();
dto.setExplanation(result.getExplanation());
dto.setOutcome(result.getOutcome().name());
dto.setVerificationStepName(result.getVerificationStepName());
return dto;
}
@Override
public ProcessorNode updateProcessor(ProcessorDTO processorDTO) {
ProcessorNode processor = locateProcessor(processorDTO.getId());

View File

@ -21,6 +21,8 @@ import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.components.state.StateMap;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.ReloadComponent;
import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.ScheduledState;
@ -28,13 +30,21 @@ import org.apache.nifi.controller.exception.ComponentLifeCycleException;
import org.apache.nifi.controller.exception.ValidationException;
import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
import org.apache.nifi.controller.reporting.ReportingTaskProvider;
import org.apache.nifi.controller.service.StandardConfigurationContext;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.logging.LogRepository;
import org.apache.nifi.logging.repository.NopLogRepository;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.parameter.ParameterLookup;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.processor.SimpleProcessLogger;
import org.apache.nifi.scheduling.SchedulingStrategy;
import org.apache.nifi.util.BundleUtils;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.web.NiFiCoreException;
import org.apache.nifi.web.ResourceNotFoundException;
import org.apache.nifi.web.api.dto.BundleDTO;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.dto.ReportingTaskDTO;
import org.apache.nifi.web.dao.ComponentStateDAO;
import org.apache.nifi.web.dao.ReportingTaskDAO;
@ -48,12 +58,14 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.RejectedExecutionException;
import java.util.regex.Matcher;
import java.util.stream.Collectors;
public class StandardReportingTaskDAO extends ComponentDAO implements ReportingTaskDAO {
private ReportingTaskProvider reportingTaskProvider;
private ComponentStateDAO componentStateDAO;
private ReloadComponent reloadComponent;
private FlowController flowController;
private ReportingTaskNode locateReportingTask(final String reportingTaskId) {
// get the reporting task
@ -242,6 +254,41 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
verifyUpdate(reportingTask, reportingTaskDTO);
}
@Override
public void verifyConfigVerification(final String reportingTaskId) {
final ReportingTaskNode reportingTask = locateReportingTask(reportingTaskId);
reportingTask.verifyCanPerformVerification();
}
@Override
public List<ConfigVerificationResultDTO> verifyConfiguration(final String reportingTaskId, final ReportingTaskDTO reportingTask) {
final ReportingTaskNode taskNode = locateReportingTask(reportingTaskId);
final LogRepository logRepository = new NopLogRepository();
final ComponentLog configVerificationLog = new SimpleProcessLogger(taskNode.getReportingTask(), logRepository);
final ExtensionManager extensionManager = flowController.getExtensionManager();
final ParameterLookup parameterLookup = ParameterLookup.EMPTY;
final ConfigurationContext configurationContext = new StandardConfigurationContext(taskNode, reportingTask.getProperties(), reportingTask.getAnnotationData(),
parameterLookup, flowController.getControllerServiceProvider(), null, flowController.getVariableRegistry());
final List<ConfigVerificationResult> verificationResults = taskNode.verifyConfiguration(configurationContext, configVerificationLog, extensionManager);
final List<ConfigVerificationResultDTO> resultsDtos = verificationResults.stream()
.map(this::createConfigVerificationResultDto)
.collect(Collectors.toList());
return resultsDtos;
}
private ConfigVerificationResultDTO createConfigVerificationResultDto(final ConfigVerificationResult result) {
final ConfigVerificationResultDTO dto = new ConfigVerificationResultDTO();
dto.setExplanation(result.getExplanation());
dto.setOutcome(result.getOutcome().name());
dto.setVerificationStepName(result.getVerificationStepName());
return dto;
}
private void verifyUpdate(final ReportingTaskNode reportingTask, final ReportingTaskDTO reportingTaskDTO) {
// ensure the state, if specified, is valid
if (isNotNull(reportingTaskDTO.getState())) {
@ -380,4 +427,8 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
public void setReloadComponent(ReloadComponent reloadComponent) {
this.reloadComponent = reloadComponent;
}
public void setFlowController(FlowController flowController) {
this.flowController = flowController;
}
}

View File

@ -251,6 +251,7 @@
<property name="reportingTaskProvider" ref="reportingTaskProvider"/>
<property name="componentStateDAO" ref="componentStateDAO"/>
<property name="reloadComponent" ref="reloadComponent" />
<property name="flowController" ref="flowController" />
</bean>
<bean id="componentStateDAO" class="org.apache.nifi.web.dao.impl.StandardComponentStateDAO">
<property name="stateManagerProvider" ref="stateManagerProvider"/>

View File

@ -26,9 +26,20 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.controller.VerifiableControllerService;
import javax.jms.Connection;
import javax.jms.ConnectionFactory;
import javax.jms.ExceptionListener;
import javax.jms.JMSSecurityException;
import javax.jms.Session;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicReference;
/**
* Provides a factory service that creates and initializes
@ -50,7 +61,9 @@ import java.util.List;
+ "property and 'com.ibm.mq.jms.MQConnectionFactory.setTransportType(int)' would imply 'transportType' property.",
expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY)
@SeeAlso(classNames = {"org.apache.nifi.jms.processors.ConsumeJMS", "org.apache.nifi.jms.processors.PublishJMS"})
public class JMSConnectionFactoryProvider extends AbstractControllerService implements JMSConnectionFactoryProviderDefinition {
public class JMSConnectionFactoryProvider extends AbstractControllerService implements JMSConnectionFactoryProviderDefinition, VerifiableControllerService {
private static final String ESTABLISH_CONNECTION = "Establish Connection";
private static final String VERIFY_JMS_INTERACTION = "Verify JMS Interaction";
protected volatile JMSConnectionFactoryHandler delegate;
@ -84,4 +97,91 @@ public class JMSConnectionFactoryProvider extends AbstractControllerService impl
delegate.resetConnectionFactory(cachedFactory);
}
@Override
public List<ConfigVerificationResult> verify(final ConfigurationContext context, final ComponentLog verificationLogger, final Map<String, String> variables) {
final List<ConfigVerificationResult> results = new ArrayList<>();
final JMSConnectionFactoryHandler handler = new JMSConnectionFactoryHandler(context, verificationLogger);
final AtomicReference<Exception> failureReason = new AtomicReference<>();
final ExceptionListener listener = failureReason::set;
final Connection connection = createConnection(handler.getConnectionFactory(), results, listener, verificationLogger);
if (connection != null) {
try {
createSession(connection, results, failureReason.get(), verificationLogger);
} finally {
try {
connection.close();
} catch (final Exception ignored) {
}
}
}
return results;
}
private Connection createConnection(final ConnectionFactory connectionFactory, final List<ConfigVerificationResult> results, final ExceptionListener exceptionListener, final ComponentLog logger) {
try {
final Connection connection = connectionFactory.createConnection();
connection.setExceptionListener(exceptionListener);
results.add(new ConfigVerificationResult.Builder()
.verificationStepName(ESTABLISH_CONNECTION)
.outcome(Outcome.SUCCESSFUL)
.explanation("Successfully established a JMS Connection")
.build());
return connection;
} catch (final JMSSecurityException se) {
// If we encounter a JMS Security Exception, the documentation states that it is because of an invalid username or password.
// There is no username or password configured for the Controller Service itself, however. Those are configured in processors, etc.
// As a result, if this is encountered, we will skip verification.
logger.debug("Failed to establish a connection to the JMS Server in order to verify configuration because encountered JMS Security Exception", se);
results.add(new ConfigVerificationResult.Builder()
.verificationStepName(ESTABLISH_CONNECTION)
.outcome(Outcome.SKIPPED)
.explanation("Could not establish a Connection because doing so requires that a username and password be provided")
.build());
} catch (final Exception e) {
logger.warn("Failed to establish a connection to the JMS Server in order to verify configuration", e);
results.add(new ConfigVerificationResult.Builder()
.verificationStepName(ESTABLISH_CONNECTION)
.outcome(Outcome.FAILED)
.explanation("Was not able to establish a connection to the JMS Server: " + e.toString())
.build());
}
return null;
}
private void createSession(final Connection connection, final List<ConfigVerificationResult> results, final Exception capturedException, final ComponentLog logger) {
try {
final Session session = connection.createSession(false, Session.CLIENT_ACKNOWLEDGE);
session.close();
results.add(new ConfigVerificationResult.Builder()
.verificationStepName(VERIFY_JMS_INTERACTION)
.outcome(Outcome.SUCCESSFUL)
.explanation("Established a JMS Session with server and successfully terminated it")
.build());
} catch (final Exception e) {
final Exception failure;
if (capturedException == null) {
failure = e;
} else {
failure = capturedException;
failure.addSuppressed(e);
}
logger.warn("Failed to create a JMS Session in order to verify configuration", failure);
results.add(new ConfigVerificationResult.Builder()
.verificationStepName(VERIFY_JMS_INTERACTION)
.outcome(Outcome.FAILED)
.explanation("Was not able to create a JMS Session: " + failure.toString())
.build());
}
}
}

View File

@ -37,9 +37,11 @@ import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.VerifiableProcessor;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.serialization.RecordReaderFactory;
@ -84,7 +86,7 @@ import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_E
+ " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration.",
expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY)
@SeeAlso({ConsumeKafka_2_6.class, PublishKafka_2_6.class, PublishKafkaRecord_2_6.class})
public class ConsumeKafkaRecord_2_6 extends AbstractProcessor {
public class ConsumeKafkaRecord_2_6 extends AbstractProcessor implements VerifiableProcessor {
static final AllowableValue OFFSET_EARLIEST = new AllowableValue("earliest", "earliest", "Automatically reset the offset to the earliest offset");
static final AllowableValue OFFSET_LATEST = new AllowableValue("latest", "latest", "Automatically reset the offset to the latest offset");
@ -488,4 +490,11 @@ public class ConsumeKafkaRecord_2_6 extends AbstractProcessor {
}
}
}
@Override
public List<ConfigVerificationResult> verify(final ProcessContext context, final ComponentLog verificationLogger, final Map<String, String> attributes) {
try (final ConsumerPool consumerPool = createConsumerPool(context, verificationLogger)) {
return consumerPool.verifyConfiguration();
}
}
}

View File

@ -36,9 +36,11 @@ import org.apache.nifi.components.Validator;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.VerifiableProcessor;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
@ -77,7 +79,7 @@ import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_E
+ " In the event a dynamic property represents a property that was already set, its value will be ignored and WARN message logged."
+ " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration. ",
expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY)
public class ConsumeKafka_2_6 extends AbstractProcessor {
public class ConsumeKafka_2_6 extends AbstractProcessor implements VerifiableProcessor {
static final AllowableValue OFFSET_EARLIEST = new AllowableValue("earliest", "earliest", "Automatically reset the offset to the earliest offset");
@ -462,4 +464,11 @@ public class ConsumeKafka_2_6 extends AbstractProcessor {
}
}
}
@Override
public List<ConfigVerificationResult> verify(final ProcessContext context, final ComponentLog verificationLogger, final Map<String, String> attributes) {
try (final ConsumerPool consumerPool = createConsumerPool(context, verificationLogger)) {
return consumerPool.verifyConfiguration();
}
}
}

View File

@ -500,7 +500,7 @@ public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListe
session.adjustCounter("Parse Failures", 1, false);
}
private Map<String, String> getAttributes(final ConsumerRecord<?, ?> consumerRecord) {
protected Map<String, String> getAttributes(final ConsumerRecord<?, ?> consumerRecord) {
final Map<String, String> attributes = new HashMap<>();
if (headerNamePattern == null) {
return attributes;

View File

@ -17,28 +17,39 @@
package org.apache.nifi.processors.kafka.pubsub;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.RecordReaderFactory;
import org.apache.nifi.serialization.RecordSetWriterFactory;
import java.io.ByteArrayInputStream;
import java.io.Closeable;
import java.io.InputStream;
import java.nio.charset.Charset;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicLong;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
/**
* A pool of Kafka Consumers for a given topic. Consumers can be obtained by
@ -253,6 +264,168 @@ public class ConsumerPool implements Closeable {
return partitionsEachTopic;
}
public List<ConfigVerificationResult> verifyConfiguration() {
final List<ConfigVerificationResult> verificationResults = new ArrayList<>();
// Get a SimpleConsumerLease that we can use to communicate with Kafka
SimpleConsumerLease lease = pooledLeases.poll();
if (lease == null) {
lease = createConsumerLease();
if (lease == null) {
verificationResults.add(new ConfigVerificationResult.Builder()
.verificationStepName("Attempt connection")
.outcome(Outcome.FAILED)
.explanation("Could not obtain a Lease")
.build());
return verificationResults;
}
}
try {
final Consumer<byte[], byte[]> consumer = lease.consumer;
try {
consumer.groupMetadata();
} catch (final Exception e) {
logger.error("Failed to fetch Consumer Group Metadata in order to verify processor configuration", e);
verificationResults.add(new ConfigVerificationResult.Builder()
.verificationStepName("Attempt connection")
.outcome(Outcome.FAILED)
.explanation("Could not fetch Consumer Group Metadata: " + e)
.build());
}
try {
if (topicPattern == null) {
final Map<String, Long> messagesToConsumePerTopic = new HashMap<>();
for (final String topicName : topics) {
final List<PartitionInfo> partitionInfos = consumer.partitionsFor(topicName);
final Set<TopicPartition> topicPartitions = partitionInfos.stream()
.map(partitionInfo -> new TopicPartition(partitionInfo.topic(), partitionInfo.partition()))
.collect(Collectors.toSet());
final Map<TopicPartition, Long> endOffsets = consumer.endOffsets(topicPartitions, Duration.ofSeconds(30));
final Map<TopicPartition, Long> beginningOffsets = consumer.beginningOffsets(topicPartitions, Duration.ofSeconds(30));
final Map<TopicPartition, OffsetAndMetadata> committedOffsets = consumer.committed(topicPartitions, Duration.ofSeconds(30));
for (final TopicPartition topicPartition : endOffsets.keySet()) {
long endOffset = endOffsets.get(topicPartition);
// When no messages have been added to a topic, end offset is 0. However, after the first message is added,
// the end offset points to where the next message will be. I.e., it goes from 0 to 2. We want the offset
// of the last message, not the offset of where the next one will be. So we subtract one.
if (endOffset > 0) {
endOffset--;
}
final long beginningOffset = beginningOffsets.getOrDefault(topicPartition, 0L);
if (endOffset <= beginningOffset) {
messagesToConsumePerTopic.merge(topicPartition.topic(), 0L, Long::sum);
continue;
}
final OffsetAndMetadata offsetAndMetadata = committedOffsets.get(topicPartition);
final long committedOffset = offsetAndMetadata == null ? 0L : offsetAndMetadata.offset();
final long currentOffset = Math.max(beginningOffset, committedOffset);
final long messagesToConsume = endOffset - currentOffset;
messagesToConsumePerTopic.merge(topicPartition.topic(), messagesToConsume, Long::sum);
}
}
verificationResults.add(new ConfigVerificationResult.Builder()
.verificationStepName("Check Offsets")
.outcome(Outcome.SUCCESSFUL)
.explanation("Successfully determined offsets for " + messagesToConsumePerTopic.size() + " topics. Number of messages left to consume per topic: " + messagesToConsumePerTopic)
.build());
logger.info("Successfully determined offsets for {} topics. Number of messages left to consume per topic: {}", messagesToConsumePerTopic.size(), messagesToConsumePerTopic);
} else {
verificationResults.add(new ConfigVerificationResult.Builder()
.verificationStepName("Determine Topic Offsets")
.outcome(Outcome.SKIPPED)
.explanation("Cannot determine Topic Offsets because a Topic Wildcard was used instead of an explicit Topic Name")
.build());
}
} catch (final Exception e) {
logger.error("Failed to determine Topic Offsets in order to verify configuration", e);
verificationResults.add(new ConfigVerificationResult.Builder()
.verificationStepName("Determine Topic Offsets")
.outcome(Outcome.FAILED)
.explanation("Could not fetch Topic Offsets: " + e)
.build());
}
if (readerFactory != null) {
final ConfigVerificationResult checkDataResult = checkRecordIsParsable(lease);
verificationResults.add(checkDataResult);
}
return verificationResults;
} finally {
lease.close(true);
}
}
private ConfigVerificationResult checkRecordIsParsable(final SimpleConsumerLease consumerLease) {
final ConsumerRecords<byte[], byte[]> consumerRecords = consumerLease.consumer.poll(Duration.ofSeconds(30));
final Map<String, Integer> parseFailuresPerTopic = new HashMap<>();
final Map<String, String> latestParseFailureDescription = new HashMap<>();
final Map<String, Integer> recordsPerTopic = new HashMap<>();
for (final ConsumerRecord<byte[], byte[]> consumerRecord : consumerRecords) {
recordsPerTopic.merge(consumerRecord.topic(), 1, Integer::sum);
final Map<String, String> attributes = consumerLease.getAttributes(consumerRecord);
final byte[] recordBytes = consumerRecord.value() == null ? new byte[0] : consumerRecord.value();
try (final InputStream in = new ByteArrayInputStream(recordBytes)) {
final RecordReader reader = readerFactory.createRecordReader(attributes, in, recordBytes.length, logger);
while (reader.nextRecord() != null) {
}
} catch (final Exception e) {
parseFailuresPerTopic.merge(consumerRecord.topic(), 1, Integer::sum);
latestParseFailureDescription.put(consumerRecord.topic(), e.toString());
}
}
// Note here that we do not commit the offsets. We will just let the consumer close without committing the offsets, which
// will roll back the consumption of the messages.
if (recordsPerTopic.isEmpty()) {
return new ConfigVerificationResult.Builder()
.verificationStepName("Parse Records")
.outcome(Outcome.SKIPPED)
.explanation("Received no messages to attempt parsing within the 30 second timeout")
.build();
}
if (parseFailuresPerTopic.isEmpty()) {
return new ConfigVerificationResult.Builder()
.verificationStepName("Parse Records")
.outcome(Outcome.SUCCESSFUL)
.explanation("Was able to parse all Records consumed from topics. Number of Records consumed from each topic: " + recordsPerTopic)
.build();
} else {
final Map<String, String> failureDescriptions = new HashMap<>();
for (final String topic : recordsPerTopic.keySet()) {
final int records = recordsPerTopic.get(topic);
final Integer failures = parseFailuresPerTopic.get(topic);
final String failureReason = latestParseFailureDescription.get(topic);
final String description = "Failed to parse " + failures + " out of " + records + " records. Sample failure reason: " + failureReason;
failureDescriptions.put(topic, description);
}
return new ConfigVerificationResult.Builder()
.verificationStepName("Parse Records")
.outcome(Outcome.FAILED)
.explanation("With the configured Record Reader, failed to parse at least one Record. Failures per topic: " + failureDescriptions)
.build();
}
}
/**
* Obtains a consumer from the pool if one is available or lazily
* initializes a new one if deemed necessary.

View File

@ -35,11 +35,14 @@ import org.apache.nifi.components.PropertyDescriptor.Builder;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.VerifiableProcessor;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.processor.util.FlowFileFilters;
@ -95,7 +98,7 @@ import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.FAILUR
@WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Kafka for this FlowFile. This attribute is added only to "
+ "FlowFiles that are routed to success.")
@SeeAlso({PublishKafka_2_6.class, ConsumeKafka_2_6.class, ConsumeKafkaRecord_2_6.class})
public class PublishKafkaRecord_2_6 extends AbstractProcessor {
public class PublishKafkaRecord_2_6 extends AbstractProcessor implements VerifiableProcessor {
protected static final String MSG_COUNT = "msg.count";
static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("all", "Guarantee Replicated Delivery",
@ -570,4 +573,12 @@ public class PublishKafkaRecord_2_6 extends AbstractProcessor {
return (session, flowFiles) -> session.transfer(flowFiles, REL_FAILURE);
}
}
@Override
public List<ConfigVerificationResult> verify(final ProcessContext context, final ComponentLog verificationLogger, final Map<String, String> attributes) {
final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(attributes).getValue();
try (final PublisherPool pool = createPublisherPool(context)) {
return pool.verifyConfiguration(topic);
}
}
}

View File

@ -34,11 +34,14 @@ import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.VerifiableProcessor;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.processor.util.FlowFileFilters;
@ -81,7 +84,7 @@ import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.FAILUR
@WritesAttribute(attribute = "msg.count", description = "The number of messages that were sent to Kafka for this FlowFile. This attribute is added only to "
+ "FlowFiles that are routed to success. If the <Message Demarcator> Property is not set, this will always be 1, but if the Property is set, it may "
+ "be greater than 1.")
public class PublishKafka_2_6 extends AbstractProcessor {
public class PublishKafka_2_6 extends AbstractProcessor implements VerifiableProcessor {
protected static final String MSG_COUNT = "msg.count";
static final AllowableValue DELIVERY_REPLICATED = new AllowableValue("all", "Guarantee Replicated Delivery",
@ -531,4 +534,11 @@ public class PublishKafka_2_6 extends AbstractProcessor {
return null;
}
@Override
public List<ConfigVerificationResult> verify(final ProcessContext context, final ComponentLog verificationLogger, final Map<String, String> attributes) {
final String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(attributes).getValue();
try (final PublisherPool pool = createPublisherPool(context)) {
return pool.verifyConfiguration(topic);
}
}
}

View File

@ -21,9 +21,12 @@ import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.header.Headers;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.schema.access.SchemaNotFoundException;
import org.apache.nifi.serialization.RecordSetWriter;
import org.apache.nifi.serialization.RecordSetWriterFactory;
@ -41,7 +44,9 @@ import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
@ -308,4 +313,28 @@ public class PublisherLease implements Closeable {
return tracker;
}
public List<ConfigVerificationResult> verifyConfiguration(final String topic) {
final List<ConfigVerificationResult> verificationResults = new ArrayList<>();
try {
final List<PartitionInfo> partitionInfos = producer.partitionsFor(topic);
verificationResults.add(new ConfigVerificationResult.Builder()
.verificationStepName("Determine Topic Partitions")
.outcome(Outcome.SUCCESSFUL)
.explanation("Determined that there are " + partitionInfos.size() + " partitions for topic " + topic)
.build());
} catch (final Exception e) {
logger.error("Failed to determine Partition Information for Topic {} in order to verify configuration", topic, e);
verificationResults.add(new ConfigVerificationResult.Builder()
.verificationStepName("Determine Topic Partitions")
.outcome(Outcome.FAILED)
.explanation("Could not fetch Partition Information: " + e)
.build());
}
return verificationResults;
}
}

View File

@ -20,10 +20,12 @@ package org.apache.nifi.processors.kafka.pubsub;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.Producer;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.components.ConfigVerificationResult;
import java.io.Closeable;
import java.nio.charset.Charset;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
@ -113,4 +115,10 @@ public class PublisherPool implements Closeable {
protected int available() {
return publisherQueue.size();
}
public List<ConfigVerificationResult> verifyConfiguration(final String topic) {
try (final PublisherLease lease = obtainPublisher()) {
return lease.verifyConfiguration(topic);
}
}
}

View File

@ -16,27 +16,17 @@
*/
package org.apache.nifi.reporting;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.text.DateFormat;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.function.Supplier;
import javax.json.JsonArray;
import javax.json.JsonObjectBuilder;
import javax.json.JsonValue;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.remote.Transaction;
import org.apache.nifi.remote.TransferDirection;
import org.apache.nifi.remote.client.SiteToSiteClient;
import org.apache.nifi.reporting.s2s.SiteToSiteUtils;
import org.apache.nifi.schema.access.SchemaNotFoundException;
@ -65,10 +55,28 @@ import org.codehaus.jackson.JsonToken;
import org.codehaus.jackson.map.ObjectMapper;
import org.codehaus.jackson.node.ArrayNode;
import javax.json.JsonArray;
import javax.json.JsonObjectBuilder;
import javax.json.JsonValue;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.text.DateFormat;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.function.Supplier;
/**
* Base class for ReportingTasks that send data over site-to-site.
*/
public abstract class AbstractSiteToSiteReportingTask extends AbstractReportingTask {
public abstract class AbstractSiteToSiteReportingTask extends AbstractReportingTask implements VerifiableReportingTask {
private static final String ESTABLISH_COMMUNICATION = "Establish Site-to-Site Connection";
protected static final String LAST_EVENT_ID_KEY = "last_event_id";
protected static final String DESTINATION_URL_PATH = "/nifi";
@ -114,7 +122,7 @@ public abstract class AbstractSiteToSiteReportingTask extends AbstractReportingT
return properties;
}
public void setup(final ReportingContext reportContext) throws IOException {
public void setup(final PropertyContext reportContext) throws IOException {
if (siteToSiteClient == null) {
siteToSiteClient = SiteToSiteUtils.getClient(reportContext, getLogger(), null);
}
@ -507,4 +515,41 @@ public abstract class AbstractSiteToSiteReportingTask extends AbstractReportingT
}
}
@Override
public List<ConfigVerificationResult> verify(final ConfigurationContext context, final ComponentLog verificationLogger) {
final List<ConfigVerificationResult> verificationResults = new ArrayList<>();
try (final SiteToSiteClient client = SiteToSiteUtils.getClient(context, verificationLogger, null)) {
final Transaction transaction = client.createTransaction(TransferDirection.SEND);
// If transaction is null, indicates that all nodes are penalized
if (transaction == null) {
verificationResults.add(new ConfigVerificationResult.Builder()
.verificationStepName(ESTABLISH_COMMUNICATION)
.outcome(Outcome.SKIPPED)
.explanation("All nodes in destination NiFi are currently 'penalized', meaning that there have been recent failures communicating with the destination NiFi, or that" +
" the NiFi instance is applying backpressure")
.build());
} else {
transaction.cancel("Just verifying configuration");
verificationResults.add(new ConfigVerificationResult.Builder()
.verificationStepName(ESTABLISH_COMMUNICATION)
.outcome(Outcome.SUCCESSFUL)
.explanation("Established connection to destination NiFi instance and Received indication that it is ready to ready to receive data")
.build());
}
} catch (final Exception e) {
verificationLogger.error("Failed to establish site-to-site connection", e);
verificationResults.add(new ConfigVerificationResult.Builder()
.verificationStepName(ESTABLISH_COMMUNICATION)
.outcome(Outcome.FAILED)
.explanation("Failed to establish Site-to-Site Connection: " + e)
.build());
}
return verificationResults;
}
}

View File

@ -17,29 +17,11 @@
package org.apache.nifi.reporting;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.when;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import javax.json.Json;
import javax.json.JsonObject;
import javax.json.JsonReader;
import javax.json.JsonValue;
import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.remote.Transaction;
import org.apache.nifi.remote.TransferDirection;
@ -53,6 +35,24 @@ import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import javax.json.Json;
import javax.json.JsonObject;
import javax.json.JsonReader;
import javax.json.JsonValue;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.when;
public class TestSiteToSiteBulletinReportingTask {
@Test
@ -183,7 +183,7 @@ public class TestSiteToSiteBulletinReportingTask {
final List<byte[]> dataSent = new ArrayList<>();
@Override
public void setup(ReportingContext reportContext) throws IOException {
public void setup(PropertyContext reportContext) {
if(siteToSiteClient == null) {
final SiteToSiteClient client = Mockito.mock(SiteToSiteClient.class);
final Transaction transaction = Mockito.mock(Transaction.class);

View File

@ -17,33 +17,13 @@
package org.apache.nifi.reporting;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.when;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.UUID;
import javax.json.Json;
import javax.json.JsonArray;
import javax.json.JsonObject;
import javax.json.JsonReader;
import javax.json.JsonValue;
import org.apache.commons.lang3.SystemUtils;
import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.controller.status.ProcessGroupStatus;
import org.apache.nifi.controller.status.ProcessorStatus;
import org.apache.nifi.logging.ComponentLog;
@ -64,6 +44,26 @@ import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import javax.json.Json;
import javax.json.JsonArray;
import javax.json.JsonObject;
import javax.json.JsonReader;
import javax.json.JsonValue;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.UUID;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.when;
public class TestSiteToSiteMetricsReportingTask {
private ReportingContext context;
@ -304,7 +304,7 @@ public class TestSiteToSiteMetricsReportingTask {
final List<byte[]> dataSent = new ArrayList<>();
@Override
public void setup(ReportingContext reportContext) throws IOException {
public void setup(PropertyContext reportContext) {
if(siteToSiteClient == null) {
final SiteToSiteClient client = Mockito.mock(SiteToSiteClient.class);
final Transaction transaction = Mockito.mock(Transaction.class);

View File

@ -17,30 +17,12 @@
package org.apache.nifi.reporting;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.mockito.Mockito.when;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger;
import javax.json.Json;
import javax.json.JsonObject;
import javax.json.JsonReader;
import javax.json.JsonValue;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.controller.status.ConnectionStatus;
import org.apache.nifi.controller.status.ProcessGroupStatus;
@ -65,8 +47,24 @@ import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import javax.json.Json;
import javax.json.JsonObject;
import javax.json.JsonReader;
import javax.json.JsonValue;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.mockito.Mockito.when;
public class TestSiteToSiteProvenanceReportingTask {
@ -677,7 +675,7 @@ public class TestSiteToSiteProvenanceReportingTask {
final List<byte[]> dataSent = new ArrayList<>();
@Override
public void setup(ReportingContext reportContext) throws IOException {
public void setup(PropertyContext reportContext) {
if(siteToSiteClient == null) {
final SiteToSiteClient client = Mockito.mock(SiteToSiteClient.class);
final Transaction transaction = Mockito.mock(Transaction.class);

View File

@ -17,33 +17,9 @@
package org.apache.nifi.reporting;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.when;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import javax.json.Json;
import javax.json.JsonNumber;
import javax.json.JsonObject;
import javax.json.JsonReader;
import javax.json.JsonString;
import javax.json.JsonValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.controller.status.ConnectionStatus;
import org.apache.nifi.controller.status.PortStatus;
import org.apache.nifi.controller.status.ProcessGroupStatus;
@ -67,6 +43,29 @@ import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import javax.json.Json;
import javax.json.JsonNumber;
import javax.json.JsonObject;
import javax.json.JsonReader;
import javax.json.JsonString;
import javax.json.JsonValue;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.when;
public class TestSiteToSiteStatusReportingTask {
private ReportingContext context;
@ -538,7 +537,7 @@ public class TestSiteToSiteStatusReportingTask {
final List<byte[]> dataSent = new ArrayList<>();
@Override
public void setup(ReportingContext reportContext) throws IOException {
public void setup(PropertyContext reportContext) {
if(siteToSiteClient == null) {
final SiteToSiteClient client = Mockito.mock(SiteToSiteClient.class);
final Transaction transaction = Mockito.mock(Transaction.class);

View File

@ -262,7 +262,7 @@ public class StandardStatelessEngine implements StatelessEngine<VersionedFlowSna
final Map<PropertyDescriptor, PropertyConfiguration> fullPropertyMap = buildConfiguredAndDefaultPropertyMap(component, explicitlyConfiguredPropertyMap);
final ValidationContext validationContext = new StandardValidationContext(controllerServiceProvider, fullPropertyMap,
null, null, componentId, VariableRegistry.EMPTY_REGISTRY, null);
null, null, componentId, VariableRegistry.EMPTY_REGISTRY, null, true);
final Collection<ValidationResult> validationResults = component.validate(validationContext);
return validationResults.stream()

View File

@ -0,0 +1,141 @@
/*
* 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.cs.tests.system;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.controller.VerifiableControllerService;
import org.apache.nifi.logging.ComponentLog;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
import static org.apache.nifi.processor.util.StandardValidators.POSITIVE_INTEGER_VALIDATOR;
public class EnsureControllerServiceConfigurationCorrect extends AbstractControllerService implements VerifiableControllerService {
static final PropertyDescriptor SUCCESSFUL_VERIFICATION = new PropertyDescriptor.Builder()
.name("Successful Verification")
.displayName("Successful Verification")
.description("Whether or not Verification should succeed")
.required(true)
.allowableValues("true", "false")
.build();
static final PropertyDescriptor VERIFICATION_STEPS = new PropertyDescriptor.Builder()
.name("Verification Steps")
.displayName("Verification Steps")
.description("The number of steps to use in the Verification")
.required(true)
.addValidator(POSITIVE_INTEGER_VALIDATOR)
.expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
.defaultValue("1")
.build();
static final PropertyDescriptor EXCEPTION_ON_VERIFICATION = new PropertyDescriptor.Builder()
.name("Exception on Verification")
.displayName("Exception on Verification")
.description("If true, attempting to perform verification will throw a RuntimeException")
.required(true)
.allowableValues("true", "false")
.defaultValue("false")
.build();
static final PropertyDescriptor FAILURE_NODE_NUMBER = new PropertyDescriptor.Builder()
.name("Failure Node Number")
.displayName("Failure Node Number")
.description("The Node Number to Fail On")
.required(false)
.addValidator(POSITIVE_INTEGER_VALIDATOR)
.build();
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return Arrays.asList(SUCCESSFUL_VERIFICATION, VERIFICATION_STEPS, EXCEPTION_ON_VERIFICATION, FAILURE_NODE_NUMBER);
}
@Override
public List<ConfigVerificationResult> verify(final ConfigurationContext context, final ComponentLog verificationLogger, final Map<String, String> attributes) {
final boolean exception = context.getProperty(EXCEPTION_ON_VERIFICATION).asBoolean();
if (exception) {
throw new RuntimeException("Intentional Exception - Processor was configured to throw an Exception when performing config verification");
}
final List<ConfigVerificationResult> results = new ArrayList<>();
final int iterations;
try {
iterations = context.getProperty(VERIFICATION_STEPS).evaluateAttributeExpressions(attributes).asInteger();
} catch (final NumberFormatException nfe) {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Determine Number of Verification Steps")
.outcome(Outcome.FAILED)
.explanation("Invalid value for the number of Verification Steps")
.build());
return results;
}
final boolean success = context.getProperty(SUCCESSFUL_VERIFICATION).asBoolean();
final Outcome outcome = success ? Outcome.SUCCESSFUL : Outcome.FAILED;
for (int i=0; i < iterations; i++) {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Verification Step #" + i)
.outcome(outcome)
.explanation("Verification Step #" + i)
.build());
}
// Consider the 'Failure Node Number' Property. This makes it easy to get different results from different nodes for testing purposes
final Integer failureNodeNum = context.getProperty(FAILURE_NODE_NUMBER).asInteger();
if (failureNodeNum == null) {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Fail Based on Node Number")
.outcome(Outcome.SKIPPED)
.explanation("Not configured to Fail based on node number")
.build());
} else {
final String currentNodeNumberString = System.getProperty("nodeNumber");
final Integer currentNodeNumber = currentNodeNumberString == null ? null : Integer.parseInt(currentNodeNumberString);
final boolean shouldFail = Objects.equals(failureNodeNum, currentNodeNumber);
if (shouldFail) {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Fail Based on Node Number")
.outcome(Outcome.FAILED)
.explanation("This node is Node Number " + currentNodeNumberString + " and configured to fail on this Node Number")
.build());
} else {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Fail Based on Node Number")
.outcome(Outcome.SUCCESSFUL)
.explanation("This node is Node Number " + currentNodeNumberString + " and configured not to fail on this Node Number")
.build());
}
}
return results;
}
}

View File

@ -0,0 +1,149 @@
/*
* 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.processors.tests.system;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyDescriptor.Builder;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.VerifiableProcessor;
import org.apache.nifi.processor.exception.ProcessException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
import static org.apache.nifi.processor.util.StandardValidators.POSITIVE_INTEGER_VALIDATOR;
public class EnsureProcessorConfigurationCorrect extends AbstractProcessor implements VerifiableProcessor {
static final PropertyDescriptor SUCCESSFUL_VERIFICATION = new Builder()
.name("Successful Verification")
.displayName("Successful Verification")
.description("Whether or not Verification should succeed")
.required(true)
.allowableValues("true", "false")
.build();
static final PropertyDescriptor VERIFICATION_STEPS = new Builder()
.name("Verification Steps")
.displayName("Verification Steps")
.description("The number of steps to use in the Verification")
.required(true)
.addValidator(POSITIVE_INTEGER_VALIDATOR)
.expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
.defaultValue("1")
.build();
static final PropertyDescriptor EXCEPTION_ON_VERIFICATION = new Builder()
.name("Exception on Verification")
.displayName("Exception on Verification")
.description("If true, attempting to perform verification will throw a RuntimeException")
.required(true)
.allowableValues("true", "false")
.defaultValue("false")
.build();
static final PropertyDescriptor FAILURE_NODE_NUMBER = new Builder()
.name("Failure Node Number")
.displayName("Failure Node Number")
.description("The Node Number to Fail On")
.required(false)
.addValidator(POSITIVE_INTEGER_VALIDATOR)
.build();
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return Arrays.asList(SUCCESSFUL_VERIFICATION, VERIFICATION_STEPS, EXCEPTION_ON_VERIFICATION, FAILURE_NODE_NUMBER);
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
}
@Override
public List<ConfigVerificationResult> verify(final ProcessContext context, final ComponentLog verificationLogger, final Map<String, String> attributes) {
final boolean exception = context.getProperty(EXCEPTION_ON_VERIFICATION).asBoolean();
if (exception) {
throw new RuntimeException("Intentional Exception - Processor was configured to throw an Exception when performing config verification");
}
final List<ConfigVerificationResult> results = new ArrayList<>();
final int iterations;
try {
iterations = context.getProperty(VERIFICATION_STEPS).evaluateAttributeExpressions(attributes).asInteger();
} catch (final NumberFormatException nfe) {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Determine Number of Verification Steps")
.outcome(Outcome.FAILED)
.explanation("Invalid value for the number of Verification Steps")
.build());
return results;
}
final boolean success = context.getProperty(SUCCESSFUL_VERIFICATION).asBoolean();
final Outcome outcome = success ? Outcome.SUCCESSFUL : Outcome.FAILED;
for (int i=0; i < iterations; i++) {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Verification Step #" + i)
.outcome(outcome)
.explanation("Verification Step #" + i)
.build());
}
// Consider the 'Failure Node Number' Property. This makes it easy to get different results from different nodes for testing purposes
final Integer failureNodeNum = context.getProperty(FAILURE_NODE_NUMBER).asInteger();
if (failureNodeNum == null) {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Fail Based on Node Number")
.outcome(Outcome.SKIPPED)
.explanation("Not configured to Fail based on node number")
.build());
} else {
final String currentNodeNumberString = System.getProperty("nodeNumber");
final Integer currentNodeNumber = currentNodeNumberString == null ? null : Integer.parseInt(currentNodeNumberString);
final boolean shouldFail = Objects.equals(failureNodeNum, currentNodeNumber);
if (shouldFail) {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Fail Based on Node Number")
.outcome(Outcome.FAILED)
.explanation("This node is Node Number " + currentNodeNumberString + " and configured to fail on this Node Number")
.build());
} else {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Fail Based on Node Number")
.outcome(Outcome.SUCCESSFUL)
.explanation("This node is Node Number " + currentNodeNumberString + " and configured not to fail on this Node Number")
.build());
}
}
return results;
}
}

View File

@ -0,0 +1,143 @@
/*
* 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.reporting;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.logging.ComponentLog;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
import static org.apache.nifi.processor.util.StandardValidators.POSITIVE_INTEGER_VALIDATOR;
public class EnsureReportingTaskConfigurationCorrect extends AbstractReportingTask implements VerifiableReportingTask {
static final PropertyDescriptor SUCCESSFUL_VERIFICATION = new PropertyDescriptor.Builder()
.name("Successful Verification")
.displayName("Successful Verification")
.description("Whether or not Verification should succeed")
.required(true)
.allowableValues("true", "false")
.build();
static final PropertyDescriptor VERIFICATION_STEPS = new PropertyDescriptor.Builder()
.name("Verification Steps")
.displayName("Verification Steps")
.description("The number of steps to use in the Verification")
.required(true)
.addValidator(POSITIVE_INTEGER_VALIDATOR)
.expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
.defaultValue("1")
.build();
static final PropertyDescriptor EXCEPTION_ON_VERIFICATION = new PropertyDescriptor.Builder()
.name("Exception on Verification")
.displayName("Exception on Verification")
.description("If true, attempting to perform verification will throw a RuntimeException")
.required(true)
.allowableValues("true", "false")
.defaultValue("false")
.build();
static final PropertyDescriptor FAILURE_NODE_NUMBER = new PropertyDescriptor.Builder()
.name("Failure Node Number")
.displayName("Failure Node Number")
.description("The Node Number to Fail On")
.required(false)
.addValidator(POSITIVE_INTEGER_VALIDATOR)
.build();
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return Arrays.asList(SUCCESSFUL_VERIFICATION, VERIFICATION_STEPS, EXCEPTION_ON_VERIFICATION, FAILURE_NODE_NUMBER);
}
@Override
public void onTrigger(final ReportingContext context) {
}
@Override
public List<ConfigVerificationResult> verify(final ConfigurationContext context, final ComponentLog verificationLogger) {
final boolean exception = context.getProperty(EXCEPTION_ON_VERIFICATION).asBoolean();
if (exception) {
throw new RuntimeException("Intentional Exception - Processor was configured to throw an Exception when performing config verification");
}
final List<ConfigVerificationResult> results = new ArrayList<>();
final int iterations;
try {
iterations = context.getProperty(VERIFICATION_STEPS).asInteger();
} catch (final NumberFormatException nfe) {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Determine Number of Verification Steps")
.outcome(Outcome.FAILED)
.explanation("Invalid value for the number of Verification Steps")
.build());
return results;
}
final boolean success = context.getProperty(SUCCESSFUL_VERIFICATION).asBoolean();
final Outcome outcome = success ? Outcome.SUCCESSFUL : Outcome.FAILED;
for (int i=0; i < iterations; i++) {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Verification Step #" + i)
.outcome(outcome)
.explanation("Verification Step #" + i)
.build());
}
// Consider the 'Failure Node Number' Property. This makes it easy to get different results from different nodes for testing purposes
final Integer failureNodeNum = context.getProperty(FAILURE_NODE_NUMBER).asInteger();
if (failureNodeNum == null) {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Fail Based on Node Number")
.outcome(Outcome.SKIPPED)
.explanation("Not configured to Fail based on node number")
.build());
} else {
final String currentNodeNumberString = System.getProperty("nodeNumber");
final Integer currentNodeNumber = currentNodeNumberString == null ? null : Integer.parseInt(currentNodeNumberString);
final boolean shouldFail = Objects.equals(failureNodeNum, currentNodeNumber);
if (shouldFail) {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Fail Based on Node Number")
.outcome(Outcome.FAILED)
.explanation("This node is Node Number " + currentNodeNumberString + " and configured to fail on this Node Number")
.build());
} else {
results.add(new ConfigVerificationResult.Builder()
.verificationStepName("Fail Based on Node Number")
.outcome(Outcome.SUCCESSFUL)
.explanation("This node is Node Number " + currentNodeNumberString + " and configured not to fail on this Node Number")
.build());
}
}
return results;
}
}

View File

@ -13,6 +13,7 @@
# See the License for the specific language governing permissions and
# limitations under the License.
org.apache.nifi.cs.tests.system.EnsureControllerServiceConfigurationCorrect
org.apache.nifi.cs.tests.system.FakeControllerService1
org.apache.nifi.cs.tests.system.StandardCountService
org.apache.nifi.cs.tests.system.StandardSleepService

View File

@ -19,6 +19,7 @@ org.apache.nifi.processors.tests.system.ConcatenateFlowFiles
org.apache.nifi.processors.tests.system.DependOnProperties
org.apache.nifi.processors.tests.system.DoNotTransferFlowFile
org.apache.nifi.processors.tests.system.Duplicate
org.apache.nifi.processors.tests.system.EnsureProcessorConfigurationCorrect
org.apache.nifi.processors.tests.system.EvaluatePropertiesWithDifferentELScopes
org.apache.nifi.processors.tests.system.FakeProcessor
org.apache.nifi.processors.tests.system.FakeDynamicPropertiesProcessor

View File

@ -13,4 +13,5 @@
# See the License for the specific language governing permissions and
# limitations under the License.
org.apache.nifi.reporting.EnsureReportingTaskConfigurationCorrect
org.apache.nifi.reporting.WriteToFileReportingTask

View File

@ -29,6 +29,7 @@ import org.apache.nifi.toolkit.cli.impl.client.nifi.ConnectionClient;
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClient;
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
import org.apache.nifi.web.api.dto.BundleDTO;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.dto.ConnectableDTO;
import org.apache.nifi.web.api.dto.ConnectionDTO;
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
@ -45,9 +46,13 @@ import org.apache.nifi.web.api.dto.ProcessGroupDTO;
import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
import org.apache.nifi.web.api.dto.ProcessorDTO;
import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
import org.apache.nifi.web.api.dto.ReportingTaskDTO;
import org.apache.nifi.web.api.dto.RevisionDTO;
import org.apache.nifi.web.api.dto.VariableDTO;
import org.apache.nifi.web.api.dto.VariableRegistryDTO;
import org.apache.nifi.web.api.dto.VerifyControllerServiceConfigRequestDTO;
import org.apache.nifi.web.api.dto.VerifyProcessorConfigRequestDTO;
import org.apache.nifi.web.api.dto.VerifyReportingTaskConfigRequestDTO;
import org.apache.nifi.web.api.dto.flow.FlowDTO;
import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO;
import org.apache.nifi.web.api.dto.provenance.ProvenanceDTO;
@ -75,10 +80,14 @@ import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity;
import org.apache.nifi.web.api.entity.ProcessorEntity;
import org.apache.nifi.web.api.entity.ProvenanceEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
import org.apache.nifi.web.api.entity.ScheduleComponentsEntity;
import org.apache.nifi.web.api.entity.VariableEntity;
import org.apache.nifi.web.api.entity.VariableRegistryEntity;
import org.apache.nifi.web.api.entity.VariableRegistryUpdateRequestEntity;
import org.apache.nifi.web.api.entity.VerifyControllerServiceConfigRequestEntity;
import org.apache.nifi.web.api.entity.VerifyProcessorConfigRequestEntity;
import org.apache.nifi.web.api.entity.VerifyReportingTaskConfigRequestEntity;
import org.junit.Assert;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -345,7 +354,11 @@ public class NiFiClientUtil {
while (true) {
final ProcessorEntity entity = nifiClient.getProcessorClient().getProcessor(processorId);
final String state = entity.getComponent().getState();
if (!expectedState.equals(state)) {
// We've reached the desired state if the state equal the expected state, OR if we expect stopped and the state is disabled (because disabled implies stopped)
final boolean desiredStateReached = expectedState.equals(state) || ("STOPPED".equalsIgnoreCase(expectedState) && "DISABLED".equalsIgnoreCase(state));
if (!desiredStateReached) {
Thread.sleep(10L);
continue;
}
@ -362,6 +375,19 @@ public class NiFiClientUtil {
}
}
public ControllerServiceEntity updateControllerService(final ControllerServiceEntity currentEntity, final Map<String, String> properties) throws NiFiClientException, IOException {
final ControllerServiceDTO dto = new ControllerServiceDTO();
dto.setProperties(properties);
dto.setId(currentEntity.getId());
final ControllerServiceEntity entity = new ControllerServiceEntity();
entity.setId(currentEntity.getId());
entity.setComponent(dto);
entity.setRevision(currentEntity.getRevision());
return nifiClient.getControllerServicesClient().updateControllerService(entity);
}
public ControllerServiceEntity enableControllerService(final ControllerServiceEntity entity) throws NiFiClientException, IOException {
final ControllerServiceRunStatusEntity runStatusEntity = new ControllerServiceRunStatusEntity();
runStatusEntity.setState("ENABLED");
@ -953,4 +979,123 @@ public class NiFiClientUtil {
return current;
}
public List<ConfigVerificationResultDTO> verifyProcessorConfig(final String processorId, final Map<String, String> properties) throws InterruptedException, IOException, NiFiClientException {
return verifyProcessorConfig(processorId, properties, Collections.emptyMap());
}
public List<ConfigVerificationResultDTO> verifyProcessorConfig(final String processorId, final Map<String, String> properties, final Map<String, String> attributes)
throws NiFiClientException, IOException, InterruptedException {
final ProcessorConfigDTO processorConfig = new ProcessorConfigDTO();
processorConfig.setProperties(properties);
final VerifyProcessorConfigRequestDTO requestDto = new VerifyProcessorConfigRequestDTO();
requestDto.setProcessorId(processorId);
requestDto.setProcessorConfig(processorConfig);
requestDto.setAttributes(attributes);
final VerifyProcessorConfigRequestEntity verificationRequest = new VerifyProcessorConfigRequestEntity();
verificationRequest.setRequest(requestDto);
VerifyProcessorConfigRequestEntity results = nifiClient.getProcessorClient().submitConfigVerificationRequest(verificationRequest);
while (!results.getRequest().isComplete()) {
Thread.sleep(50L);
results = nifiClient.getProcessorClient().getConfigVerificationRequest(processorId, results.getRequest().getRequestId());
}
nifiClient.getProcessorClient().deleteConfigVerificationRequest(processorId, results.getRequest().getRequestId());
return results.getRequest().getResults();
}
public List<ConfigVerificationResultDTO> verifyControllerServiceConfig(final String serviceId, final Map<String, String> properties)
throws InterruptedException, IOException,NiFiClientException {
return verifyControllerServiceConfig(serviceId, properties, Collections.emptyMap());
}
public List<ConfigVerificationResultDTO> verifyControllerServiceConfig(final String serviceId, final Map<String, String> properties, final Map<String, String> attributes)
throws NiFiClientException, IOException, InterruptedException {
final ControllerServiceDTO serviceDto = new ControllerServiceDTO();
serviceDto.setProperties(properties);
serviceDto.setId(serviceId);
final VerifyControllerServiceConfigRequestDTO requestDto = new VerifyControllerServiceConfigRequestDTO();
requestDto.setControllerService(serviceDto);
requestDto.setAttributes(attributes);
requestDto.setControllerServiceId(serviceId);
final VerifyControllerServiceConfigRequestEntity verificationRequest = new VerifyControllerServiceConfigRequestEntity();
verificationRequest.setRequest(requestDto);
VerifyControllerServiceConfigRequestEntity results = nifiClient.getControllerServicesClient().submitConfigVerificationRequest(verificationRequest);
while (!results.getRequest().isComplete()) {
Thread.sleep(50L);
results = nifiClient.getControllerServicesClient().getConfigVerificationRequest(serviceId, results.getRequest().getRequestId());
}
nifiClient.getControllerServicesClient().deleteConfigVerificationRequest(serviceId, results.getRequest().getRequestId());
return results.getRequest().getResults();
}
public List<ConfigVerificationResultDTO> verifyReportingTaskConfig(final String taskId, final Map<String, String> properties)
throws InterruptedException, IOException,NiFiClientException {
final ReportingTaskDTO taskDto = new ReportingTaskDTO();
taskDto.setProperties(properties);
taskDto.setId(taskId);
final VerifyReportingTaskConfigRequestDTO requestDto = new VerifyReportingTaskConfigRequestDTO();
requestDto.setReportingTaskId(taskId);
requestDto.setReportingTask(taskDto);
final VerifyReportingTaskConfigRequestEntity verificationRequest = new VerifyReportingTaskConfigRequestEntity();
verificationRequest.setRequest(requestDto);
VerifyReportingTaskConfigRequestEntity results = nifiClient.getReportingTasksClient().submitConfigVerificationRequest(verificationRequest);
while (!results.getRequest().isComplete()) {
Thread.sleep(50L);
results = nifiClient.getReportingTasksClient().getConfigVerificationRequest(taskId, results.getRequest().getRequestId());
}
nifiClient.getReportingTasksClient().deleteConfigVerificationRequest(taskId, results.getRequest().getRequestId());
return results.getRequest().getResults();
}
public ReportingTaskEntity createReportingTask(final String simpleTypeName) throws NiFiClientException, IOException {
return createReportingTask(NiFiSystemIT.TEST_REPORTING_TASK_PACKAGE + "." + simpleTypeName, NiFiSystemIT.NIFI_GROUP_ID, NiFiSystemIT.TEST_EXTENSIONS_ARTIFACT_ID, nifiVersion);
}
public ReportingTaskEntity createReportingTask(final String type, final String bundleGroupId, final String artifactId, final String version)
throws NiFiClientException, IOException {
final ReportingTaskDTO dto = new ReportingTaskDTO();
dto.setType(type);
final BundleDTO bundle = new BundleDTO();
bundle.setGroup(bundleGroupId);
bundle.setArtifact(artifactId);
bundle.setVersion(version);
dto.setBundle(bundle);
final ReportingTaskEntity entity = new ReportingTaskEntity();
entity.setComponent(dto);
entity.setRevision(createNewRevision());
return nifiClient.getControllerClient().createReportingTask(entity);
}
public ReportingTaskEntity updateReportingTask(final ReportingTaskEntity currentEntity, final Map<String, String> properties) throws NiFiClientException, IOException {
final ReportingTaskDTO dto = new ReportingTaskDTO();
dto.setProperties(properties);
dto.setId(currentEntity.getId());
final ReportingTaskEntity entity = new ReportingTaskEntity();
entity.setId(currentEntity.getId());
entity.setComponent(dto);
entity.setRevision(currentEntity.getRevision());
return nifiClient.getReportingTasksClient().updateReportingTask(entity);
}
}

View File

@ -54,6 +54,7 @@ public abstract class NiFiSystemIT {
public static final String TEST_EXTENSIONS_ARTIFACT_ID = "nifi-system-test-extensions-nar";
public static final String TEST_PROCESSORS_PACKAGE = "org.apache.nifi.processors.tests.system";
public static final String TEST_CS_PACKAGE = "org.apache.nifi.cs.tests.system";
public static final String TEST_REPORTING_TASK_PACKAGE = "org.apache.nifi.reporting";
private static final Pattern FRAMEWORK_NAR_PATTERN = Pattern.compile("nifi-framework-nar-(.*?)\\.nar");
private static final File LIB_DIR = new File("target/nifi-lib-assembly/lib");

View File

@ -0,0 +1,62 @@
/*
* 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.tests.system.verification;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.tests.system.NiFiInstanceFactory;
import org.apache.nifi.tests.system.SpawnedClusterNiFiInstanceFactory;
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
import org.junit.Test;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
public class ClusteredVerifiableControllerServiceSystemIT extends VerifiableControllerServiceSystemIT {
@Override
protected NiFiInstanceFactory getInstanceFactory() {
return new SpawnedClusterNiFiInstanceFactory(
"src/test/resources/conf/clustered/node1/bootstrap.conf",
"src/test/resources/conf/clustered/node2/bootstrap.conf");
}
@Test
public void testDifferentResultsFromDifferentNodes() throws InterruptedException, IOException, NiFiClientException {
final ControllerServiceEntity service = getClientUtil().createControllerService("EnsureControllerServiceConfigurationCorrect");
final Map<String, String> properties = new HashMap<>();
properties.put("Successful Verification", "true");
properties.put("Failure Node Number", "2");
getClientUtil().updateControllerService(service, properties);
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyControllerServiceConfig(service.getId(), properties);
assertEquals(3, resultList.size());
// First verification result will be component validation.
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
// Second verification result will be verification results
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(1).getOutcome());
// Third verification result is for Fail On Primary Node
assertEquals(Outcome.FAILED.name(), resultList.get(2).getOutcome());
}
}

View File

@ -0,0 +1,63 @@
/*
* 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.tests.system.verification;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.tests.system.NiFiInstanceFactory;
import org.apache.nifi.tests.system.SpawnedClusterNiFiInstanceFactory;
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.entity.ProcessorEntity;
import org.junit.Test;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
public class ClusteredVerifiableProcessorSystemIT extends VerifiableProcessorSystemIT {
@Override
protected NiFiInstanceFactory getInstanceFactory() {
return new SpawnedClusterNiFiInstanceFactory(
"src/test/resources/conf/clustered/node1/bootstrap.conf",
"src/test/resources/conf/clustered/node2/bootstrap.conf");
}
@Test
public void testDifferentResultsFromDifferentNodes() throws InterruptedException, IOException, NiFiClientException {
final ProcessorEntity processor = getClientUtil().createProcessor("EnsureProcessorConfigurationCorrect");
final Map<String, String> properties = new HashMap<>();
properties.put("Successful Verification", "true");
properties.put("Failure Node Number", "2");
getClientUtil().updateProcessorProperties(processor, properties);
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyProcessorConfig(processor.getId(), properties);
assertEquals(3, resultList.size());
// First verification result will be component validation.
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
// Second verification result will be verification results
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(1).getOutcome());
// Third verification result is for Fail On Primary Node
assertEquals(Outcome.FAILED.name(), resultList.get(2).getOutcome());
}
}

View File

@ -0,0 +1,64 @@
/*
* 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.tests.system.verification;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.tests.system.NiFiInstanceFactory;
import org.apache.nifi.tests.system.SpawnedClusterNiFiInstanceFactory;
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
import org.junit.Test;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
public class ClusteredVerifiableReportingTaskSystemIT extends VerifiableReportingTaskSystemIT {
@Override
protected NiFiInstanceFactory getInstanceFactory() {
return new SpawnedClusterNiFiInstanceFactory(
"src/test/resources/conf/clustered/node1/bootstrap.conf",
"src/test/resources/conf/clustered/node2/bootstrap.conf");
}
@Test
public void testDifferentResultsFromDifferentNodes() throws InterruptedException, IOException, NiFiClientException {
final ReportingTaskEntity reportingTask = getClientUtil().createReportingTask("EnsureReportingTaskConfigurationCorrect");
final Map<String, String> properties = new HashMap<>();
properties.put("Successful Verification", "true");
properties.put("Failure Node Number", "2");
getClientUtil().updateReportingTask(reportingTask, properties);
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyReportingTaskConfig(reportingTask.getId(), properties);
assertEquals(3, resultList.size());
// First verification result will be component validation.
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
// Second verification result will be verification results
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(1).getOutcome());
// Third verification result is for Fail On Primary Node
assertEquals(Outcome.FAILED.name(), resultList.get(2).getOutcome());
}
}

View File

@ -0,0 +1,195 @@
/*
* 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.tests.system.verification;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.tests.system.NiFiSystemIT;
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
import org.junit.Test;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
public class VerifiableControllerServiceSystemIT extends NiFiSystemIT {
@Test
public void testVerificationWithValidConfigWhenComponentValid() throws NiFiClientException, IOException, InterruptedException {
final ControllerServiceEntity service = getClientUtil().createControllerService("EnsureControllerServiceConfigurationCorrect");
final Map<String, String> properties = Collections.singletonMap("Successful Verification", "true");
getClientUtil().updateControllerService(service, properties);
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyControllerServiceConfig(service.getId(), properties);
assertEquals(3, resultList.size());
// First verification result will be component validation.
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
// Second verification result will be verification results
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(1).getOutcome());
// Third verification result is for Fail On Primary Node
assertEquals(Outcome.SKIPPED.name(), resultList.get(2).getOutcome());
}
@Test
public void testVerifyWithInvalidConfigWhenComponentValid() throws NiFiClientException, IOException, InterruptedException {
final ControllerServiceEntity service = getClientUtil().createControllerService("EnsureControllerServiceConfigurationCorrect");
final Map<String, String> properties = Collections.singletonMap("Successful Verification", "true");
getClientUtil().updateControllerService(service, properties);
// Verify with properties that will give us failed verification
final Map<String, String> invalidProperties = Collections.singletonMap("Successful Verification", "false");
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyControllerServiceConfig(service.getId(), invalidProperties);
assertEquals(3, resultList.size());
// First verification result will be component validation.
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
// Second verification result will be FAILED because the 'Successful Verification' property is set to false
assertEquals(Outcome.FAILED.name(), resultList.get(1).getOutcome());
// Third verification result is for Fail On Primary Node
assertEquals(Outcome.SKIPPED.name(), resultList.get(2).getOutcome());
}
@Test
public void testVerifyWithAttributes() throws NiFiClientException, IOException, InterruptedException {
final ControllerServiceEntity service = getClientUtil().createControllerService("EnsureControllerServiceConfigurationCorrect");
final Map<String, String> properties = new HashMap<>();
properties.put("Successful Verification", "true");
properties.put("Verification Steps", "${steps}");
getClientUtil().updateControllerService(service, properties);
final Map<String, String> goodAttributes = Collections.singletonMap("steps", "5");
final Map<String, String> badAttributes = Collections.singletonMap("steps", "foo");
// Verify using attributes that should give us a successful verification
List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyControllerServiceConfig(service.getId(), properties, goodAttributes);
assertEquals("Got unexpected results: " + resultList, 7, resultList.size());
// Should have SUCCESS for validation, then 5 successes for the steps. Then 1 skipped for the Fail on Primary Node
for (int i=0; i < resultList.size() - 1; i++) {
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(i).getOutcome());
}
assertEquals(Outcome.SKIPPED.name(), resultList.get(resultList.size() - 1).getOutcome());
// Verify using attributes that should give us a failed verification
resultList = getClientUtil().verifyControllerServiceConfig(service.getId(), properties, badAttributes);
assertEquals(2, resultList.size());
// First verification result will be component validation.
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
// Second verification result will be FAILED because the number of Verification Steps are invalid
assertEquals(Outcome.FAILED.name(), resultList.get(1).getOutcome());
}
@Test
public void testVerificationWithValidConfigWhenComponentInvalid() throws NiFiClientException, IOException, InterruptedException {
final ControllerServiceEntity service = getClientUtil().createControllerService("EnsureControllerServiceConfigurationCorrect");
final Map<String, String> invalidProperties = Collections.singletonMap("Successful Verification", "foo");
getClientUtil().updateControllerService(service, invalidProperties);
final Map<String, String> validProperties = Collections.singletonMap("Successful Verification", "true");
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyControllerServiceConfig(service.getId(), validProperties);
assertEquals(3, resultList.size());
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(1).getOutcome());
assertEquals(Outcome.SKIPPED.name(), resultList.get(2).getOutcome());
}
@Test
public void testVerifyWithInvalidConfigWhenComponentInvalid() throws InterruptedException, IOException, NiFiClientException {
final ControllerServiceEntity service = getClientUtil().createControllerService("EnsureControllerServiceConfigurationCorrect");
final Map<String, String> invalidProperties = Collections.singletonMap("Successful Verification", "foo");
getClientUtil().updateControllerService(service, invalidProperties);
final Map<String, String> otherInvalidProperties = Collections.singletonMap("Successful Verification", "bar");
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyControllerServiceConfig(service.getId(), otherInvalidProperties);
assertEquals(1, resultList.size());
for (final ConfigVerificationResultDTO resultDto : resultList) {
assertEquals(Outcome.FAILED.name(), resultDto.getOutcome());
}
}
@Test
public void testVerificationWithValidConfigWhenComponentRunning() throws InterruptedException, IOException, NiFiClientException {
final ControllerServiceEntity service = getClientUtil().createControllerService("EnsureControllerServiceConfigurationCorrect");
final Map<String, String> properties = Collections.singletonMap("Successful Verification", "true");
getClientUtil().updateControllerService(service, properties);
getClientUtil().enableControllerService(service);
assertThrows(NiFiClientException.class, () -> {
getClientUtil().verifyControllerServiceConfig(service.getId(), properties);
});
}
@Test
public void testVerifyWhenExceptionThrown() throws InterruptedException, IOException, NiFiClientException {
final ControllerServiceEntity service = getClientUtil().createControllerService("EnsureControllerServiceConfigurationCorrect");
final Map<String, String> properties = new HashMap<>();
properties.put("Successful Verification", "true");
properties.put("Exception on Verification", "true");
getClientUtil().updateControllerService(service, properties);
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyControllerServiceConfig(service.getId(), properties);
assertEquals(2, resultList.size());
// Results should show that validation is successful but that there was a failure in performing verification
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
assertEquals(Outcome.FAILED.name(), resultList.get(1).getOutcome());
}
@Test
public void testValidProcessorWithoutVerifiableControllerServiceAnnotation() throws NiFiClientException, IOException, InterruptedException {
final ControllerServiceEntity service = getClientUtil().createControllerService("StandardSleepService");
// Even though processor does not implement VerifiableProcessor, validation should still be run
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyControllerServiceConfig(service.getId(), Collections.emptyMap());
assertEquals(1, resultList.size());
// Even though GenerateFlowFile is not connected, it should be valid because connections are not considered when verifying the processor
assertEquals("Unexpected results: " + resultList, Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
}
@Test
public void testInvalidConfigForProcessorWithoutVerifiableControllerServiceAnnotation() throws NiFiClientException, IOException, InterruptedException {
final ControllerServiceEntity service = getClientUtil().createControllerService("StandardSleepService");
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyControllerServiceConfig(service.getId(), Collections.singletonMap("Validate Sleep Time", "foo"));
assertEquals(1, resultList.size());
assertEquals("Unexpected results: " + resultList, Outcome.FAILED.name(), resultList.get(0).getOutcome());
}
}

View File

@ -0,0 +1,226 @@
/*
* 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.tests.system.verification;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.tests.system.NiFiSystemIT;
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.entity.ProcessorEntity;
import org.junit.Test;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
public class VerifiableProcessorSystemIT extends NiFiSystemIT {
@Test
public void testVerificationWithValidConfigWhenComponentValid() throws NiFiClientException, IOException, InterruptedException {
final ProcessorEntity processor = getClientUtil().createProcessor("EnsureProcessorConfigurationCorrect");
final Map<String, String> properties = Collections.singletonMap("Successful Verification", "true");
getClientUtil().updateProcessorProperties(processor, properties);
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyProcessorConfig(processor.getId(), properties);
assertEquals(3, resultList.size());
// First verification result will be component validation.
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
// Second verification result will be verification results
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(1).getOutcome());
// Third verification result is for Fail On Primary Node
assertEquals(Outcome.SKIPPED.name(), resultList.get(2).getOutcome());
}
@Test
public void testVerifyWithInvalidConfigWhenComponentValid() throws NiFiClientException, IOException, InterruptedException {
final ProcessorEntity processor = getClientUtil().createProcessor("EnsureProcessorConfigurationCorrect");
// Make processor valid
final Map<String, String> properties = Collections.singletonMap("Successful Verification", "true");
getClientUtil().updateProcessorProperties(processor, properties);
// Verify with properties that will give us failed verification
final Map<String, String> invalidProperties = Collections.singletonMap("Successful Verification", "false");
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyProcessorConfig(processor.getId(), invalidProperties);
assertEquals(3, resultList.size());
// First verification result will be component validation.
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
// Second verification result will be FAILED because the 'Successful Verification' property is set to false
assertEquals(Outcome.FAILED.name(), resultList.get(1).getOutcome());
// Third verification result is for Fail On Primary Node
assertEquals(Outcome.SKIPPED.name(), resultList.get(2).getOutcome());
}
@Test
public void testVerifyWithAttributes() throws NiFiClientException, IOException, InterruptedException {
final ProcessorEntity processor = getClientUtil().createProcessor("EnsureProcessorConfigurationCorrect");
final Map<String, String> properties = new HashMap<>();
properties.put("Successful Verification", "true");
properties.put("Verification Steps", "${steps}");
getClientUtil().updateProcessorProperties(processor, properties);
final Map<String, String> goodAttributes = Collections.singletonMap("steps", "5");
final Map<String, String> badAttributes = Collections.singletonMap("steps", "foo");
// Verify using attributes that should give us a successful verification
List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyProcessorConfig(processor.getId(), properties, goodAttributes);
assertEquals("Got unexpected results: " + resultList, 7, resultList.size());
// Should have SUCCESS for validation, then 5 successes for the steps. Then 1 skipped for the Fail on Primary Node
for (int i=0; i < resultList.size() - 1; i++) {
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(i).getOutcome());
}
assertEquals(Outcome.SKIPPED.name(), resultList.get(resultList.size() - 1).getOutcome());
// Verify using attributes that should give us a failed verification
resultList = getClientUtil().verifyProcessorConfig(processor.getId(), properties, badAttributes);
assertEquals(2, resultList.size());
// First verification result will be component validation.
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
// Second verification result will be FAILED because the number of Verification Steps are invalid
assertEquals(Outcome.FAILED.name(), resultList.get(1).getOutcome());
}
@Test
public void testVerificationWithValidConfigWhenComponentInvalid() throws NiFiClientException, IOException, InterruptedException {
final ProcessorEntity processor = getClientUtil().createProcessor("EnsureProcessorConfigurationCorrect");
final Map<String, String> invalidProperties = Collections.singletonMap("Successful Verification", "foo");
getClientUtil().updateProcessorProperties(processor, invalidProperties);
// Wait until the processor has become invalid
getClientUtil().waitForInvalidProcessor(processor.getId());
final Map<String, String> validProperties = Collections.singletonMap("Successful Verification", "true");
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyProcessorConfig(processor.getId(), validProperties);
assertEquals(3, resultList.size());
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(1).getOutcome());
assertEquals(Outcome.SKIPPED.name(), resultList.get(2).getOutcome());
}
@Test
public void testVerifyWithInvalidConfigWhenComponentInvalid() throws InterruptedException, IOException, NiFiClientException {
final ProcessorEntity processor = getClientUtil().createProcessor("EnsureProcessorConfigurationCorrect");
final Map<String, String> invalidProperties = Collections.singletonMap("Successful Verification", "foo");
getClientUtil().updateProcessorProperties(processor, invalidProperties);
// Wait until the processor has become invalid
getClientUtil().waitForInvalidProcessor(processor.getId());
final Map<String, String> otherInvalidProperties = Collections.singletonMap("Successful Verification", "bar");
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyProcessorConfig(processor.getId(), otherInvalidProperties);
assertEquals(1, resultList.size());
for (final ConfigVerificationResultDTO resultDto : resultList) {
assertEquals(Outcome.FAILED.name(), resultDto.getOutcome());
}
}
@Test
public void testVerificationWithValidConfigWhenComponentRunning() throws InterruptedException, IOException, NiFiClientException {
final ProcessorEntity processor = getClientUtil().createProcessor("EnsureProcessorConfigurationCorrect");
final Map<String, String> properties = Collections.singletonMap("Successful Verification", "true");
getClientUtil().updateProcessorProperties(processor, properties);
getClientUtil().startProcessGroupComponents("root");
getClientUtil().waitForProcessorState(processor.getId(), "RUNNING");
assertThrows(NiFiClientException.class, () -> {
getClientUtil().verifyProcessorConfig(processor.getId(), properties);
});
}
@Test
public void testVerifyWithInvalidConfigWhenComponentDisabled() throws InterruptedException, IOException, NiFiClientException {
final ProcessorEntity processor = getClientUtil().createProcessor("EnsureProcessorConfigurationCorrect");
getNifiClient().getProcessorClient().disableProcessor(processor);
final Map<String, String> validProperties = Collections.singletonMap("Successful Verification", "true");
getClientUtil().updateProcessorProperties(processor, validProperties);
List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyProcessorConfig(processor.getId(), validProperties);
assertEquals(3, resultList.size());
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(1).getOutcome());
assertEquals(Outcome.SKIPPED.name(), resultList.get(2).getOutcome());
final Map<String, String> failureProperties = Collections.singletonMap("Successful Verification", "false");
resultList = getClientUtil().verifyProcessorConfig(processor.getId(), failureProperties);
assertEquals(3, resultList.size());
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
assertEquals(Outcome.FAILED.name(), resultList.get(1).getOutcome());
assertEquals(Outcome.SKIPPED.name(), resultList.get(2).getOutcome());
}
@Test
public void testVerifyWhenExceptionThrown() throws InterruptedException, IOException, NiFiClientException {
final ProcessorEntity processor = getClientUtil().createProcessor("EnsureProcessorConfigurationCorrect");
final Map<String, String> properties = new HashMap<>();
properties.put("Successful Verification", "true");
properties.put("Exception on Verification", "true");
getClientUtil().updateProcessorProperties(processor, properties);
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyProcessorConfig(processor.getId(), properties);
assertEquals(2, resultList.size());
// Results should show that validation is successful but that there was a failure in performing verification
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
assertEquals(Outcome.FAILED.name(), resultList.get(1).getOutcome());
}
@Test
public void testValidProcessorWithoutVerifiableProcessorAnnotation() throws NiFiClientException, IOException, InterruptedException {
final ProcessorEntity processor = getClientUtil().createProcessor("GenerateFlowFile");
// Even though processor does not implement VerifiableProcessor, validation should still be run
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyProcessorConfig(processor.getId(), Collections.emptyMap());
assertEquals(1, resultList.size());
// Even though GenerateFlowFile is not connected, it should be valid because connections are not considered when verifying the processor
assertEquals("Unexpected results: " + resultList, Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
}
@Test
public void testInvalidConfigForProcessorWithoutVerifiableProcessorAnnotation() throws NiFiClientException, IOException, InterruptedException {
final ProcessorEntity processor = getClientUtil().createProcessor("GenerateFlowFile");
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyProcessorConfig(processor.getId(), Collections.singletonMap("File Size", "foo"));
assertEquals(1, resultList.size());
assertEquals("Unexpected results: " + resultList, Outcome.FAILED.name(), resultList.get(0).getOutcome());
}
}

View File

@ -0,0 +1,174 @@
/*
* 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.tests.system.verification;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.tests.system.NiFiSystemIT;
import org.apache.nifi.toolkit.cli.impl.client.nifi.NiFiClientException;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
import org.apache.nifi.web.api.entity.ReportingTaskRunStatusEntity;
import org.junit.Test;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
public class VerifiableReportingTaskSystemIT extends NiFiSystemIT {
@Test
public void testVerificationWithValidConfigWhenComponentValid() throws NiFiClientException, IOException, InterruptedException {
final ReportingTaskEntity reportingTask = getClientUtil().createReportingTask("EnsureReportingTaskConfigurationCorrect");
final Map<String, String> properties = Collections.singletonMap("Successful Verification", "true");
getClientUtil().updateReportingTask(reportingTask, properties);
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyReportingTaskConfig(reportingTask.getId(), properties);
assertEquals(3, resultList.size());
// First verification result will be component validation.
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
// Second verification result will be verification results
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(1).getOutcome());
// Third verification result is for Fail On Primary Node
assertEquals(Outcome.SKIPPED.name(), resultList.get(2).getOutcome());
}
@Test
public void testVerifyWithInvalidConfigWhenComponentValid() throws NiFiClientException, IOException, InterruptedException {
final ReportingTaskEntity reportingTask = getClientUtil().createReportingTask("EnsureReportingTaskConfigurationCorrect");
final Map<String, String> properties = Collections.singletonMap("Successful Verification", "true");
getClientUtil().updateReportingTask(reportingTask, properties);
// Verify with properties that will give us failed verification
final Map<String, String> invalidProperties = Collections.singletonMap("Successful Verification", "false");
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyReportingTaskConfig(reportingTask.getId(), invalidProperties);
assertEquals(3, resultList.size());
// First verification result will be component validation.
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
// Second verification result will be FAILED because the 'Successful Verification' property is set to false
assertEquals(Outcome.FAILED.name(), resultList.get(1).getOutcome());
// Third verification result is for Fail On Primary Node
assertEquals(Outcome.SKIPPED.name(), resultList.get(2).getOutcome());
}
@Test
public void testVerificationWithValidConfigWhenComponentInvalid() throws NiFiClientException, IOException, InterruptedException {
final ReportingTaskEntity reportingTask = getClientUtil().createReportingTask("EnsureReportingTaskConfigurationCorrect");
final Map<String, String> invalidProperties = Collections.singletonMap("Successful Verification", "foo");
getClientUtil().updateReportingTask(reportingTask, invalidProperties);
final Map<String, String> validProperties = Collections.singletonMap("Successful Verification", "true");
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyReportingTaskConfig(reportingTask.getId(), validProperties);
assertEquals(3, resultList.size());
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(1).getOutcome());
assertEquals(Outcome.SKIPPED.name(), resultList.get(2).getOutcome());
}
@Test
public void testVerifyWithInvalidConfigWhenComponentInvalid() throws InterruptedException, IOException, NiFiClientException {
final ReportingTaskEntity reportingTask = getClientUtil().createReportingTask("EnsureReportingTaskConfigurationCorrect");
final Map<String, String> invalidProperties = Collections.singletonMap("Successful Verification", "foo");
getClientUtil().updateReportingTask(reportingTask, invalidProperties);
final Map<String, String> otherInvalidProperties = Collections.singletonMap("Successful Verification", "bar");
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyReportingTaskConfig(reportingTask.getId(), otherInvalidProperties);
assertEquals(1, resultList.size());
for (final ConfigVerificationResultDTO resultDto : resultList) {
assertEquals(Outcome.FAILED.name(), resultDto.getOutcome());
}
}
@Test
public void testVerificationWithValidConfigWhenComponentRunning() throws IOException, NiFiClientException {
final ReportingTaskEntity reportingTask = getClientUtil().createReportingTask("EnsureReportingTaskConfigurationCorrect");
final Map<String, String> properties = Collections.singletonMap("Successful Verification", "true");
getClientUtil().updateReportingTask(reportingTask, properties);
// Start reporting task
final ReportingTaskRunStatusEntity runStatusEntity = new ReportingTaskRunStatusEntity();
runStatusEntity.setRevision(reportingTask.getRevision());
runStatusEntity.setState("RUNNING");
getNifiClient().getReportingTasksClient().activateReportingTask(reportingTask.getId(), runStatusEntity);
assertThrows(NiFiClientException.class, () -> getClientUtil().verifyReportingTaskConfig(reportingTask.getId(), properties));
}
@Test
public void testVerifyWhenExceptionThrown() throws InterruptedException, IOException, NiFiClientException {
final ReportingTaskEntity reportingTask = getClientUtil().createReportingTask("EnsureReportingTaskConfigurationCorrect");
final Map<String, String> properties = new HashMap<>();
properties.put("Successful Verification", "true");
properties.put("Exception on Verification", "true");
getClientUtil().updateReportingTask(reportingTask, properties);
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyReportingTaskConfig(reportingTask.getId(), properties);
assertEquals(2, resultList.size());
// Results should show that validation is successful but that there was a failure in performing verification
assertEquals(Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
assertEquals(Outcome.FAILED.name(), resultList.get(1).getOutcome());
}
@Test
public void testValidProcessorWithoutVerifiableReportingTaskAnnotation() throws NiFiClientException, IOException, InterruptedException {
final ReportingTaskEntity reportingTask = getClientUtil().createReportingTask("WriteToFileReportingTask");
// Even though processor does not implement VerifiableProcessor, validation should still be run
final Map<String, String> properties = new HashMap<>();
properties.put("Filename", "./logs");
properties.put("Text", "Hello World");
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyReportingTaskConfig(reportingTask.getId(), properties);
assertEquals(1, resultList.size());
// Even though GenerateFlowFile is not connected, it should be valid because connections are not considered when verifying the processor
assertEquals("Unexpected results: " + resultList, Outcome.SUCCESSFUL.name(), resultList.get(0).getOutcome());
}
@Test
public void testInvalidConfigForProcessorWithoutVerifiableReportingTaskAnnotation() throws NiFiClientException, IOException, InterruptedException {
final ReportingTaskEntity reportingTask = getClientUtil().createReportingTask("WriteToFileReportingTask");
final Map<String, String> properties = new HashMap<>();
properties.put("Filename", "/foo-i-do-not-exist");
properties.put("Text", "Hello World");
final List<ConfigVerificationResultDTO> resultList = getClientUtil().verifyReportingTaskConfig(reportingTask.getId(), properties);
assertEquals(1, resultList.size());
assertEquals("Unexpected results: " + resultList, Outcome.FAILED.name(), resultList.get(0).getOutcome());
}
}

View File

@ -22,7 +22,11 @@ working.dir=./target/node1
graceful.shutdown.seconds=20
# JVM memory settings
java.arg.2= -Xms512m
java.arg.2=-Xms512m
java.arg.3=-Xmx512m
java.arg.14=-Djava.awt.headless=true
#java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8002
java.arg.nodeNum=-DnodeNumber=1

View File

@ -151,6 +151,9 @@
<logger name="org.apache.nifi.web.api.AccessResource" level="INFO" additivity="false">
<appender-ref ref="USER_FILE"/>
</logger>
<logger name="org.apache.nifi.web.api" level="DEBUG" additivity="false">
<appender-ref ref="USER_FILE"/>
</logger>
<!--

View File

@ -22,7 +22,11 @@ working.dir=./target/node2
graceful.shutdown.seconds=20
# JVM memory settings
java.arg.2= -Xms512m
java.arg.2=-Xms512m
java.arg.3=-Xmx512m
java.arg.14=-Djava.awt.headless=true
#java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8003
java.arg.nodeNum=-DnodeNumber=2

Some files were not shown because too many files have changed in this diff Show More