NIFI-4149 - Indicate if EL is evaluated against FFs or not

- take into account input requirement for documentation rendering
- Renamed variable registry scope and added comments
- Doc + change in mock framework to check scope + update of components + UI
This commit is contained in:
Pierre Villard 2017-09-19 09:27:46 +02:00 committed by Mark Payne
parent 7ff38f690d
commit 4c787799ff
314 changed files with 2211 additions and 1710 deletions

View File

@ -24,6 +24,7 @@ import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.expression.ExpressionLanguageScope;
/**
* An annotation that may be placed on a {@link ConfigurableComponent} to
@ -38,9 +39,13 @@ public @interface DynamicProperty {
String name();
@Deprecated
boolean supportsExpressionLanguage() default false;
String value();
String description();
ExpressionLanguageScope expressionLanguageScope() default ExpressionLanguageScope.NONE;
}

View File

@ -24,6 +24,7 @@ import java.util.List;
import java.util.Set;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.expression.ExpressionLanguageScope;
/**
* An immutable object for holding information about a type of component
@ -78,7 +79,13 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
* indicates whether or not this property supports the Attribute Expression
* Language
*/
@Deprecated
private final boolean expressionLanguageSupported;
/**
* indicates whether or nor this property will evaluate expression language
* against the flow file attributes
*/
private final ExpressionLanguageScope expressionLanguageScope;
/**
* indicates whether or not this property represents resources that should be added
* to the classpath for this instance of the component
@ -109,6 +116,7 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
this.dynamic = builder.dynamic;
this.dynamicallyModifiesClasspath = builder.dynamicallyModifiesClasspath;
this.expressionLanguageSupported = builder.expressionLanguageSupported;
this.expressionLanguageScope = builder.expressionLanguageScope;
this.controllerServiceDefinition = builder.controllerServiceDefinition;
this.validators = new ArrayList<>(builder.validators);
}
@ -236,7 +244,11 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
private List<AllowableValue> allowableValues = null;
private boolean required = false;
private boolean sensitive = false;
@Deprecated
private boolean expressionLanguageSupported = false;
private ExpressionLanguageScope expressionLanguageScope = ExpressionLanguageScope.NONE;
private boolean dynamic = false;
private boolean dynamicallyModifiesClasspath = false;
private Class<? extends ControllerService> controllerServiceDefinition;
@ -253,6 +265,7 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
this.dynamic = specDescriptor.dynamic;
this.dynamicallyModifiesClasspath = specDescriptor.dynamicallyModifiesClasspath;
this.expressionLanguageSupported = specDescriptor.expressionLanguageSupported;
this.expressionLanguageScope = specDescriptor.expressionLanguageScope;
this.controllerServiceDefinition = specDescriptor.getControllerServiceDefinition();
this.validators = new ArrayList<>(specDescriptor.validators);
return this;
@ -297,11 +310,23 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
* @param supported true if yes; false otherwise
* @return the builder
*/
@Deprecated
public Builder expressionLanguageSupported(final boolean supported) {
this.expressionLanguageSupported = supported;
return this;
}
/**
* Sets the scope of the expression language evaluation
*
* @param expressionLanguageScope scope of the expression language evaluation
* @return the builder
*/
public Builder expressionLanguageSupported(final ExpressionLanguageScope expressionLanguageScope) {
this.expressionLanguageScope = expressionLanguageScope;
return this;
}
/**
* @param description of the property
* @return the builder
@ -513,7 +538,11 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
}
public boolean isExpressionLanguageSupported() {
return expressionLanguageSupported;
return expressionLanguageSupported || !expressionLanguageScope.equals(ExpressionLanguageScope.NONE);
}
public ExpressionLanguageScope getExpressionLanguageScope() {
return expressionLanguageScope;
}
public boolean isDynamicClasspathModifier() {

View File

@ -0,0 +1,68 @@
/*
* 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.expression;
/**
* Indicates the scope of expression language on a property descriptor.
*
* Scope of the expression language is hierarchical.
* NONE -> VARIABLE_REGISTRY -> FLOWFILE_ATTRIBUTES
*
* When scope is set to FlowFiles attributes, variables are evaluated
* against attributes of each incoming flow file. If no matching attribute
* is found, variable registry will be checked.
*
* NONE - expression language is not supported
*
* VARIABLE_REGISTRY is hierarchically constructed as below:
* |---- Variables defined at process group level and then, recursively, up
* | to the higher process group until the root process group.
* |--- Variables defined in custom properties files through the
* | nifi.variable.registry.properties property in nifi.properties file.
* |-- Environment variables defined at JVM level and system properties.
*
* FLOWFILE_ATTRIBUTES - will check attributes of each individual flow file
*
*/
public enum ExpressionLanguageScope {
/**
* Expression language is disabled
*/
NONE("Not Supported"),
/**
* Expression language is evaluated against variables in registry
*/
VARIABLE_REGISTRY("Variable Registry Only"),
/**
* Expression language is evaluated per flow file using attributes
*/
FLOWFILE_ATTRIBUTES("Variable Registry and FlowFile Attributes");
private String description;
private ExpressionLanguageScope(String description) {
this.description = description;
}
public String getDescription() {
return this.description;
}
}

View File

@ -42,6 +42,7 @@ import org.apache.nifi.bootstrap.notification.NotificationType;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
@ -58,20 +59,20 @@ public class EmailNotificationService extends AbstractNotificationService {
.description("The Port used for SMTP communications")
.required(true)
.defaultValue("25")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.PORT_VALIDATOR)
.build();
public static final PropertyDescriptor SMTP_USERNAME = new PropertyDescriptor.Builder()
.name("SMTP Username")
.description("Username for the SMTP account")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.required(false)
.build();
public static final PropertyDescriptor SMTP_PASSWORD = new PropertyDescriptor.Builder()
.name("SMTP Password")
.description("Password for the SMTP account")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.required(false)
.sensitive(true)
@ -80,7 +81,7 @@ public class EmailNotificationService extends AbstractNotificationService {
.name("SMTP Auth")
.description("Flag indicating whether authentication should be used")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.BOOLEAN_VALIDATOR)
.defaultValue("true")
.build();
@ -88,7 +89,7 @@ public class EmailNotificationService extends AbstractNotificationService {
.name("SMTP TLS")
.description("Flag indicating whether TLS should be enabled")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.BOOLEAN_VALIDATOR)
.defaultValue("false")
.build();
@ -96,7 +97,7 @@ public class EmailNotificationService extends AbstractNotificationService {
.name("SMTP Socket Factory")
.description("Socket Factory to use for SMTP Connection")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.defaultValue("javax.net.ssl.SSLSocketFactory")
.build();
@ -104,7 +105,7 @@ public class EmailNotificationService extends AbstractNotificationService {
.name("SMTP X-Mailer Header")
.description("X-Mailer used in the header of the outgoing email")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.defaultValue("NiFi")
.build();
@ -112,7 +113,7 @@ public class EmailNotificationService extends AbstractNotificationService {
.name("Content Type")
.description("Mime Type used to interpret the contents of the email, such as text/plain or text/html")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.defaultValue("text/plain")
.build();
@ -120,28 +121,28 @@ public class EmailNotificationService extends AbstractNotificationService {
.name("From")
.description("Specifies the Email address to use as the sender")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor TO = new PropertyDescriptor.Builder()
.name("To")
.description("The recipients to include in the To-Line of the email")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor CC = new PropertyDescriptor.Builder()
.name("CC")
.description("The recipients to include in the CC-Line of the email")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor BCC = new PropertyDescriptor.Builder()
.name("BCC")
.description("The recipients to include in the BCC-Line of the email")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();

View File

@ -30,6 +30,7 @@ import org.apache.nifi.bootstrap.notification.NotificationType;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.expression.AttributeExpression;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.security.util.SslContextFactory;
@ -54,21 +55,21 @@ public class HttpNotificationService extends AbstractNotificationService {
.name("URL")
.description("The URL to send the notification to.")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.URL_VALIDATOR)
.build();
public static final PropertyDescriptor PROP_CONNECTION_TIMEOUT = new PropertyDescriptor.Builder()
.name("Connection timeout")
.description("Max wait time for connection to remote service.")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.defaultValue("10s")
.build();
public static final PropertyDescriptor PROP_WRITE_TIMEOUT = new PropertyDescriptor.Builder()
.name("Write timeout")
.description("Max wait time for remote service to read the request sent.")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.defaultValue("10s")
.build();
@ -163,7 +164,7 @@ public class HttpNotificationService extends AbstractNotificationService {
.name(propertyDescriptorName)
.addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
.dynamic(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
}

View File

@ -147,8 +147,18 @@ icon (
image:iconInfo.png["Info"]
) next to the name of the Property. Hovering over this icon with the mouse will provide a tooltip that
provides helpful information about the Property. This information includes a description of the Property,
the default value (if any), historically configured values (if any), and whether or not this Property
supports the expression language.
the default value (if any), historically configured values (if any), and the evaluation scope of this
property for expression language. There are three values and the evaluation scope of the expression
language is hierarchical: NONE -> VARIABLE_REGISTRY -> FLOWFILE_ATTRIBUTES.
* NONE - expression language is not supported for this property
* VARIABLE_REGISTRY is hierarchically constructed as below:
** Variables defined at process group level and then, recursively, up to the higher process group until
the root process group.
** Variables defined in custom properties files through the nifi.variable.registry.properties property
in nifi.properties file.
** Environment variables defined at JVM level and system properties.
* FLOWFILE_ATTRIBUTES - will use attributes of each individual flow file.
[[editor]]

View File

@ -21,6 +21,7 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceLookup;
@ -97,4 +98,8 @@ public abstract class MockControllerServiceLookup implements ControllerServiceLo
final ControllerServiceConfiguration status = controllerServiceMap.get(serviceIdentifier);
return status == null ? null : serviceIdentifier;
}
public InputRequirement getInputRequirement() {
return null;
}
}

View File

@ -28,6 +28,7 @@ import java.util.Map;
import java.util.Objects;
import java.util.Set;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.attribute.expression.language.Query;
import org.apache.nifi.attribute.expression.language.Query.Range;
import org.apache.nifi.components.ConfigurableComponent;
@ -58,6 +59,7 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
private boolean allowExpressionValidation = true;
private volatile boolean incomingConnection = true;
private volatile boolean nonLoopConnection = true;
private volatile InputRequirement inputRequirement = null;
private int maxConcurrentTasks = 1;
private volatile Set<Relationship> connections = new HashSet<>();
@ -67,7 +69,7 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
private volatile boolean isPrimaryNode;
public MockProcessContext(final ConfigurableComponent component) {
this(component, new MockStateManager(component),VariableRegistry.EMPTY_REGISTRY);
this(component, new MockStateManager(component), VariableRegistry.EMPTY_REGISTRY);
}
/**
@ -79,6 +81,7 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
*/
public MockProcessContext(final ConfigurableComponent component, final StateManager stateManager, final VariableRegistry variableRegistry) {
this.component = Objects.requireNonNull(component);
this.inputRequirement = component.getClass().getAnnotation(InputRequirement.class);
this.stateManager = stateManager;
this.variableRegistry = variableRegistry;
}
@ -410,4 +413,10 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
}
isPrimaryNode = primaryNode;
}
@Override
public InputRequirement getInputRequirement() {
return inputRequirement;
}
}

View File

@ -20,14 +20,16 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.attribute.expression.language.Query;
import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
import org.apache.nifi.attribute.expression.language.Query.Range;
import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.expression.AttributeValueDecorator;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.exception.ProcessException;
@ -36,10 +38,12 @@ import org.apache.nifi.registry.VariableRegistry;
public class MockPropertyValue implements PropertyValue {
private final String rawValue;
private final Boolean expectExpressions;
private final ControllerServiceLookup serviceLookup;
private final ExpressionLanguageScope expressionLanguageScope;
private final MockControllerServiceLookup serviceLookup;
private final PropertyDescriptor propertyDescriptor;
private final PropertyValue stdPropValue;
private final VariableRegistry variableRegistry;
private boolean expressionsEvaluated = false;
public MockPropertyValue(final String rawValue) {
@ -59,12 +63,12 @@ public class MockPropertyValue implements PropertyValue {
}
private MockPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup, final PropertyDescriptor propertyDescriptor, final boolean alreadyEvaluated,
final VariableRegistry variableRegistry) {
final VariableRegistry variableRegistry) {
this.stdPropValue = new StandardPropertyValue(rawValue, serviceLookup, variableRegistry);
this.rawValue = rawValue;
this.serviceLookup = serviceLookup;
this.serviceLookup = (MockControllerServiceLookup) serviceLookup;
this.expectExpressions = propertyDescriptor == null ? null : propertyDescriptor.isExpressionLanguageSupported();
this.expressionLanguageScope = propertyDescriptor == null ? null : propertyDescriptor.getExpressionLanguageScope();
this.propertyDescriptor = propertyDescriptor;
this.expressionsEvaluated = alreadyEvaluated;
this.variableRegistry = variableRegistry;
@ -80,6 +84,55 @@ public class MockPropertyValue implements PropertyValue {
}
}
private void validateExpressionScope(boolean attributesAvailable) {
// language scope is not null, we have attributes available but scope is not equal to FF attributes
// it means that we're not evaluating against flow file attributes even though attributes are available
if(expressionLanguageScope != null
&& (attributesAvailable && !ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope))) {
throw new IllegalStateException("Attempting to evaluate expression language for " + propertyDescriptor.getName()
+ " using flow file attributes but the scope evaluation is set to " + expressionLanguageScope + ". The"
+ " proper scope should be set in the property descriptor using"
+ " PropertyDescriptor.Builder.expressionLanguageSupported(ExpressionLanguageScope)");
}
// if the service lookup is an instance of the validation context, we're in the validate() method
// at this point we don't have any flow file available and we should not care about the scope
// even though it is defined as FLOWFILE_ATTRIBUTES
if(expressionLanguageScope != null
&& ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope)
&& this.serviceLookup instanceof MockValidationContext) {
return;
}
// we check if the input requirement is INPUT_FORBIDDEN
// in that case, we don't care if attributes are not available even though scope is FLOWFILE_ATTRIBUTES
// it likely means that the property has been defined in a common/abstract class used by multiple processors with
// different input requirements.
if(expressionLanguageScope != null
&& ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope)
&& (this.serviceLookup.getInputRequirement() == null
|| this.serviceLookup.getInputRequirement().value().equals(InputRequirement.Requirement.INPUT_FORBIDDEN))) {
return;
}
// if we have a processor where input requirement is INPUT_ALLOWED, we need to check if there is an
// incoming connection or not. If not, we don't care if attributes are not available even though scope is FLOWFILE_ATTRIBUTES
if(expressionLanguageScope != null
&& ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope)
&& !((MockProcessContext) this.serviceLookup).hasIncomingConnection()) {
return;
}
// we're trying to evaluate against flow files attributes but we don't have any attributes available.
if(expressionLanguageScope != null
&& (!attributesAvailable && ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope))) {
throw new IllegalStateException("Attempting to evaluate expression language for " + propertyDescriptor.getName()
+ " without using flow file attributes but the scope evaluation is set to " + expressionLanguageScope + ". The"
+ " proper scope should be set in the property descriptor using"
+ " PropertyDescriptor.Builder.expressionLanguageSupported(ExpressionLanguageScope)");
}
}
@Override
public String getValue() {
ensureExpressionsEvaluated();
@ -185,6 +238,8 @@ public class MockPropertyValue implements PropertyValue {
return this;
}
validateExpressionScope(flowFile != null || additionalAttributes != null);
final PropertyValue newValue = stdPropValue.evaluateAttributeExpressions(flowFile, additionalAttributes, decorator, stateValues);
return new MockPropertyValue(newValue.getValue(), serviceLookup, propertyDescriptor, true, variableRegistry);
}

View File

@ -35,7 +35,7 @@ import org.apache.nifi.expression.ExpressionLanguageCompiler;
import org.apache.nifi.registry.VariableRegistry;
public class MockValidationContext implements ValidationContext, ControllerServiceLookup {
public class MockValidationContext extends MockControllerServiceLookup implements ValidationContext, ControllerServiceLookup {
private final MockProcessContext context;
private final Map<String, Boolean> expressionLanguageSupported;

View File

@ -25,6 +25,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.controller.status.ProcessGroupStatus;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.reporting.AbstractReportingTask;
import org.apache.nifi.reporting.ReportingContext;
@ -60,7 +61,7 @@ public class AmbariReportingTask extends AbstractReportingTask {
.name("Metrics Collector URL")
.description("The URL of the Ambari Metrics Collector Service")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.defaultValue("http://localhost:6188/ws/v1/timeline/metrics")
.addValidator(StandardValidators.URL_VALIDATOR)
.build();
@ -69,7 +70,7 @@ public class AmbariReportingTask extends AbstractReportingTask {
.name("Application ID")
.description("The Application ID to be included in the metrics sent to Ambari")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.defaultValue("nifi")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -78,7 +79,7 @@ public class AmbariReportingTask extends AbstractReportingTask {
.name("Hostname")
.description("The Hostname of this NiFi instance to be included in the metrics sent to Ambari")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.defaultValue("${hostname(true)}")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -88,7 +89,7 @@ public class AmbariReportingTask extends AbstractReportingTask {
.description("If specified, the reporting task will send metrics about this process group only. If"
+ " not, the root process group is used and global metrics are sent.")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();

View File

@ -38,6 +38,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.Validator;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
@ -84,7 +85,7 @@ public class PublishAMQP extends AbstractAMQPProcessor<AMQPPublisher> {
+ "It is an optional property. If kept empty the messages will be sent to a default AMQP exchange.")
.required(true)
.defaultValue("")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(Validator.VALID)
.build();
public static final PropertyDescriptor ROUTING_KEY = new PropertyDescriptor.Builder()
@ -94,7 +95,7 @@ public class PublishAMQP extends AbstractAMQPProcessor<AMQPPublisher> {
+ "corresponds to a destination queue name, otherwise a binding from the Exchange to a Queue via Routing Key must be set "
+ "(usually by the AMQP administrator)")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();

View File

@ -16,54 +16,9 @@
*/
package org.apache.nifi.atlas.reporting;
import com.sun.jersey.api.client.ClientResponse;
import org.apache.atlas.ApplicationProperties;
import org.apache.atlas.AtlasServiceException;
import org.apache.commons.lang3.StringUtils;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.config.SslConfigs;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
import org.apache.nifi.annotation.behavior.Stateful;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
import org.apache.nifi.atlas.NiFiAtlasHook;
import org.apache.nifi.atlas.NiFiAtlasClient;
import org.apache.nifi.atlas.NiFiFlow;
import org.apache.nifi.atlas.NiFiFlowAnalyzer;
import org.apache.nifi.atlas.provenance.AnalysisContext;
import org.apache.nifi.atlas.provenance.StandardAnalysisContext;
import org.apache.nifi.atlas.provenance.lineage.CompleteFlowPathLineage;
import org.apache.nifi.atlas.provenance.lineage.LineageStrategy;
import org.apache.nifi.atlas.provenance.lineage.SimpleFlowPathLineage;
import org.apache.nifi.atlas.resolver.ClusterResolver;
import org.apache.nifi.atlas.resolver.ClusterResolvers;
import org.apache.nifi.atlas.resolver.RegexClusterResolver;
import org.apache.nifi.atlas.security.AtlasAuthN;
import org.apache.nifi.atlas.security.Basic;
import org.apache.nifi.atlas.security.Kerberos;
import org.apache.nifi.components.AllowableValue;
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.components.state.Scope;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.controller.status.ProcessGroupStatus;
import org.apache.nifi.kerberos.KerberosCredentialsService;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceRepository;
import org.apache.nifi.reporting.AbstractReportingTask;
import org.apache.nifi.reporting.EventAccess;
import org.apache.nifi.reporting.ReportingContext;
import org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer;
import org.apache.nifi.ssl.SSLContextService;
import static org.apache.commons.lang3.StringUtils.isEmpty;
import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_BATCH_SIZE;
import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_START_POSITION;
import java.io.File;
import java.io.FileInputStream;
@ -88,11 +43,56 @@ import java.util.Set;
import java.util.function.Consumer;
import java.util.stream.Stream;
import static org.apache.commons.lang3.StringUtils.isEmpty;
import static org.apache.nifi.atlas.reporting.ReportLineageToAtlas.NIFI_KERBEROS_KEYTAB;
import static org.apache.nifi.atlas.reporting.ReportLineageToAtlas.NIFI_KERBEROS_PRINCIPAL;
import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_BATCH_SIZE;
import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_START_POSITION;
import org.apache.atlas.ApplicationProperties;
import org.apache.atlas.AtlasServiceException;
import org.apache.commons.lang3.StringUtils;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.config.SslConfigs;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
import org.apache.nifi.annotation.behavior.Stateful;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
import org.apache.nifi.atlas.NiFiAtlasClient;
import org.apache.nifi.atlas.NiFiAtlasHook;
import org.apache.nifi.atlas.NiFiFlow;
import org.apache.nifi.atlas.NiFiFlowAnalyzer;
import org.apache.nifi.atlas.provenance.AnalysisContext;
import org.apache.nifi.atlas.provenance.StandardAnalysisContext;
import org.apache.nifi.atlas.provenance.lineage.CompleteFlowPathLineage;
import org.apache.nifi.atlas.provenance.lineage.LineageStrategy;
import org.apache.nifi.atlas.provenance.lineage.SimpleFlowPathLineage;
import org.apache.nifi.atlas.resolver.ClusterResolver;
import org.apache.nifi.atlas.resolver.ClusterResolvers;
import org.apache.nifi.atlas.resolver.RegexClusterResolver;
import org.apache.nifi.atlas.security.AtlasAuthN;
import org.apache.nifi.atlas.security.Basic;
import org.apache.nifi.atlas.security.Kerberos;
import org.apache.nifi.components.AllowableValue;
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.components.state.Scope;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.controller.status.ProcessGroupStatus;
import org.apache.nifi.kerberos.KerberosCredentialsService;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceRepository;
import org.apache.nifi.reporting.AbstractReportingTask;
import org.apache.nifi.reporting.EventAccess;
import org.apache.nifi.reporting.ReportingContext;
import org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer;
import org.apache.nifi.ssl.SSLContextService;
import com.sun.jersey.api.client.ClientResponse;
@Tags({"atlas", "lineage"})
@CapabilityDescription("Report NiFi flow data set level lineage to Apache Atlas." +
@ -102,7 +102,8 @@ import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.
" in addition to NiFi provenance events providing detailed event level lineage." +
" See 'Additional Details' for further description and limitations.")
@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
@DynamicProperty(name = "hostnamePattern.<ClusterName>", value = "hostname Regex patterns", description = RegexClusterResolver.PATTERN_PROPERTY_PREFIX_DESC)
@DynamicProperty(name = "hostnamePattern.<ClusterName>", value = "hostname Regex patterns",
description = RegexClusterResolver.PATTERN_PROPERTY_PREFIX_DESC, expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY)
// In order for each reporting task instance to have its own static objects such as KafkaNotification.
@RequiresInstanceClassLoading
public class ReportLineageToAtlas extends AbstractReportingTask {
@ -115,7 +116,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
" For accessing Atlas behind Knox gateway, specify Knox gateway URL" +
" (e.g. https://knox-hostname:8443/gateway/{topology-name}/atlas).")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.build();
@ -135,7 +136,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
.displayName("Atlas Username")
.description("User name to communicate with Atlas.")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.build();
@ -145,7 +146,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
.description("Password to communicate with Atlas.")
.required(false)
.sensitive(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.build();
@ -156,7 +157,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
" If not specified and 'Create Atlas Configuration File' is disabled," +
" then, 'atlas-application.properties' file under root classpath is used.")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.build();
@ -166,7 +167,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
.description("NiFi URL is used in Atlas to represent this NiFi cluster (or standalone instance)." +
" It is recommended to use one that can be accessible remotely instead of using 'localhost'.")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.URL_VALIDATOR)
.build();
@ -178,7 +179,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
" Cluster name mappings can be configured by user defined properties." +
" See additional detail for detail.")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.build();
@ -189,7 +190,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
" automatically when this Reporting Task starts." +
" Note that the existing configuration file will be overwritten.")
.required(true)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.allowableValues("true", "false")
.defaultValue("false")
.build();
@ -210,7 +211,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
" NOTE: Once this reporting task has started, restarting NiFi is required to changed this property" +
" as Atlas library holds a unmodifiable static reference to Kafka client.")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.build();
@ -224,7 +225,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
.description("Protocol used to communicate with Kafka brokers to send Atlas hook notification messages." +
" Corresponds to Kafka's 'security.protocol' property.")
.required(true)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.allowableValues(SEC_PLAINTEXT, SEC_SSL, SEC_SASL_PLAINTEXT, SEC_SASL_SSL)
.defaultValue(SEC_PLAINTEXT.getValue())
.build();
@ -237,7 +238,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
" This principal will be set into 'sasl.jaas.config' Kafka's property.")
.required(false)
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor NIFI_KERBEROS_KEYTAB = new PropertyDescriptor.Builder()
.name("nifi-kerberos-keytab")
@ -247,7 +248,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
" This principal will be set into 'sasl.jaas.config' Kafka's property.")
.required(false)
.addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor KERBEROS_CREDENTIALS_SERVICE = new PropertyDescriptor.Builder()
.name("kerberos-credentials-service")
@ -267,7 +268,7 @@ public class ReportLineageToAtlas extends AbstractReportingTask {
" It is ignored unless one of the SASL options of the <Security Protocol> are selected.")
.required(false)
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.defaultValue("kafka")
.build();

View File

@ -20,6 +20,7 @@ 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.util.StandardValidators;
import java.util.ArrayList;
@ -47,7 +48,7 @@ public class RegexClusterResolver implements ClusterResolver {
.Builder().name(propertyDescriptorName)
.description(PATTERN_PROPERTY_PREFIX_DESC)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.dynamic(true)
.sensitive(false)
.build();

View File

@ -182,12 +182,14 @@ public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceCl
config.setConnectionTimeout(commsTimeout);
config.setSocketTimeout(commsTimeout);
final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
if (sslContextService != null) {
final SSLContext sslContext = sslContextService.createSSLContext(SSLContextService.ClientAuth.NONE);
// NIFI-3788: Changed hostnameVerifier from null to DHV (BrowserCompatibleHostnameVerifier is deprecated)
SdkTLSSocketFactory sdkTLSSocketFactory = new SdkTLSSocketFactory(sslContext, new DefaultHostnameVerifier());
config.getApacheHttpClientConfig().setSslSocketFactory(sdkTLSSocketFactory);
if(this.getSupportedPropertyDescriptors().contains(SSL_CONTEXT_SERVICE)) {
final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
if (sslContextService != null) {
final SSLContext sslContext = sslContextService.createSSLContext(SSLContextService.ClientAuth.NONE);
// NIFI-3788: Changed hostnameVerifier from null to DHV (BrowserCompatibleHostnameVerifier is deprecated)
SdkTLSSocketFactory sdkTLSSocketFactory = new SdkTLSSocketFactory(sslContext, new DefaultHostnameVerifier());
config.getApacheHttpClientConfig().setSslSocketFactory(sdkTLSSocketFactory);
}
}
if (context.getProperty(PROXY_HOST).isSet()) {

View File

@ -17,6 +17,7 @@
package org.apache.nifi.processors.aws.credentials.provider.factory;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.util.StandardValidators;
/**
@ -37,7 +38,7 @@ public class CredentialPropertyDescriptors {
public static final PropertyDescriptor USE_DEFAULT_CREDENTIALS = new PropertyDescriptor.Builder()
.name("default-credentials")
.displayName("Use Default Credentials")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(false)
.addValidator(StandardValidators.BOOLEAN_VALIDATOR)
.sensitive(false)
@ -50,7 +51,7 @@ public class CredentialPropertyDescriptors {
public static final PropertyDescriptor CREDENTIALS_FILE = new PropertyDescriptor.Builder()
.name("Credentials File")
.displayName("Credentials File")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(false)
.addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
.description("Path to a file containing AWS access key and secret key in properties file format.")
@ -59,7 +60,7 @@ public class CredentialPropertyDescriptors {
public static final PropertyDescriptor ACCESS_KEY = new PropertyDescriptor.Builder()
.name("Access Key")
.displayName("Access Key")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.sensitive(true)
@ -68,7 +69,7 @@ public class CredentialPropertyDescriptors {
public static final PropertyDescriptor SECRET_KEY = new PropertyDescriptor.Builder()
.name("Secret Key")
.displayName("Secret Key")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.sensitive(true)
@ -83,7 +84,7 @@ public class CredentialPropertyDescriptors {
public static final PropertyDescriptor PROFILE_NAME = new PropertyDescriptor.Builder()
.name("profile-name")
.displayName("Profile Name")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.sensitive(false)
@ -93,7 +94,7 @@ public class CredentialPropertyDescriptors {
public static final PropertyDescriptor USE_ANONYMOUS_CREDENTIALS = new PropertyDescriptor.Builder()
.name("anonymous-credentials")
.displayName("Use Anonymous Credentials")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(false)
.addValidator(StandardValidators.BOOLEAN_VALIDATOR)
.sensitive(false)
@ -110,7 +111,7 @@ public class CredentialPropertyDescriptors {
public static final PropertyDescriptor ASSUME_ROLE_ARN = new PropertyDescriptor.Builder()
.name("Assume Role ARN")
.displayName("Assume Role ARN")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.sensitive(false)
@ -123,7 +124,7 @@ public class CredentialPropertyDescriptors {
public static final PropertyDescriptor ASSUME_ROLE_NAME = new PropertyDescriptor.Builder()
.name("Assume Role Session Name")
.displayName("Assume Role Session Name")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.sensitive(false)
@ -148,7 +149,7 @@ public class CredentialPropertyDescriptors {
public static final PropertyDescriptor ASSUME_ROLE_EXTERNAL_ID = new PropertyDescriptor.Builder()
.name("assume-role-external-id")
.displayName("Assume Role External ID")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.sensitive(false)
@ -162,7 +163,7 @@ public class CredentialPropertyDescriptors {
public static final PropertyDescriptor ASSUME_ROLE_PROXY_HOST = new PropertyDescriptor.Builder()
.name("assume-role-proxy-host")
.displayName("Assume Role Proxy Host")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.sensitive(false)
@ -172,7 +173,7 @@ public class CredentialPropertyDescriptors {
public static final PropertyDescriptor ASSUME_ROLE_PROXY_PORT = new PropertyDescriptor.Builder()
.name("assume-role-proxy-port")
.displayName("Assume Role Proxy Port")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(false)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.sensitive(false)

View File

@ -31,6 +31,7 @@ import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.lifecycle.OnStopped;
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.ProcessSession;
@ -82,7 +83,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
public static final PropertyDescriptor TABLE = new PropertyDescriptor.Builder()
.name("Table Name")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.description("The DynamoDB table name")
.build();
@ -90,7 +91,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
public static final PropertyDescriptor HASH_KEY_VALUE = new PropertyDescriptor.Builder()
.name("Hash Key Value")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.description("The hash key value of the item")
.defaultValue("${dynamodb.item.hash.key.value}")
@ -100,7 +101,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
.name("Range Key Value")
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.defaultValue("${dynamodb.item.range.key.value}")
.build();
@ -123,7 +124,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
public static final PropertyDescriptor HASH_KEY_NAME = new PropertyDescriptor.Builder()
.name("Hash Key Name")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.description("The hash key name of the item")
.build();
@ -131,7 +132,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
public static final PropertyDescriptor RANGE_KEY_NAME = new PropertyDescriptor.Builder()
.name("Range Key Name")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.description("The range key name of the item")
.build();
@ -139,7 +140,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
public static final PropertyDescriptor JSON_DOCUMENT = new PropertyDescriptor.Builder()
.name("Json Document attribute")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.description("The Json document to be retrieved from the dynamodb item")
.build();
@ -147,7 +148,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
.name("Batch items for each request (between 1 and 50)")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.createLongValidator(1, 50, true))
.defaultValue("1")
.description("The items to be retrieved in one batch")
@ -158,7 +159,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
.description("Character set of data in the document")
.addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.defaultValue(Charset.defaultCharset().name())
.build();

View File

@ -17,6 +17,7 @@
package org.apache.nifi.processors.aws.kinesis.firehose;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.kinesis.AbstractBaseKinesisProcessor;
@ -34,7 +35,7 @@ public abstract class AbstractKinesisFirehoseProcessor extends AbstractBaseKines
public static final PropertyDescriptor KINESIS_FIREHOSE_DELIVERY_STREAM_NAME = new PropertyDescriptor.Builder()
.name("Amazon Kinesis Firehose Delivery Stream Name")
.description("The name of kinesis firehose delivery stream")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -72,6 +73,7 @@ public abstract class AbstractKinesisFirehoseProcessor extends AbstractBaseKines
*
* @deprecated use {@link #createClient(ProcessContext, AWSCredentialsProvider, ClientConfiguration)} instead
*/
@Deprecated
@Override
protected AmazonKinesisFirehoseClient createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
getLogger().info("Creating client using aws credentials");

View File

@ -17,6 +17,7 @@
package org.apache.nifi.processors.aws.kinesis.stream;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.kinesis.AbstractBaseKinesisProcessor;
@ -35,7 +36,7 @@ public abstract class AbstractKinesisStreamProcessor extends AbstractBaseKinesis
.name("kinesis-stream-name")
.displayName("Amazon Kinesis Stream Name")
.description("The name of Kinesis Stream")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();

View File

@ -17,6 +17,7 @@
package org.apache.nifi.processors.aws.lambda;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
@ -34,7 +35,7 @@ public abstract class AbstractAWSLambdaProcessor extends AbstractAWSCredentialsP
public static final PropertyDescriptor AWS_LAMBDA_FUNCTION_NAME = new PropertyDescriptor.Builder()
.name("Amazon Lambda Name")
.description("The Lambda Function Name")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -43,7 +44,7 @@ public abstract class AbstractAWSLambdaProcessor extends AbstractAWSCredentialsP
.name("Amazon Lambda Qualifier (version)")
.description("The Lambda Function Version")
.defaultValue("$LATEST")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();

View File

@ -23,6 +23,7 @@ import java.util.List;
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;
@ -47,7 +48,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
public static final PropertyDescriptor FULL_CONTROL_USER_LIST = new PropertyDescriptor.Builder()
.name("FullControl User List")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.description("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have Full Control for an object")
.defaultValue("${s3.permissions.full.users}")
@ -55,7 +56,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
public static final PropertyDescriptor READ_USER_LIST = new PropertyDescriptor.Builder()
.name("Read Permission User List")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.description("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have Read Access for an object")
.defaultValue("${s3.permissions.read.users}")
@ -63,7 +64,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
public static final PropertyDescriptor WRITE_USER_LIST = new PropertyDescriptor.Builder()
.name("Write Permission User List")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.description("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have Write Access for an object")
.defaultValue("${s3.permissions.write.users}")
@ -71,7 +72,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
public static final PropertyDescriptor READ_ACL_LIST = new PropertyDescriptor.Builder()
.name("Read ACL User List")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.description("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have permissions to read the Access Control List for an object")
.defaultValue("${s3.permissions.readacl.users}")
@ -79,7 +80,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
public static final PropertyDescriptor WRITE_ACL_LIST = new PropertyDescriptor.Builder()
.name("Write ACL User List")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.description("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have permissions to change the Access Control List for an object")
.defaultValue("${s3.permissions.writeacl.users}")
@ -88,7 +89,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
.name("canned-acl")
.displayName("Canned ACL")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.description("Amazon Canned ACL for an object, one of: BucketOwnerFullControl, BucketOwnerRead, LogDeliveryWrite, AuthenticatedRead, PublicReadWrite, PublicRead, Private; " +
"will be ignored if any other ACL/permission/owner property is specified")
@ -97,14 +98,14 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
public static final PropertyDescriptor OWNER = new PropertyDescriptor.Builder()
.name("Owner")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.description("The Amazon ID to use for the object's owner")
.defaultValue("${s3.owner}")
.build();
public static final PropertyDescriptor BUCKET = new PropertyDescriptor.Builder()
.name("Bucket")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -112,7 +113,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
.name("Object Key")
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.defaultValue("${filename}")
.build();
public static final PropertyDescriptor SIGNER_OVERRIDE = new PropertyDescriptor.Builder()

View File

@ -18,6 +18,7 @@ package org.apache.nifi.processors.aws.sns;
import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
@ -37,7 +38,7 @@ public abstract class AbstractSNSProcessor extends AbstractAWSCredentialsProvide
public static final PropertyDescriptor ARN = new PropertyDescriptor.Builder()
.name("Amazon Resource Name (ARN)")
.description("The name of the resource to which notifications should be published")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -45,7 +46,7 @@ public abstract class AbstractSNSProcessor extends AbstractAWSCredentialsProvide
public static final PropertyDescriptor ARN_TYPE = new PropertyDescriptor.Builder()
.name("ARN Type")
.description("The type of Amazon Resource Name that is being used.")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.allowableValues(ARN_TYPE_TOPIC, ARN_TYPE_TARGET)
.defaultValue(ARN_TYPE_TOPIC.getValue())

View File

@ -17,6 +17,7 @@
package org.apache.nifi.processors.aws.sqs;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
@ -40,7 +41,7 @@ public abstract class AbstractSQSProcessor extends AbstractAWSCredentialsProvide
.name("Queue URL")
.description("The URL of the queue to act upon")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.build();

View File

@ -37,6 +37,7 @@ import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator;
import org.apache.nifi.expression.AttributeExpression;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
@ -62,7 +63,7 @@ import com.amazonaws.services.cloudwatch.model.StatisticSet;
"minimum, maximum, sum and sample count.")
@DynamicProperty(name = "Dimension Name", value = "Dimension Value",
description = "Allows dimension name/value pairs to be added to the metric. AWS supports a maximum of 10 dimensions.",
supportsExpressionLanguage = true)
expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
@Tags({"amazon", "aws", "cloudwatch", "metrics", "put", "publish"})
public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor<AmazonCloudWatchClient> {
@ -93,7 +94,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
.displayName("Namespace")
.description("The namespace for the metric data for CloudWatch")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -101,7 +102,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
.name("MetricName")
.displayName("Metric Name")
.description("The name of the metric")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.addValidator(new StandardValidators.StringLengthValidator(1, 255))
.build();
@ -110,7 +111,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
.name("Value")
.displayName("Value")
.description("The value for the metric. Must be a double")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(false)
.addValidator(DOUBLE_VALIDATOR)
.build();
@ -119,7 +120,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
.name("Timestamp")
.displayName("Timestamp")
.description("A point in time expressed as the number of milliseconds since Jan 1, 1970 00:00:00 UTC. If not specified, the default value is set to the time the metric data was received")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(false)
.addValidator(StandardValidators.LONG_VALIDATOR)
.build();
@ -128,7 +129,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
.name("Unit")
.displayName("Unit")
.description("The unit of the metric. (e.g Seconds, Bytes, Megabytes, Percent, Count, Kilobytes/Second, Terabits/Second, Count/Second) For details see http://docs.aws.amazon.com/AmazonCloudWatch/latest/APIReference/API_MetricDatum.html")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -137,7 +138,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
.name("maximum")
.displayName("Maximum")
.description("The maximum value of the sample set. Must be a double")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(false)
.addValidator(DOUBLE_VALIDATOR)
.build();
@ -146,7 +147,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
.name("minimum")
.displayName("Minimum")
.description("The minimum value of the sample set. Must be a double")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(false)
.addValidator(DOUBLE_VALIDATOR)
.build();
@ -155,7 +156,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
.name("sampleCount")
.displayName("Sample Count")
.description("The number of samples used for the statistic set. Must be a double")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(false)
.addValidator(DOUBLE_VALIDATOR)
.build();
@ -164,7 +165,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
.name("sum")
.displayName("Sum")
.description("The sum of values for the sample set. Must be a double")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(false)
.addValidator(DOUBLE_VALIDATOR)
.build();
@ -188,7 +189,7 @@ public class PutCloudWatchMetric extends AbstractAWSCredentialsProviderProcessor
return new PropertyDescriptor.Builder()
.name(propertyDescriptorName)
.addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.dynamic(true)
.build();
}

View File

@ -35,6 +35,7 @@ import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.ProcessContext;
@ -72,7 +73,7 @@ public class PutKinesisStream extends AbstractKinesisStreamProcessor {
.displayName("Amazon Kinesis Stream Partition Key")
.name("amazon-kinesis-stream-partition-key")
.description("The partition key attribute. If it is not set, a random value is used")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.defaultValue("${kinesis.partition.key}")
.required(false)
.addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR).build();

View File

@ -33,6 +33,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
@ -51,7 +52,7 @@ public class DeleteS3Object extends AbstractS3Processor {
.name("Version")
.description("The Version of the Object to delete")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(false)
.build();

View File

@ -33,6 +33,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.processor.ProcessContext;
@ -69,7 +70,7 @@ public class FetchS3Object extends AbstractS3Processor {
.name("Version")
.description("The Version of the Object to download")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(false)
.build();

View File

@ -40,6 +40,7 @@ import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.components.state.StateMap;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.processor.ProcessContext;
@ -84,7 +85,7 @@ public class ListS3 extends AbstractS3Processor {
public static final PropertyDescriptor DELIMITER = new PropertyDescriptor.Builder()
.name("delimiter")
.displayName("Delimiter")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.description("The string used to delimit directories within the bucket. Please consult the AWS documentation " +
@ -94,7 +95,7 @@ public class ListS3 extends AbstractS3Processor {
public static final PropertyDescriptor PREFIX = new PropertyDescriptor.Builder()
.name("prefix")
.displayName("Prefix")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.description("The prefix used to filter the object list. In most cases, it should end with a forward slash ('/').")
@ -103,7 +104,7 @@ public class ListS3 extends AbstractS3Processor {
public static final PropertyDescriptor USE_VERSIONS = new PropertyDescriptor.Builder()
.name("use-versions")
.displayName("Use Versions")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.addValidator(StandardValidators.BOOLEAN_VALIDATOR)
.allowableValues("true", "false")
@ -114,7 +115,7 @@ public class ListS3 extends AbstractS3Processor {
public static final PropertyDescriptor LIST_TYPE = new PropertyDescriptor.Builder()
.name("list-type")
.displayName("List Type")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.addValidator(StandardValidators.INTEGER_VALIDATOR)
.allowableValues(

View File

@ -50,6 +50,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.processor.DataUnit;
@ -104,7 +105,7 @@ import com.amazonaws.services.s3.model.UploadPartResult;
@DynamicProperty(name = "The name of a User-Defined Metadata field to add to the S3 Object",
value = "The value of a User-Defined Metadata field to add to the S3 Object",
description = "Allows user-defined metadata to be added to the S3 object as key/value pairs",
supportsExpressionLanguage = true)
expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
@ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the S3 object")
@WritesAttributes({
@WritesAttribute(attribute = "s3.bucket", description = "The S3 bucket where the Object was put in S3"),
@ -129,7 +130,7 @@ public class PutS3Object extends AbstractS3Processor {
public static final PropertyDescriptor EXPIRATION_RULE_ID = new PropertyDescriptor.Builder()
.name("Expiration Time Rule")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -143,7 +144,7 @@ public class PutS3Object extends AbstractS3Processor {
"no content type is provided and cannot be determined by the filename, the default content type " +
"\"application/octet-stream\" will be used.")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -237,7 +238,7 @@ public class PutS3Object extends AbstractS3Processor {
return new PropertyDescriptor.Builder()
.name(propertyDescriptorName)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.dynamic(true)
.build();
}
@ -408,7 +409,7 @@ public class PutS3Object extends AbstractS3Processor {
/*
* If necessary, run age off for existing uploads in AWS S3 and local state
*/
ageoffS3Uploads(context, s3, now);
ageoffS3Uploads(context, s3, now, bucket);
/*
* Then
@ -717,16 +718,15 @@ public class PutS3Object extends AbstractS3Processor {
private final AtomicLong lastS3AgeOff = new AtomicLong(0L);
private final DateFormat logFormat = new SimpleDateFormat();
protected void ageoffS3Uploads(final ProcessContext context, final AmazonS3Client s3, final long now) {
MultipartUploadListing oldUploads = getS3AgeoffListAndAgeoffLocalState(context, s3, now);
protected void ageoffS3Uploads(final ProcessContext context, final AmazonS3Client s3, final long now, String bucket) {
MultipartUploadListing oldUploads = getS3AgeoffListAndAgeoffLocalState(context, s3, now, bucket);
for (MultipartUpload upload : oldUploads.getMultipartUploads()) {
abortS3MultipartUpload(s3, oldUploads.getBucketName(), upload);
}
}
protected MultipartUploadListing getS3AgeoffListAndAgeoffLocalState(final ProcessContext context, final AmazonS3Client s3, final long now) {
protected MultipartUploadListing getS3AgeoffListAndAgeoffLocalState(final ProcessContext context, final AmazonS3Client s3, final long now, String bucket) {
final long ageoff_interval = context.getProperty(MULTIPART_S3_AGEOFF_INTERVAL).asTimePeriod(TimeUnit.MILLISECONDS);
final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions().getValue();
final Long maxAge = context.getProperty(MULTIPART_S3_MAX_AGE).asTimePeriod(TimeUnit.MILLISECONDS);
final long ageCutoff = now - maxAge;

View File

@ -31,6 +31,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
@ -53,7 +54,7 @@ public class PutSNS extends AbstractSNSProcessor {
.name("Character Set")
.description("The character set in which the FlowFile's content is encoded")
.defaultValue("UTF-8")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
.required(true)
.build();
@ -69,7 +70,7 @@ public class PutSNS extends AbstractSNSProcessor {
public static final PropertyDescriptor SUBJECT = new PropertyDescriptor.Builder()
.name("E-mail Subject")
.description("The optional subject to use for any subscribers that are subscribed via E-mail")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -90,7 +91,7 @@ public class PutSNS extends AbstractSNSProcessor {
return new PropertyDescriptor.Builder()
.name(propertyDescriptorName)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(false)
.dynamic(true)
.build();

View File

@ -28,6 +28,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
@ -47,7 +48,7 @@ public class DeleteSQS extends AbstractSQSProcessor {
public static final PropertyDescriptor RECEIPT_HANDLE = new PropertyDescriptor.Builder()
.name("Receipt Handle")
.description("The identifier that specifies the receipt of the message")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.defaultValue("${sqs.receipt.handle}")

View File

@ -38,6 +38,7 @@ import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.AttributeExpression.ResultType;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
@ -85,7 +86,7 @@ public class GetSQS extends AbstractSQSProcessor {
public static final PropertyDescriptor VISIBILITY_TIMEOUT = new PropertyDescriptor.Builder()
.name("Visibility Timeout")
.description("The amount of time after a message is received but not deleted that the message is hidden from other consumers")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.defaultValue("15 mins")
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
@ -101,7 +102,7 @@ public class GetSQS extends AbstractSQSProcessor {
public static final PropertyDescriptor DYNAMIC_QUEUE_URL = new PropertyDescriptor.Builder()
.fromPropertyDescriptor(QUEUE_URL)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
.build();
@ -109,7 +110,7 @@ public class GetSQS extends AbstractSQSProcessor {
.name("Receive Message Wait Time")
.description("The maximum amount of time to wait on a long polling receive call. Setting this to a value of 1 second or greater will "
+ "reduce the number of SQS requests and decrease fetch latency at the cost of a constantly active thread.")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.defaultValue("0 sec")
.addValidator(StandardValidators.createTimePeriodValidator(0, TimeUnit.SECONDS, 20, TimeUnit.SECONDS)) // 20 seconds is the maximum allowed by SQS

View File

@ -36,6 +36,7 @@ import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
@ -53,7 +54,7 @@ import com.amazonaws.services.sqs.model.SendMessageBatchRequestEntry;
@CapabilityDescription("Publishes a message to an Amazon Simple Queuing Service Queue")
@DynamicProperty(name = "The name of a Message Attribute to add to the message", value = "The value of the Message Attribute",
description = "Allows the user to add key/value pairs as Message Attributes by adding a property whose name will become the name of "
+ "the Message Attribute and value will become the value of the Message Attribute", supportsExpressionLanguage = true)
+ "the Message Attribute and value will become the value of the Message Attribute", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
public class PutSQS extends AbstractSQSProcessor {
public static final PropertyDescriptor DELAY = new PropertyDescriptor.Builder()
@ -78,7 +79,7 @@ public class PutSQS extends AbstractSQSProcessor {
protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
return new PropertyDescriptor.Builder()
.name(propertyDescriptorName)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(false)
.dynamic(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)

View File

@ -16,6 +16,8 @@
*/
package org.apache.nifi.processors.aws.cloudwatch;
import java.io.IOException;
import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
import org.apache.nifi.processors.aws.sns.PutSNS;
@ -23,8 +25,6 @@ import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Test;
import java.io.IOException;
/**
* Provides integration level testing with actual AWS CloudWatch resources for {@link PutCloudWatchMetric} and requires additional configuration and resources to work.
*/
@ -50,7 +50,6 @@ public class ITPutCloudWatchMetric {
@Test
public void testPublishWithCredentialsProviderService() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(new PutCloudWatchMetric());
runner.setValidateExpressionUsage(false);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);

View File

@ -822,13 +822,13 @@ public class ITPutS3Object extends AbstractS3IT {
// initiation times in whole seconds.
Long now = System.currentTimeMillis();
MultipartUploadListing uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now);
MultipartUploadListing uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now, BUCKET_NAME);
Assert.assertEquals(3, uploadList.getMultipartUploads().size());
MultipartUpload upload0 = uploadList.getMultipartUploads().get(0);
processor.abortS3MultipartUpload(client, BUCKET_NAME, upload0);
uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now+1000);
uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now+1000, BUCKET_NAME);
Assert.assertEquals(2, uploadList.getMultipartUploads().size());
final Map<String, String> attrs = new HashMap<>();
@ -836,7 +836,7 @@ public class ITPutS3Object extends AbstractS3IT {
runner.enqueue(getResourcePath(SAMPLE_FILE_RESOURCE_NAME), attrs);
runner.run();
uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now+2000);
uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now+2000, BUCKET_NAME);
Assert.assertEquals(0, uploadList.getMultipartUploads().size());
}

View File

@ -16,18 +16,18 @@
*/
package org.apache.nifi.processors.aws.sns;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Test;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.nio.file.Paths;
import java.util.HashMap;
import java.util.Map;
import static org.junit.Assert.assertTrue;
import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Test;
/**
* Provides integration level testing with actual AWS S3 resources for {@link PutSNS} and requires additional configuration and resources to work.
@ -39,7 +39,6 @@ public class ITPutSNS {
@Test
public void testPublish() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new PutSNS());
runner.setValidateExpressionUsage(false);
runner.setProperty(PutSNS.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutSNS.ARN, "arn:aws:sns:us-west-2:100515378163:test-topic-1");
assertTrue(runner.setProperty("DynamicProperty", "hello!").isValid());
@ -56,7 +55,6 @@ public class ITPutSNS {
public void testPublishWithCredentialsProviderService() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(new PutSNS());
String snsArn = "Add Sns arn here";
runner.setValidateExpressionUsage(false);
runner.setProperty(PutSNS.ARN, snsArn);
assertTrue(runner.setProperty("DynamicProperty", "hello!").isValid());
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();

View File

@ -16,6 +16,9 @@
*/
package org.apache.nifi.processors.aws.sns;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
@ -25,19 +28,15 @@ import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import com.amazonaws.services.sns.AmazonSNSClient;
import com.amazonaws.services.sns.model.AmazonSNSException;
import com.amazonaws.services.sns.model.PublishRequest;
import com.amazonaws.services.sns.model.PublishResult;
import org.junit.Before;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.mockito.Mockito;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import com.amazonaws.services.sns.AmazonSNSClient;
import com.amazonaws.services.sns.model.AmazonSNSException;
import com.amazonaws.services.sns.model.PublishRequest;
import com.amazonaws.services.sns.model.PublishResult;
public class TestPutSNS {
@ -51,6 +50,7 @@ public class TestPutSNS {
public void setUp() {
mockSNSClient = Mockito.mock(AmazonSNSClient.class);
mockPutSNS = new PutSNS() {
@Override
protected AmazonSNSClient getClient() {
actualSNSClient = client;
return mockSNSClient;
@ -61,7 +61,6 @@ public class TestPutSNS {
@Test
public void testPublish() throws IOException {
runner.setValidateExpressionUsage(false);
runner.setProperty(PutSNS.CREDENTIALS_FILE, "src/test/resources/mock-aws-credentials.properties");
runner.setProperty(PutSNS.ARN, "arn:aws:sns:us-west-2:123456789012:test-topic-1");
runner.setProperty(PutSNS.SUBJECT, "${eval.subject}");
@ -92,7 +91,6 @@ public class TestPutSNS {
@Test
public void testPublishFailure() throws IOException {
runner.setValidateExpressionUsage(false);
runner.setProperty(PutSNS.ARN, "arn:aws:sns:us-west-2:123456789012:test-topic-1");
final Map<String, String> ffAttributes = new HashMap<>();
ffAttributes.put("filename", "1.txt");

View File

@ -60,7 +60,6 @@ public class ITPutSQS {
runner.setProperty(PutSQS.TIMEOUT, "30 secs");
String queueUrl = "Add queue url here";
runner.setProperty(PutSQS.QUEUE_URL, queueUrl);
runner.setValidateExpressionUsage(false);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);

View File

@ -16,25 +16,24 @@
*/
package org.apache.nifi.processors.aws.sqs;
import static org.junit.Assert.assertEquals;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import com.amazonaws.services.sqs.AmazonSQSClient;
import com.amazonaws.services.sqs.model.AmazonSQSException;
import com.amazonaws.services.sqs.model.SendMessageBatchRequest;
import com.amazonaws.services.sqs.model.SendMessageBatchResult;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.mockito.Mockito;
import static org.junit.Assert.assertEquals;
import com.amazonaws.services.sqs.AmazonSQSClient;
import com.amazonaws.services.sqs.model.AmazonSQSException;
import com.amazonaws.services.sqs.model.SendMessageBatchRequest;
import com.amazonaws.services.sqs.model.SendMessageBatchResult;
public class TestPutSQS {
@ -48,6 +47,7 @@ public class TestPutSQS {
public void setUp() {
mockSQSClient = Mockito.mock(AmazonSQSClient.class);
mockPutSQS = new PutSQS() {
@Override
protected AmazonSQSClient getClient() {
actualSQSClient = client;
return mockSQSClient;
@ -58,7 +58,6 @@ public class TestPutSQS {
@Test
public void testSimplePut() throws IOException {
runner.setValidateExpressionUsage(false);
runner.setProperty(PutSQS.QUEUE_URL, "https://sqs.us-west-2.amazonaws.com/123456789012/test-queue-000000000");
Assert.assertTrue(runner.setProperty("x-custom-prop", "hello").isValid());
@ -83,7 +82,6 @@ public class TestPutSQS {
@Test
public void testPutException() throws IOException {
runner.setValidateExpressionUsage(false);
runner.setProperty(PutSQS.QUEUE_URL, "https://sqs.us-west-2.amazonaws.com/123456789012/test-queue-000000000");
final Map<String, String> attrs = new HashMap<>();

View File

@ -19,6 +19,7 @@ package org.apache.nifi.processors.azure;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.util.StandardValidators;
@ -33,10 +34,24 @@ import java.util.Set;
public abstract class AbstractAzureBlobProcessor extends AbstractProcessor {
public static final PropertyDescriptor BLOB = new PropertyDescriptor.Builder().name("blob").displayName("Blob").description("The filename of the blob")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR).expressionLanguageSupported(true).required(true).defaultValue("${azure.blobname}").build();
public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success").description("All successfully processed FlowFiles are routed to this relationship").build();
public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure").description("Unsuccessful operations will be transferred to the failure relationship.").build();
public static final PropertyDescriptor BLOB = new PropertyDescriptor.Builder()
.name("blob")
.displayName("Blob")
.description("The filename of the blob")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.defaultValue("${azure.blobname}")
.build();
public static final Relationship REL_SUCCESS = new Relationship.Builder()
.name("success")
.description("All successfully processed FlowFiles are routed to this relationship")
.build();
public static final Relationship REL_FAILURE = new Relationship.Builder()
.name("failure")
.description("Unsuccessful operations will be transferred to the failure relationship.")
.build();
private static final List<PropertyDescriptor> PROPERTIES = Collections
.unmodifiableList(Arrays.asList(

View File

@ -37,6 +37,7 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.logging.ComponentLog;
@ -95,7 +96,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
.displayName("Event Hub Namespace")
.description("The Azure Namespace that the Event Hub is assigned to. This is generally equal to <Event Hub Name>-ns.")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.required(true)
.build();
static final PropertyDescriptor EVENT_HUB_NAME = new PropertyDescriptor.Builder()
@ -103,7 +104,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
.displayName("Event Hub Name")
.description("The name of the Azure Event Hub to pull messages from.")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.required(true)
.build();
// TODO: Do we need to support custom service endpoints as GetAzureEventHub does? Is it possible?
@ -112,7 +113,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
.displayName("Shared Access Policy Name")
.description("The name of the Event Hub Shared Access Policy. This Policy must have Listen permissions.")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.required(true)
.build();
static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
@ -120,7 +121,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
.displayName("Shared Access Policy Primary Key")
.description("The primary key of the Event Hub Shared Access Policy.")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.sensitive(true)
.required(true)
.build();
@ -129,7 +130,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
.displayName("Event Hub Consumer Group")
.description("The name of the Event Hub Consumer Group to use.")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.defaultValue("$Default")
.required(true)
.build();
@ -139,7 +140,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
.description("The hostname of this Event Hub Consumer instance." +
" If not specified, an unique identifier is generated in 'nifi-<UUID>' format.")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.required(false)
.build();
@ -149,7 +150,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
.description("The Record Reader to use for reading received messages." +
" The Event Hub name can be referred by Expression Language '${eventhub.name}' to access a schema.")
.identifiesControllerService(RecordReaderFactory.class)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(false)
.build();
static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
@ -159,7 +160,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
" The Event Hub name can be referred by Expression Language '${eventhub.name}' to access a schema." +
" If not specified, each message will create a FlowFile.")
.identifiesControllerService(RecordSetWriterFactory.class)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(false)
.build();
@ -188,7 +189,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
" https://github.com/Azure/azure-event-hubs-java/issues/125")
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.defaultValue("300")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.required(true)
.build();
static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
@ -202,7 +203,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
" The higher number, the higher throughput, but possibly less consistent.")
.defaultValue("10")
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.required(true)
.build();
static final PropertyDescriptor RECEIVE_TIMEOUT = new PropertyDescriptor.Builder()
@ -211,7 +212,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
.description("The amount of time this consumer should wait to receive the Prefetch Count before returning.")
.defaultValue("1 min")
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.required(true)
.build();
static final PropertyDescriptor STORAGE_ACCOUNT_NAME = new PropertyDescriptor.Builder()
@ -219,7 +220,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
.displayName("Storage Account Name")
.description("Name of the Azure Storage account to store Event Hub Consumer Group state.")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.required(true)
.build();
static final PropertyDescriptor STORAGE_ACCOUNT_KEY = new PropertyDescriptor.Builder()
@ -228,7 +229,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
.description("The Azure Storage account key to store Event Hub Consumer Group state.")
.sensitive(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.required(true)
.build();
static final PropertyDescriptor STORAGE_CONTAINER_NAME = new PropertyDescriptor.Builder()
@ -237,7 +238,7 @@ public class ConsumeAzureEventHub extends AbstractSessionFactoryProcessor {
.description("Name of the Azure Storage Container to store Event Hub Consumer Group state." +
" If not specified, Event Hub name is used.")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.required(false)
.build();

View File

@ -30,6 +30,7 @@ import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
@ -80,14 +81,14 @@ public class GetAzureEventHub extends AbstractProcessor {
.name("Event Hub Namespace")
.description("The Azure Namespace that the Event Hub is assigned to. This is generally equal to <Event Hub Name>-ns")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.build();
static final PropertyDescriptor SERVICE_BUS_ENDPOINT = new PropertyDescriptor.Builder()
.name("Service Bus Endpoint")
.description("To support Namespaces in non-standard Host URIs ( not .servicebus.windows.net, ie .servicebus.chinacloudapi.cn) select from the drop down acceptable options ")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.allowableValues(".servicebus.windows.net",".servicebus.chinacloudapi.cn")
.defaultValue(".servicebus.windows.net")
.required(true)
@ -96,14 +97,14 @@ public class GetAzureEventHub extends AbstractProcessor {
.name("Shared Access Policy Name")
.description("The name of the Event Hub Shared Access Policy. This Policy must have Listen permissions.")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.build();
static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
.name("Shared Access Policy Primary Key")
.description("The primary key of the Event Hub Shared Access Policy")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.sensitive(true)
.required(true)
.build();
@ -113,14 +114,14 @@ public class GetAzureEventHub extends AbstractProcessor {
.description("The number of partitions that the Event Hub has. Only this number of partitions will be used, "
+ "so it is important to ensure that if the number of partitions changes that this value be updated. Otherwise, some messages may not be consumed.")
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.build();
static final PropertyDescriptor CONSUMER_GROUP = new PropertyDescriptor.Builder()
.name("Event Hub Consumer Group")
.description("The name of the Event Hub Consumer Group to use when pulling events")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.defaultValue("$Default")
.required(true)
.build();
@ -130,21 +131,21 @@ public class GetAzureEventHub extends AbstractProcessor {
.description("A timestamp (ISO-8061 Instant) formatted as YYYY-MM-DDThhmmss.sssZ (2016-01-01T01:01:01.000Z) from which messages "
+ "should have been enqueued in the EventHub to start reading from")
.addValidator(StandardValidators.ISO8061_INSTANT_VALIDATOR)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(false)
.build();
static final PropertyDescriptor RECEIVER_FETCH_SIZE = new PropertyDescriptor.Builder()
.name("Partition Recivier Fetch Size")
.description("The number of events that a receiver should fetch from an EventHubs partition before returning. Default(100)")
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(false)
.build();
static final PropertyDescriptor RECEIVER_FETCH_TIMEOUT = new PropertyDescriptor.Builder()
.name("Partiton Receiver Timeout (millseconds)")
.description("The amount of time a Partition Receiver should wait to receive the Fetch Size before returning. Default(60000)")
.addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(false)
.build();

View File

@ -31,6 +31,7 @@ import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
@ -70,21 +71,21 @@ public class PutAzureEventHub extends AbstractProcessor {
.name("Event Hub Namespace")
.description("The Azure Namespace that the Event Hub is assigned to. This is generally equal to <Event Hub Name>-ns")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.build();
static final PropertyDescriptor ACCESS_POLICY = new PropertyDescriptor.Builder()
.name("Shared Access Policy Name")
.description("The name of the Event Hub Shared Access Policy. This Policy must have Send permissions.")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.build();
static final PropertyDescriptor POLICY_PRIMARY_KEY = new PropertyDescriptor.Builder()
.name("Shared Access Policy Primary Key")
.description("The primary key of the Event Hub Shared Access Policy")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.sensitive(true)
.required(true)
.build();

View File

@ -38,6 +38,7 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processor.util.list.AbstractListProcessor;
@ -78,9 +79,14 @@ import java.util.Map;
"where the previous node left off, without duplicating the data.")
public class ListAzureBlobStorage extends AbstractListProcessor<BlobInfo> {
private static final PropertyDescriptor PROP_PREFIX = new PropertyDescriptor.Builder().name("prefix").displayName("Prefix").description("Search prefix for listing")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR).expressionLanguageSupported(true).required(false).build();
private static final PropertyDescriptor PROP_PREFIX = new PropertyDescriptor.Builder()
.name("prefix")
.displayName("Prefix")
.description("Search prefix for listing")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.required(false)
.build();
private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
AzureStorageUtils.CONTAINER,

View File

@ -24,6 +24,7 @@ import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.components.PropertyDescriptor;
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.ProcessContext;
@ -40,7 +41,9 @@ public final class AzureStorageUtils {
public static final String BLOCK = "Block";
public static final String PAGE = "Page";
public static final PropertyDescriptor ACCOUNT_KEY = new PropertyDescriptor.Builder().name("storage-account-key").displayName("Storage Account Key")
public static final PropertyDescriptor ACCOUNT_KEY = new PropertyDescriptor.Builder()
.name("storage-account-key")
.displayName("Storage Account Key")
.description("The storage account key. This is an admin-like password providing access to every container in this account. It is recommended " +
"one uses Shared Access Signature (SAS) token instead for fine-grained control with policies. " +
"There are certain risks in allowing the account key to be stored as a flowfile " +
@ -48,18 +51,34 @@ public final class AzureStorageUtils {
"be fetched dynamically from a flow file attribute, care must be taken to restrict access to " +
"the event provenance data (e.g. by strictly controlling the policies governing provenance for this Processor). " +
"In addition, the provenance repositories may be put on encrypted disk partitions.")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR).expressionLanguageSupported(true).required(false).sensitive(true).build();
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(false)
.sensitive(true)
.build();
public static final PropertyDescriptor ACCOUNT_NAME = new PropertyDescriptor.Builder().name("storage-account-name").displayName("Storage Account Name")
public static final PropertyDescriptor ACCOUNT_NAME = new PropertyDescriptor.Builder()
.name("storage-account-name")
.displayName("Storage Account Name")
.description("The storage account name. There are certain risks in allowing the account name to be stored as a flowfile " +
"attribute. While it does provide for a more flexible flow by allowing the account name to " +
"be fetched dynamically from a flowfile attribute, care must be taken to restrict access to " +
"the event provenance data (e.g. by strictly controlling the policies governing provenance for this Processor). " +
"In addition, the provenance repositories may be put on encrypted disk partitions.")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR).expressionLanguageSupported(true).required(true).sensitive(true).build();
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.sensitive(true)
.build();
public static final PropertyDescriptor CONTAINER = new PropertyDescriptor.Builder().name("container-name").displayName("Container Name")
.description("Name of the Azure storage container").addValidator(StandardValidators.NON_EMPTY_VALIDATOR).expressionLanguageSupported(true).required(true).build();
public static final PropertyDescriptor CONTAINER = new PropertyDescriptor.Builder()
.name("container-name")
.displayName("Container Name")
.description("Name of the Azure storage container")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.build();
public static final PropertyDescriptor PROP_SAS_TOKEN = new PropertyDescriptor.Builder()
.name("storage-sas-token")
@ -71,7 +90,7 @@ public final class AzureStorageUtils {
"the event provenance data (e.g. by strictly controlling the policies governing provenance for this Processor). " +
"In addition, the provenance repositories may be put on encrypted disk partitions.")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.sensitive(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();

View File

@ -16,18 +16,19 @@
*/
package org.apache.nifi.processors.azure.storage;
import com.microsoft.azure.storage.StorageException;
import com.microsoft.azure.storage.blob.CloudBlobContainer;
import java.io.IOException;
import java.net.URISyntaxException;
import java.security.InvalidKeyException;
import java.util.UUID;
import org.apache.nifi.processors.azure.AbstractAzureBlobStorageIT;
import org.apache.nifi.processors.azure.storage.utils.AzureStorageUtils;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Test;
import java.io.IOException;
import java.net.URISyntaxException;
import java.security.InvalidKeyException;
import java.util.UUID;
import com.microsoft.azure.storage.StorageException;
import com.microsoft.azure.storage.blob.CloudBlobContainer;
public class ITDeleteAzureBlobStorage extends AbstractAzureBlobStorageIT{
@ -42,8 +43,6 @@ public class ITDeleteAzureBlobStorage extends AbstractAzureBlobStorageIT{
final TestRunner runner = TestRunners.newTestRunner(DeleteAzureBlobStorage.class);
try {
runner.setValidateExpressionUsage(true);
runner.setProperty(AzureStorageUtils.ACCOUNT_NAME, AzureTestUtil.getAccountName());
runner.setProperty(AzureStorageUtils.ACCOUNT_KEY, AzureTestUtil.getAccountKey());
runner.setProperty(AzureStorageUtils.CONTAINER, containerName);

View File

@ -53,8 +53,6 @@ public class ITFetchAzureBlobStorage {
final TestRunner runner = TestRunners.newTestRunner(new FetchAzureBlobStorage());
try {
runner.setValidateExpressionUsage(true);
runner.setProperty(AzureStorageUtils.ACCOUNT_NAME, AzureTestUtil.getAccountName());
runner.setProperty(AzureStorageUtils.ACCOUNT_KEY, AzureTestUtil.getAccountKey());
runner.setProperty(AzureStorageUtils.CONTAINER, containerName);

View File

@ -42,8 +42,6 @@ public class ITPutAzureStorageBlob {
final TestRunner runner = TestRunners.newTestRunner(new PutAzureBlobStorage());
try {
runner.setValidateExpressionUsage(true);
runner.setProperty(AzureStorageUtils.ACCOUNT_NAME, AzureTestUtil.getAccountName());
runner.setProperty(AzureStorageUtils.ACCOUNT_KEY, AzureTestUtil.getAccountKey());
runner.setProperty(AzureStorageUtils.CONTAINER, containerName);

View File

@ -33,6 +33,7 @@ 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.expression.ExpressionLanguageScope;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
@ -65,7 +66,7 @@ public abstract class AbstractCassandraProcessor extends AbstractProcessor {
+ "comma-separated and in hostname:port format. Example node1:port,node2:port,...."
+ " The default client port for Cassandra is 9042, but the port(s) must be explicitly specified.")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
.build();
@ -74,7 +75,7 @@ public abstract class AbstractCassandraProcessor extends AbstractProcessor {
.description("The Cassandra Keyspace to connect to. If no keyspace is specified, the query will need to "
+ "include the keyspace name before any table reference.")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -100,7 +101,7 @@ public abstract class AbstractCassandraProcessor extends AbstractProcessor {
.name("Username")
.description("Username to access the Cassandra cluster")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -109,7 +110,7 @@ public abstract class AbstractCassandraProcessor extends AbstractProcessor {
.description("Password to access the Cassandra cluster")
.required(false)
.sensitive(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -125,7 +126,7 @@ public abstract class AbstractCassandraProcessor extends AbstractProcessor {
.name("Character Set")
.description("Specifies the character set of the record data.")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.defaultValue("UTF-8")
.addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
.build();

View File

@ -40,6 +40,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnShutdown;
import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
@ -98,7 +99,7 @@ public class PutCassandraQL extends AbstractCassandraProcessor {
+ "Time Unit, such as: nanos, millis, secs, mins, hrs, days. A value of zero means there is no limit. ")
.defaultValue("0 seconds")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.build();

View File

@ -40,6 +40,7 @@ import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.lifecycle.OnShutdown;
import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.annotation.behavior.WritesAttribute;
@ -95,7 +96,7 @@ public class QueryCassandra extends AbstractCassandraProcessor {
.description("CQL select query")
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.build();
public static final PropertyDescriptor QUERY_TIMEOUT = new PropertyDescriptor.Builder()
@ -105,7 +106,7 @@ public class QueryCassandra extends AbstractCassandraProcessor {
+ "Time Unit, such as: nanos, millis, secs, mins, hrs, days. A value of zero means there is no limit. ")
.defaultValue("0 seconds")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.build();
@ -115,7 +116,7 @@ public class QueryCassandra extends AbstractCassandraProcessor {
+ "and means there is no limit.")
.defaultValue("0")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.INTEGER_VALIDATOR)
.build();

View File

@ -63,6 +63,7 @@ import org.apache.nifi.components.state.StateMap;
import org.apache.nifi.distributed.cache.client.Deserializer;
import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
import org.apache.nifi.distributed.cache.client.Serializer;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
import org.apache.nifi.processor.ProcessContext;
@ -174,7 +175,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
.defaultValue("30 seconds")
.required(true)
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor HOSTS = new PropertyDescriptor.Builder()
@ -185,9 +186,8 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
+ "the hosts in the list in order. If one node goes down and failover is enabled for the cluster, then the processor will connect "
+ "to the active node (assuming its host entry is specified in this property. The default port for MySQL connections is 3306.")
.required(true)
.expressionLanguageSupported(false)
.addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor DRIVER_NAME = new PropertyDescriptor.Builder()
@ -197,7 +197,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
.defaultValue("com.mysql.jdbc.Driver")
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor DRIVER_LOCATION = new PropertyDescriptor.Builder()
@ -208,7 +208,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
.defaultValue(null)
.required(false)
.addValidator(StandardValidators.createListValidator(true, true, StandardValidators.createURLorFileValidator()))
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
@ -217,7 +217,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
.description("Username to access the MySQL cluster")
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
@ -227,7 +227,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
.required(false)
.sensitive(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor SERVER_ID = new PropertyDescriptor.Builder()
@ -238,7 +238,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
+ "the replication group. If the Server ID is not specified, it defaults to 65535.")
.required(false)
.addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor DIST_CACHE_CLIENT = new PropertyDescriptor.Builder()
@ -297,7 +297,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
.defaultValue("0 seconds")
.required(true)
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor INIT_SEQUENCE_ID = new PropertyDescriptor.Builder()
@ -309,7 +309,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
+ "processor to guarantee ordered delivery of CDC events.")
.required(false)
.addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor INIT_BINLOG_FILENAME = new PropertyDescriptor.Builder()
@ -321,7 +321,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
+ "Language is supported to enable the use of the Variable Registry and/or environment properties.")
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor INIT_BINLOG_POSITION = new PropertyDescriptor.Builder()
@ -334,7 +334,7 @@ public class CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
+ "and/or environment properties.")
.required(false)
.addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
private static List<PropertyDescriptor> propDescriptors;

View File

@ -29,6 +29,7 @@ import org.apache.nifi.confluent.schemaregistry.client.SchemaRegistryClient;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.schema.access.SchemaField;
import org.apache.nifi.schema.access.SchemaNotFoundException;
@ -68,7 +69,7 @@ public class ConfluentSchemaRegistry extends AbstractControllerService implement
.name("url")
.displayName("Schema Registry URLs")
.description("A comma-separated list of URLs of the Schema Registry to interact with")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.defaultValue("http://localhost:8081")
.required(true)
.addValidator(new MultipleURLValidator())
@ -107,7 +108,7 @@ public class ConfluentSchemaRegistry extends AbstractControllerService implement
.displayName("Communications Timeout")
.description("Specifies how long to wait to receive data from the Schema Registry before considering the communications a failure")
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.defaultValue("30 secs")
.required(true)
.build();

View File

@ -25,6 +25,7 @@ import java.util.Set;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.couchbase.CouchbaseAttributes;
import org.apache.nifi.couchbase.CouchbaseClusterControllerService;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
@ -52,7 +53,7 @@ public abstract class AbstractCouchbaseProcessor extends AbstractProcessor {
public static final PropertyDescriptor DOC_ID = new PropertyDescriptor.Builder().name("Document Id")
.description("A static, fixed Couchbase document id, or an expression to construct the Couchbase document id.")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();

View File

@ -94,7 +94,7 @@ public class GetCouchbaseKey extends AbstractCouchbaseProcessor {
final long startNanos = System.nanoTime();
final ComponentLog logger = getLogger();
String docId = null;
if (!StringUtils.isEmpty(context.getProperty(DOC_ID).getValue())) {
if (context.getProperty(DOC_ID).isSet()) {
docId = context.getProperty(DOC_ID).evaluateAttributeExpressions(inFile).getValue();
} else {
final byte[] content = new byte[(int) inFile.getSize()];

View File

@ -24,7 +24,6 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
@ -120,7 +119,7 @@ public class PutCouchbaseKey extends AbstractCouchbaseProcessor {
});
String docId = flowFile.getAttribute(CoreAttributes.UUID.key());
if (!StringUtils.isEmpty(context.getProperty(DOC_ID).getValue())) {
if (context.getProperty(DOC_ID).isSet()) {
docId = context.getProperty(DOC_ID).evaluateAttributeExpressions(flowFile).getValue();
}

View File

@ -40,7 +40,6 @@ public class TestCouchbaseClusterService {
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.couchbase.TestCouchbaseClusterService", "debug");
testRunner = TestRunners.newTestRunner(PutCouchbaseKey.class);
testRunner.setValidateExpressionUsage(false);
}
@Test

View File

@ -74,7 +74,6 @@ public class TestGetCouchbaseKey {
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.processors.couchbase.TestGetCouchbaseKey", "debug");
testRunner = TestRunners.newTestRunner(GetCouchbaseKey.class);
testRunner.setValidateExpressionUsage(false);
}
private void setupMockBucket(Bucket bucket) throws InitializationException {

View File

@ -73,7 +73,6 @@ public class TestPutCouchbaseKey {
System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.processors.couchbase.TestPutCouchbaseKey", "debug");
testRunner = TestRunners.newTestRunner(PutCouchbaseKey.class);
testRunner.setValidateExpressionUsage(false);
}
private void setupMockBucket(Bucket bucket) throws InitializationException {

View File

@ -16,7 +16,6 @@
*/
package org.apache.nifi.reporting.datadog;
import com.codahale.metrics.Gauge;
import com.codahale.metrics.MetricRegistry;
import com.google.common.base.Optional;
@ -34,6 +33,7 @@ import org.apache.nifi.controller.status.ConnectionStatus;
import org.apache.nifi.controller.status.PortStatus;
import org.apache.nifi.controller.status.ProcessGroupStatus;
import org.apache.nifi.controller.status.ProcessorStatus;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.reporting.AbstractReportingTask;
import org.apache.nifi.reporting.ReportingContext;
@ -48,8 +48,6 @@ import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
@Tags({"reporting", "datadog", "metrics"})
@CapabilityDescription("Publishes metrics from NiFi to datadog. For accurate and informative reporting, components should have unique names.")
public class DataDogReportingTask extends AbstractReportingTask {
@ -73,7 +71,6 @@ public class DataDogReportingTask extends AbstractReportingTask {
static final PropertyDescriptor API_KEY = new PropertyDescriptor.Builder()
.name("API key")
.description("Datadog API key. If specified value is 'agent', local Datadog agent will be used.")
.expressionLanguageSupported(false)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -82,7 +79,7 @@ public class DataDogReportingTask extends AbstractReportingTask {
.name("Metrics prefix")
.description("Prefix to be added before every metric")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.defaultValue("nifi")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -92,7 +89,7 @@ public class DataDogReportingTask extends AbstractReportingTask {
.description("Environment, dataflow is running in. " +
"This property will be included as metrics tag.")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.defaultValue("dev")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();

View File

@ -44,6 +44,7 @@ import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.controller.api.druid.DruidTranquilityService;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.util.StandardValidators;
import org.codehaus.jackson.map.ObjectMapper;
@ -96,7 +97,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
.description("A data source is the Druid equivalent of a database table.")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor ZOOKEEPER_CONNECTION_STRING = new PropertyDescriptor.Builder()
@ -105,7 +106,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
.description("A comma-separated list of host:port pairs, each corresponding to a ZooKeeper server. Ex: localhost:2181")
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor ZOOKEEPER_RETRY_BASE_SLEEP_TIME = new PropertyDescriptor.Builder()
@ -114,7 +115,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
.description("When a connection to Zookeeper needs to be retried, this property specifies the amount of time (in milliseconds) to wait at first before retrying.")
.required(true)
.defaultValue("1000")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
.build();
@ -124,7 +125,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
.description("When a connection to Zookeeper needs to be retried, this property specifies how many times to attempt reconnection.")
.required(true)
.defaultValue("20")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.build();
@ -134,7 +135,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
.description("When a connection to Zookeeper needs to be retried, this property specifies the amount of time to sleep (in milliseconds) between retries.")
.required(true)
.defaultValue("30000")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
.build();
@ -145,7 +146,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
.required(true)
.defaultValue("druid/overlord")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor DRUID_DISCOVERY_PATH = new PropertyDescriptor.Builder()
@ -155,7 +156,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
.required(true)
.defaultValue("/druid/discovery")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor CLUSTER_PARTITIONS = new PropertyDescriptor.Builder()
@ -164,7 +165,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
.description("The number of partitions in the Druid cluster.")
.required(true)
.defaultValue("1")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.build();
@ -174,7 +175,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
.description("The replication factor for the Druid cluster.")
.required(true)
.defaultValue("1")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.build();
@ -185,7 +186,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
.required(true)
.defaultValue("timestamp")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor AGGREGATOR_JSON = new PropertyDescriptor.Builder()
@ -226,7 +227,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
return new ValidationResult.Builder().subject(subject).input(value).valid(false).explanation(subject + " is not valid Aggregator JSON").build();
}
})
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor DIMENSIONS_LIST = new PropertyDescriptor.Builder()
@ -235,7 +236,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
.description("A comma separated list of field names that will be stored as dimensions on ingest.")
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor SEGMENT_GRANULARITY = new PropertyDescriptor.Builder()
@ -285,7 +286,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
.required(true)
.defaultValue("2000")
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor MAX_PENDING_BATCHES = new PropertyDescriptor.Builder()
@ -295,7 +296,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
.required(true)
.defaultValue("5")
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor LINGER_MILLIS = new PropertyDescriptor.Builder()
@ -307,7 +308,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
.required(true)
.defaultValue("1000")
.addValidator(StandardValidators.INTEGER_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
private static final List<PropertyDescriptor> properties;
@ -381,7 +382,7 @@ public class DruidTranquilityController extends AbstractControllerService implem
final String windowPeriod = context.getProperty(WINDOW_PERIOD).getValue();
final String indexRetryPeriod = context.getProperty(INDEX_RETRY_PERIOD).getValue();
final String aggregatorJSON = context.getProperty(AGGREGATOR_JSON).evaluateAttributeExpressions().getValue();
final String dimensionsStringList = context.getProperty(DIMENSIONS_LIST).getValue();
final String dimensionsStringList = context.getProperty(DIMENSIONS_LIST).evaluateAttributeExpressions().getValue();
final int maxBatchSize = context.getProperty(MAX_BATCH_SIZE).evaluateAttributeExpressions().asInteger();
final int maxPendingBatches = context.getProperty(MAX_PENDING_BATCHES).evaluateAttributeExpressions().asInteger();
final int lingerMillis = context.getProperty(LINGER_MILLIS).evaluateAttributeExpressions().asInteger();

View File

@ -51,6 +51,8 @@ import org.apache.nifi.serialization.RecordReader;
import org.apache.nifi.serialization.RecordReaderFactory;
import org.apache.nifi.controller.api.druid.DruidTranquilityService;
import org.apache.nifi.expression.ExpressionLanguageScope;
import com.metamx.tranquility.tranquilizer.Tranquilizer;
import org.apache.nifi.serialization.RecordSetWriter;
import org.apache.nifi.serialization.RecordSetWriterFactory;
@ -86,7 +88,7 @@ public class PutDruidRecord extends AbstractSessionFactoryProcessor {
.displayName("Record Writer")
.description("The Record Writer to use in order to serialize the data to outgoing relationships.")
.identifiesControllerService(RecordSetWriterFactory.class)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.build();

View File

@ -19,6 +19,7 @@ package org.apache.nifi.processors.elasticsearch;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
@ -51,7 +52,7 @@ abstract class AbstractElasticsearch5Processor extends AbstractProcessor {
.required(true)
.defaultValue("UTF-8")
.addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.build();
public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
@ -60,7 +61,7 @@ abstract class AbstractElasticsearch5Processor extends AbstractProcessor {
.description("Username to access the Elasticsearch cluster")
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()

View File

@ -17,6 +17,7 @@
package org.apache.nifi.processors.elasticsearch;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
@ -51,7 +52,7 @@ abstract class AbstractElasticsearch5TransportClientProcessor extends AbstractEl
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.defaultValue("elasticsearch")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
protected static final PropertyDescriptor HOSTS = new PropertyDescriptor.Builder()
@ -61,7 +62,7 @@ abstract class AbstractElasticsearch5TransportClientProcessor extends AbstractEl
+ "host1:port,host2:port,.... For example testcluster:9300. This processor uses the Transport Client to "
+ "connect to hosts. The default transport client port is 9300.")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
.build();
@ -75,7 +76,7 @@ abstract class AbstractElasticsearch5TransportClientProcessor extends AbstractEl
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.dynamicallyModifiesClasspath(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
protected static final PropertyDescriptor PING_TIMEOUT = new PropertyDescriptor.Builder()
@ -86,7 +87,7 @@ abstract class AbstractElasticsearch5TransportClientProcessor extends AbstractEl
.required(true)
.defaultValue("5s")
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
protected static final PropertyDescriptor SAMPLER_INTERVAL = new PropertyDescriptor.Builder()
@ -97,7 +98,7 @@ abstract class AbstractElasticsearch5TransportClientProcessor extends AbstractEl
.required(true)
.defaultValue("5s")
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
protected final AtomicReference<Client> esClient = new AtomicReference<>();

View File

@ -26,6 +26,7 @@ import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
@ -83,7 +84,7 @@ public class DeleteElasticsearch5 extends AbstractElasticsearch5TransportClientP
.displayName("Document Identifier")
.description("The identifier for the document to be deleted")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -92,7 +93,7 @@ public class DeleteElasticsearch5 extends AbstractElasticsearch5TransportClientP
.displayName("Index")
.description("The name of the index to delete the document from")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -101,7 +102,7 @@ public class DeleteElasticsearch5 extends AbstractElasticsearch5TransportClientP
.displayName("Type")
.description("The type of this document to be deleted")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();

View File

@ -26,6 +26,7 @@ import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.logging.ComponentLog;
@ -88,7 +89,7 @@ public class FetchElasticsearch5 extends AbstractElasticsearch5TransportClientPr
.displayName("Document Identifier")
.description("The identifier for the document to be fetched")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -97,7 +98,7 @@ public class FetchElasticsearch5 extends AbstractElasticsearch5TransportClientPr
.displayName("Index")
.description("The name of the index to read from")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -106,7 +107,7 @@ public class FetchElasticsearch5 extends AbstractElasticsearch5TransportClientPr
.displayName("Type")
.description("The type of this document (used by Elasticsearch for indexing and searching)")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();

View File

@ -29,6 +29,7 @@ import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
@ -91,7 +92,7 @@ public class PutElasticsearch5 extends AbstractElasticsearch5TransportClientProc
.displayName("Identifier Attribute")
.description("The name of the attribute containing the identifier for each FlowFile")
.required(true)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.addValidator(StandardValidators.ATTRIBUTE_KEY_VALIDATOR)
.build();
@ -100,7 +101,7 @@ public class PutElasticsearch5 extends AbstractElasticsearch5TransportClientProc
.displayName("Index")
.description("The name of the index to insert into")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(NON_EMPTY_EL_VALIDATOR)
.build();
@ -109,7 +110,7 @@ public class PutElasticsearch5 extends AbstractElasticsearch5TransportClientProc
.displayName("Type")
.description("The type of this document (used by Elasticsearch for indexing and searching)")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(NON_EMPTY_EL_VALIDATOR)
.build();
@ -118,7 +119,7 @@ public class PutElasticsearch5 extends AbstractElasticsearch5TransportClientProc
.displayName("Index Operation")
.description("The type of the operation used to index (index, update, upsert)")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(NON_EMPTY_EL_VALIDATOR)
.defaultValue("index")
.build();
@ -130,7 +131,7 @@ public class PutElasticsearch5 extends AbstractElasticsearch5TransportClientProc
.required(true)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.defaultValue("100")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
private static final Set<Relationship> relationships;

View File

@ -16,6 +16,10 @@
*/
package org.apache.nifi.processors.elasticsearch;
import java.io.IOException;
import java.io.InputStream;
import java.util.HashMap;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
@ -26,10 +30,6 @@ import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
import java.io.IOException;
import java.io.InputStream;
import java.util.HashMap;
/**
* Integration test for delete processor. Please set the hosts, cluster name, index and type etc before running the integrations.
*/
@ -59,7 +59,6 @@ public class ITDeleteElasticsearch5Test {
@Test
public void testPutAndDeleteIntegrationTestSuccess() {
final TestRunner runnerPut = TestRunners.newTestRunner(new PutElasticsearch5());
runnerPut.setValidateExpressionUsage(false);
runnerPut.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, clusterName);
runnerPut.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
runnerPut.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@ -82,7 +81,6 @@ public class ITDeleteElasticsearch5Test {
runnerPut.assertAllFlowFilesTransferred(PutElasticsearch5.REL_SUCCESS, 1);
final TestRunner runnerDelete = TestRunners.newTestRunner(new DeleteElasticsearch5());
runnerDelete.setValidateExpressionUsage(false);
runnerDelete.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, clusterName);
runnerDelete.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
@ -108,7 +106,6 @@ public class ITDeleteElasticsearch5Test {
@Test
public void testDeleteIntegrationTestDocumentNotFound() {
final TestRunner runnerDelete = TestRunners.newTestRunner(new DeleteElasticsearch5());
runnerDelete.setValidateExpressionUsage(false);
runnerDelete.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, clusterName);
runnerDelete.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
@ -138,7 +135,6 @@ public class ITDeleteElasticsearch5Test {
@Test
public void testDeleteIntegrationTestBadIndex() {
final TestRunner runnerDelete = TestRunners.newTestRunner(new DeleteElasticsearch5());
runnerDelete.setValidateExpressionUsage(false);
runnerDelete.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, clusterName);
runnerDelete.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
@ -168,7 +164,6 @@ public class ITDeleteElasticsearch5Test {
@Test
public void testDeleteIntegrationTestBadType() {
final TestRunner runnerDelete = TestRunners.newTestRunner(new DeleteElasticsearch5());
runnerDelete.setValidateExpressionUsage(false);
runnerDelete.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, clusterName);
runnerDelete.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");

View File

@ -16,6 +16,13 @@
*/
package org.apache.nifi.processors.elasticsearch;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import java.io.IOException;
import java.util.HashMap;
import java.util.concurrent.ExecutionException;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
@ -28,13 +35,6 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.util.HashMap;
import java.util.concurrent.ExecutionException;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
public class TestDeleteElasticsearch5 {
private String documentId;
@ -71,7 +71,6 @@ public class TestDeleteElasticsearch5 {
runner = TestRunners.newTestRunner(mockDeleteProcessor);
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@ -227,7 +226,6 @@ public class TestDeleteElasticsearch5 {
};
runner = TestRunners.newTestRunner(mockDeleteProcessor);
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@ -275,7 +273,6 @@ public class TestDeleteElasticsearch5 {
};
runner = TestRunners.newTestRunner(mockDeleteProcessor);
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");

View File

@ -83,7 +83,6 @@ public class TestFetchElasticsearch5 {
@Test
public void testFetchElasticsearch5OnTrigger() throws IOException {
runner = TestRunners.newTestRunner(new FetchElasticsearch5TestProcessor(true)); // all docs are found
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@ -110,7 +109,6 @@ public class TestFetchElasticsearch5 {
@Test
public void testFetchElasticsearch5OnTriggerEL() throws IOException {
runner = TestRunners.newTestRunner(new FetchElasticsearch5TestProcessor(true)); // all docs are found
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "${cluster.name}");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "${hosts}");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "${ping.timeout}");
@ -147,7 +145,6 @@ public class TestFetchElasticsearch5 {
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.SAMPLER_INTERVAL, "5s");
runner.setProperty(FetchElasticsearch5.INDEX, "doc");
runner.setProperty(FetchElasticsearch5.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(FetchElasticsearch5.DOC_ID, "${doc_id}");
runner.enqueue(docExample, new HashMap<String, String>() {{
@ -171,7 +168,6 @@ public class TestFetchElasticsearch5 {
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.SAMPLER_INTERVAL, "5s");
runner.setProperty(FetchElasticsearch5.INDEX, "doc");
runner.setProperty(FetchElasticsearch5.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(FetchElasticsearch5.DOC_ID, "${doc_id}");
runner.assertNotValid();
@ -187,7 +183,6 @@ public class TestFetchElasticsearch5 {
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.SAMPLER_INTERVAL, "5s");
runner.setProperty(FetchElasticsearch5.INDEX, "doc");
runner.setProperty(FetchElasticsearch5.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(FetchElasticsearch5.DOC_ID, "${doc_id}");
// No Node Available exception
@ -273,7 +268,6 @@ public class TestFetchElasticsearch5 {
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.SAMPLER_INTERVAL, "5s");
runner.setProperty(FetchElasticsearch5.INDEX, "doc");
runner.setProperty(FetchElasticsearch5.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(FetchElasticsearch5.DOC_ID, "${doc_id}");
// Allow time for the controller service to fully initialize
@ -387,7 +381,6 @@ public class TestFetchElasticsearch5 {
System.out.println("Starting test " + new Object() {
}.getClass().getEnclosingMethod().getName());
final TestRunner runner = TestRunners.newTestRunner(new FetchElasticsearch5());
runner.setValidateExpressionUsage(true);
//Local Cluster - Mac pulled from brew
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch_brew");
@ -418,7 +411,6 @@ public class TestFetchElasticsearch5 {
System.out.println("Starting test " + new Object() {
}.getClass().getEnclosingMethod().getName());
final TestRunner runner = TestRunners.newTestRunner(new FetchElasticsearch5());
runner.setValidateExpressionUsage(true);
//Local Cluster - Mac pulled from brew
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch_brew");

View File

@ -16,6 +16,22 @@
*/
package org.apache.nifi.processors.elasticsearch;
import static org.junit.Assert.assertNotNull;
import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.io.InputStream;
import java.net.InetSocketAddress;
import java.net.MalformedURLException;
import java.util.HashMap;
import java.util.List;
import java.util.concurrent.ExecutionException;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
@ -45,23 +61,6 @@ import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.io.IOException;
import java.io.InputStream;
import java.net.InetSocketAddress;
import java.net.MalformedURLException;
import java.util.HashMap;
import java.util.List;
import java.util.concurrent.ExecutionException;
import static org.junit.Assert.assertNotNull;
import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
public class TestPutElasticsearch5 {
private InputStream docExample;
@ -81,7 +80,6 @@ public class TestPutElasticsearch5 {
@Test
public void testPutElasticSearchOnTrigger() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearch5TestProcessor(false)); // no failures
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@ -109,7 +107,6 @@ public class TestPutElasticsearch5 {
@Test
public void testPutElasticSearchOnTriggerEL() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearch5TestProcessor(false)); // no failures
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "${cluster.name}");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "${hosts}");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "${ping.timeout}");
@ -142,7 +139,6 @@ public class TestPutElasticsearch5 {
@Test
public void testPutElasticSearchOnTriggerBadDocIdentifier() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearch5TestProcessor(false)); // no failures
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@ -167,7 +163,6 @@ public class TestPutElasticsearch5 {
@Test
public void testPutElasticSearchOnTriggerWithFailures() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearch5TestProcessor(true)); // simulate failures
runner.setValidateExpressionUsage(false);
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@ -198,7 +193,6 @@ public class TestPutElasticsearch5 {
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.SAMPLER_INTERVAL, "5s");
runner.setProperty(PutElasticsearch5.INDEX, "doc");
runner.setProperty(PutElasticsearch5.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(PutElasticsearch5.ID_ATTRIBUTE, "doc_id");
// No Node Available exception
@ -255,7 +249,6 @@ public class TestPutElasticsearch5 {
@Test
public void testPutElasticsearch5OnTriggerWithNoIdAttribute() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearch5TestProcessor(true)); // simulate failures
runner.setValidateExpressionUsage(false);
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@ -276,7 +269,6 @@ public class TestPutElasticsearch5 {
@Test
public void testPutElasticsearch5OnTriggerWithIndexFromAttribute() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearch5TestProcessor(false));
runner.setValidateExpressionUsage(false);
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@ -313,7 +305,6 @@ public class TestPutElasticsearch5 {
@Test
public void testPutElasticSearchOnTriggerWithInvalidIndexOp() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearch5TestProcessor(false)); // no failures
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.HOSTS, "127.0.0.1:9300");
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.PING_TIMEOUT, "5s");
@ -448,7 +439,6 @@ public class TestPutElasticsearch5 {
System.out.println("Starting test " + new Object() {
}.getClass().getEnclosingMethod().getName());
final TestRunner runner = TestRunners.newTestRunner(new PutElasticsearch5());
runner.setValidateExpressionUsage(false);
//Local Cluster - Mac pulled from brew
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch_brew");
@ -480,7 +470,6 @@ public class TestPutElasticsearch5 {
System.out.println("Starting test " + new Object() {
}.getClass().getEnclosingMethod().getName());
final TestRunner runner = TestRunners.newTestRunner(new PutElasticsearch5());
runner.setValidateExpressionUsage(false);
//Local Cluster - Mac pulled from brew
runner.setProperty(AbstractElasticsearch5TransportClientProcessor.CLUSTER_NAME, "elasticsearch_brew");

View File

@ -26,6 +26,7 @@ import okhttp3.Response;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
@ -62,7 +63,7 @@ public abstract class AbstractElasticsearchHttpProcessor extends AbstractElastic
.description("Elasticsearch URL which will be connected to, including scheme (http, e.g.), host, and port. The default port for the REST API is 9200.")
.required(true)
.addValidator(StandardValidators.URL_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor PROXY_HOST = new PropertyDescriptor.Builder()
@ -88,7 +89,7 @@ public abstract class AbstractElasticsearchHttpProcessor extends AbstractElastic
.required(true)
.defaultValue("5 secs")
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor RESPONSE_TIMEOUT = new PropertyDescriptor.Builder()
@ -98,7 +99,7 @@ public abstract class AbstractElasticsearchHttpProcessor extends AbstractElastic
.required(true)
.defaultValue("15 secs")
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
private final AtomicReference<OkHttpClient> okHttpClientAtomicReference = new AtomicReference<>();
@ -109,7 +110,7 @@ public abstract class AbstractElasticsearchHttpProcessor extends AbstractElastic
.name(propertyDescriptorName)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.dynamic(true)
.build();
}

View File

@ -19,6 +19,7 @@ package org.apache.nifi.processors.elasticsearch;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
@ -49,7 +50,7 @@ public abstract class AbstractElasticsearchProcessor extends AbstractProcessor {
.required(true)
.defaultValue("UTF-8")
.addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.build();
public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
@ -57,7 +58,7 @@ public abstract class AbstractElasticsearchProcessor extends AbstractProcessor {
.description("Username to access the Elasticsearch cluster")
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
@ -66,7 +67,7 @@ public abstract class AbstractElasticsearchProcessor extends AbstractProcessor {
.required(false)
.sensitive(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
protected abstract void createElasticsearchClient(ProcessContext context) throws ProcessException;

View File

@ -17,6 +17,7 @@
package org.apache.nifi.processors.elasticsearch;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
@ -49,7 +50,7 @@ public abstract class AbstractElasticsearchTransportClientProcessor extends Abst
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.defaultValue("elasticsearch")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
protected static final PropertyDescriptor HOSTS = new PropertyDescriptor.Builder()
@ -58,9 +59,8 @@ public abstract class AbstractElasticsearchTransportClientProcessor extends Abst
+ "host1:port,host2:port,.... For example testcluster:9300. This processor uses the Transport Client to "
+ "connect to hosts. The default transport client port is 9300.")
.required(true)
.expressionLanguageSupported(false)
.addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor PROP_SHIELD_LOCATION = new PropertyDescriptor.Builder()
@ -71,7 +71,7 @@ public abstract class AbstractElasticsearchTransportClientProcessor extends Abst
+ "lib/ directory, doing so will prevent the Shield plugin from being loaded.")
.required(false)
.addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
protected static final PropertyDescriptor PING_TIMEOUT = new PropertyDescriptor.Builder()
@ -81,7 +81,7 @@ public abstract class AbstractElasticsearchTransportClientProcessor extends Abst
.required(true)
.defaultValue("5s")
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
protected static final PropertyDescriptor SAMPLER_INTERVAL = new PropertyDescriptor.Builder()
@ -91,7 +91,7 @@ public abstract class AbstractElasticsearchTransportClientProcessor extends Abst
.required(true)
.defaultValue("5s")
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
protected AtomicReference<Client> esClient = new AtomicReference<>();

View File

@ -26,6 +26,7 @@ import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
@ -85,7 +86,7 @@ public class FetchElasticsearch extends AbstractElasticsearchTransportClientProc
.name("Document Identifier")
.description("The identifier for the document to be fetched")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -93,7 +94,7 @@ public class FetchElasticsearch extends AbstractElasticsearchTransportClientProc
.name("Index")
.description("The name of the index to read from")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -101,7 +102,7 @@ public class FetchElasticsearch extends AbstractElasticsearchTransportClientProc
.name("Type")
.description("The type of this document (used by Elasticsearch for indexing and searching)")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();

View File

@ -32,6 +32,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
@ -70,7 +71,7 @@ import java.util.stream.Stream;
@DynamicProperty(
name = "A URL query parameter",
value = "The value to set it to",
supportsExpressionLanguage = true,
expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
description = "Adds the specified property name/value as a query parameter in the Elasticsearch URL used for processing")
public class FetchElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
@ -102,7 +103,7 @@ public class FetchElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
.displayName("Document Identifier")
.description("The identifier of the document to be fetched")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -111,7 +112,7 @@ public class FetchElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
.displayName("Index")
.description("The name of the index to read from.")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -121,7 +122,7 @@ public class FetchElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
.description("The (optional) type of this document, used by Elasticsearch for indexing and searching. If the property is empty, "
+ "the first document matching the identifier across all types will be retrieved.")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -131,7 +132,7 @@ public class FetchElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
.description("A comma-separated list of fields to retrieve from the document. If the Fields property is left blank, "
+ "then the entire document's source will be retrieved.")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();

View File

@ -28,6 +28,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.AttributeExpression;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
@ -81,7 +82,7 @@ public class PutElasticsearch extends AbstractElasticsearchTransportClientProces
.name("Identifier Attribute")
.description("The name of the attribute containing the identifier for each FlowFile")
.required(true)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.addValidator(StandardValidators.ATTRIBUTE_KEY_VALIDATOR)
.build();
@ -89,7 +90,7 @@ public class PutElasticsearch extends AbstractElasticsearchTransportClientProces
.name("Index")
.description("The name of the index to insert into")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.createAttributeExpressionLanguageValidator(
AttributeExpression.ResultType.STRING, true))
.build();
@ -98,7 +99,7 @@ public class PutElasticsearch extends AbstractElasticsearchTransportClientProces
.name("Type")
.description("The type of this document (used by Elasticsearch for indexing and searching)")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
.build();
@ -106,7 +107,7 @@ public class PutElasticsearch extends AbstractElasticsearchTransportClientProces
.name("Index Operation")
.description("The type of the operation used to index (index, update, upsert)")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
.defaultValue("index")
.build();
@ -117,7 +118,7 @@ public class PutElasticsearch extends AbstractElasticsearchTransportClientProces
.required(true)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.defaultValue("100")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
private static final Set<Relationship> relationships;

View File

@ -38,6 +38,7 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.expression.AttributeExpression;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
@ -72,7 +73,7 @@ import static org.apache.commons.lang3.StringUtils.trimToEmpty;
@DynamicProperty(
name = "A URL query parameter",
value = "The value to set it to",
supportsExpressionLanguage = true,
expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
description = "Adds the specified property name/value as a query parameter in the Elasticsearch URL used for processing")
@SystemResourceConsideration(resource = SystemResource.MEMORY)
public class PutElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
@ -94,7 +95,7 @@ public class PutElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
+ "this property may be left empty or evaluate to an empty value, in which case the document's identifier will be "
+ "auto-generated by Elasticsearch. For all other Index Operations, the attribute must evaluate to a non-empty value.")
.required(false)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.addValidator(StandardValidators.ATTRIBUTE_KEY_VALIDATOR)
.build();
@ -103,7 +104,7 @@ public class PutElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
.displayName("Index")
.description("The name of the index to insert into")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.createAttributeExpressionLanguageValidator(
AttributeExpression.ResultType.STRING, true))
.build();
@ -113,7 +114,7 @@ public class PutElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
.displayName("Type")
.description("The type of this document (used by Elasticsearch for indexing and searching)")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
.build();
@ -122,7 +123,7 @@ public class PutElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
.displayName("Index Operation")
.description("The type of the operation used to index (index, update, upsert, delete)")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
.defaultValue("index")
.build();
@ -136,7 +137,7 @@ public class PutElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
.required(true)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.defaultValue("100")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
private static final Set<Relationship> relationships;

View File

@ -37,6 +37,7 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.expression.AttributeExpression;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
@ -92,7 +93,7 @@ import static org.apache.commons.lang3.StringUtils.trimToEmpty;
@DynamicProperty(
name = "A URL query parameter",
value = "The value to set it to",
supportsExpressionLanguage = true,
expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
description = "Adds the specified property name/value as a query parameter in the Elasticsearch URL used for processing")
public class PutElasticsearchHttpRecord extends AbstractElasticsearchHttpProcessor {
@ -122,7 +123,7 @@ public class PutElasticsearchHttpRecord extends AbstractElasticsearchHttpProcess
+ "auto-generated by Elasticsearch. For all other Index Operations, the field's value must be non-empty.")
.required(false)
.addValidator(new RecordPathValidator())
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.build();
static final PropertyDescriptor INDEX = new PropertyDescriptor.Builder()
@ -130,7 +131,7 @@ public class PutElasticsearchHttpRecord extends AbstractElasticsearchHttpProcess
.displayName("Index")
.description("The name of the index to insert into")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.createAttributeExpressionLanguageValidator(
AttributeExpression.ResultType.STRING, true))
.build();
@ -140,7 +141,7 @@ public class PutElasticsearchHttpRecord extends AbstractElasticsearchHttpProcess
.displayName("Type")
.description("The type of this document (used by Elasticsearch for indexing and searching)")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
.build();
@ -149,7 +150,7 @@ public class PutElasticsearchHttpRecord extends AbstractElasticsearchHttpProcess
.displayName("Index Operation")
.description("The type of the operation used to index (index, update, upsert, delete)")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
.defaultValue("index")
.build();

View File

@ -32,6 +32,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
@ -76,7 +77,7 @@ import java.util.stream.Stream;
@DynamicProperty(
name = "A URL query parameter",
value = "The value to set it to",
supportsExpressionLanguage = true,
expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
description = "Adds the specified property name/value as a query parameter in the Elasticsearch URL used for processing")
public class QueryElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
@ -107,16 +108,22 @@ public class QueryElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
.build();
public static final PropertyDescriptor QUERY = new PropertyDescriptor.Builder()
.name("query-es-query").displayName("Query")
.description("The Lucene-style query to run against ElasticSearch (e.g., genre:blues AND -artist:muddy)").required(true)
.expressionLanguageSupported(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.name("query-es-query")
.displayName("Query")
.description("The Lucene-style query to run against ElasticSearch (e.g., genre:blues AND -artist:muddy)")
.required(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor INDEX = new PropertyDescriptor.Builder()
.name("query-es-index").displayName("Index")
.name("query-es-index")
.displayName("Index")
.description("The name of the index to read from. If the property is set "
+ "to _all, the query will match across all indexes.").required(true)
.expressionLanguageSupported(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+ "to _all, the query will match across all indexes.")
.required(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
@ -126,7 +133,7 @@ public class QueryElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
"The (optional) type of this query, used by Elasticsearch for indexing and searching. If the property is empty, "
+ "the the query will match across all types.")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -137,7 +144,7 @@ public class QueryElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
"A comma-separated list of fields to retrieve from the document. If the Fields property is left blank, "
+ "then the entire document's source will be retrieved.")
.required(false)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -147,20 +154,29 @@ public class QueryElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
.description(
"A sort parameter (e.g., timestamp:asc). If the Sort property is left blank, "
+ "then the results will be retrieved in document order.")
.required(false).expressionLanguageSupported(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
.required(false)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
.name("query-es-size").displayName("Page Size").defaultValue("20")
.name("query-es-size")
.displayName("Page Size")
.defaultValue("20")
.description("Determines how many documents to return per page during scrolling.")
.required(true).expressionLanguageSupported(true)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR).build();
.required(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.build();
public static final PropertyDescriptor LIMIT = new PropertyDescriptor.Builder()
.name("query-es-limit").displayName("Limit")
.name("query-es-limit")
.displayName("Limit")
.description("If set, limits the number of results that will be returned.")
.required(false).expressionLanguageSupported(true)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR).build();
.required(false)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.build();
public static final PropertyDescriptor TARGET = new PropertyDescriptor.Builder()
.name("query-es-target")
@ -170,10 +186,12 @@ public class QueryElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
+ "response will be written as the content of the flow file. In the case of 'Flow file attributes', "
+ "the original flow file (if applicable) will be cloned for each result, and all return fields will be placed "
+ "in a flow file attribute of the same name, but prefixed by 'es.result.'")
.required(true).expressionLanguageSupported(false)
.required(true)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.defaultValue(TARGET_FLOW_FILE_CONTENT)
.allowableValues(TARGET_FLOW_FILE_CONTENT, TARGET_FLOW_FILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
private static final Set<Relationship> relationships;
private static final List<PropertyDescriptor> propertyDescriptors;

View File

@ -36,6 +36,7 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.Scope;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.components.state.StateMap;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
@ -76,7 +77,7 @@ import java.util.stream.Stream;
@DynamicProperty(
name = "A URL query parameter",
value = "The value to set it to",
supportsExpressionLanguage = true,
expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
description = "Adds the specified property name/value as a query parameter in the Elasticsearch URL used for processing")
@Stateful(description = "After each successful scroll page, the latest scroll_id is persisted in scrollId as input for the next scroll call. "
+ "Once the entire query is complete, finishedQuery state will be set to true, and the processor will not execute unless this is cleared.", scopes = { Scope.LOCAL })
@ -100,9 +101,12 @@ public class ScrollElasticsearchHttp extends AbstractElasticsearchHttpProcessor
+ "flow files will be routed to failure.").build();
public static final PropertyDescriptor QUERY = new PropertyDescriptor.Builder()
.name("scroll-es-query").displayName("Query")
.description("The Lucene-style query to run against ElasticSearch (e.g., genre:blues AND -artist:muddy)").required(true)
.expressionLanguageSupported(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.name("scroll-es-query")
.displayName("Query")
.description("The Lucene-style query to run against ElasticSearch (e.g., genre:blues AND -artist:muddy)")
.required(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor SCROLL_DURATION = new PropertyDescriptor.Builder()
@ -111,7 +115,7 @@ public class ScrollElasticsearchHttp extends AbstractElasticsearchHttpProcessor
.description("The scroll duration is how long each search context is kept in memory.")
.defaultValue("1m")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(
StandardValidators.createRegexMatchingValidator(Pattern.compile("[0-9]+(m|h)")))
.build();
@ -122,7 +126,7 @@ public class ScrollElasticsearchHttp extends AbstractElasticsearchHttpProcessor
.description("The name of the index to read from. If the property is set "
+ "to _all, the query will match across all indexes.")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -134,7 +138,8 @@ public class ScrollElasticsearchHttp extends AbstractElasticsearchHttpProcessor
+ "the the query will match across all types.")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.required(false)
.expressionLanguageSupported(true).build();
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.build();
public static final PropertyDescriptor FIELDS = new PropertyDescriptor.Builder()
.name("scroll-es-fields")
@ -142,8 +147,10 @@ public class ScrollElasticsearchHttp extends AbstractElasticsearchHttpProcessor
.description(
"A comma-separated list of fields to retrieve from the document. If the Fields property is left blank, "
+ "then the entire document's source will be retrieved.")
.required(false).expressionLanguageSupported(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
.required(false)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor SORT = new PropertyDescriptor.Builder()
.name("scroll-es-sort")
@ -151,14 +158,20 @@ public class ScrollElasticsearchHttp extends AbstractElasticsearchHttpProcessor
.description(
"A sort parameter (e.g., timestamp:asc). If the Sort property is left blank, "
+ "then the results will be retrieved in document order.")
.required(false).expressionLanguageSupported(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
.required(false)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor PAGE_SIZE = new PropertyDescriptor.Builder()
.name("scroll-es-size").displayName("Page Size").defaultValue("20")
.name("scroll-es-size")
.displayName("Page Size")
.defaultValue("20")
.description("Determines how many documents to return per page during scrolling.")
.required(true).expressionLanguageSupported(true)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR).build();
.required(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.build();
private static final Set<Relationship> relationships;
private static final List<PropertyDescriptor> propertyDescriptors;

View File

@ -40,7 +40,6 @@ public class ITQueryElasticsearchHttp {
@Test
public void testFetchElasticsearchOnTrigger() throws IOException {
runner = TestRunners.newTestRunner(QueryElasticsearchHttp.class); // all docs are found
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL,
"http://localhost.internal:9200");
@ -67,7 +66,6 @@ public class ITQueryElasticsearchHttp {
@Test
public void testFetchElasticsearchOnTrigger_IncomingFile() throws IOException {
runner = TestRunners.newTestRunner(QueryElasticsearchHttp.class); // all docs are found
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL,
"http://localhost.internal:9200");

View File

@ -38,7 +38,6 @@ public class ITScrollElasticsearchHttp {
@Test
public void testFetchElasticsearchOnTrigger() throws IOException {
runner = TestRunners.newTestRunner(ScrollElasticsearchHttp.class); // all docs are found
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL,
"http://ip-172-31-49-152.ec2.internal:9200");

View File

@ -16,6 +16,23 @@
*/
package org.apache.nifi.processors.elasticsearch;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.io.InputStream;
import java.net.MalformedURLException;
import java.util.HashMap;
import java.util.concurrent.ExecutionException;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.provenance.ProvenanceEventType;
@ -44,23 +61,6 @@ import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
import java.io.IOException;
import java.io.InputStream;
import java.net.MalformedURLException;
import java.util.HashMap;
import java.util.concurrent.ExecutionException;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
public class TestFetchElasticsearch {
private InputStream docExample;
@ -81,7 +81,6 @@ public class TestFetchElasticsearch {
@Test
public void testFetchElasticsearchOnTrigger() throws IOException {
runner = TestRunners.newTestRunner(new FetchElasticsearchTestProcessor(true)); // all docs are found
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch");
runner.setProperty(AbstractElasticsearchTransportClientProcessor.HOSTS, "127.0.0.1:9300");
runner.setProperty(AbstractElasticsearchTransportClientProcessor.PING_TIMEOUT, "5s");
@ -110,7 +109,6 @@ public class TestFetchElasticsearch {
@Test
public void testFetchElasticsearchOnTriggerEL() throws IOException {
runner = TestRunners.newTestRunner(new FetchElasticsearchTestProcessor(true)); // all docs are found
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "${cluster.name}");
runner.setProperty(AbstractElasticsearchTransportClientProcessor.HOSTS, "${hosts}");
runner.setProperty(AbstractElasticsearchTransportClientProcessor.PING_TIMEOUT, "${ping.timeout}");
@ -147,7 +145,6 @@ public class TestFetchElasticsearch {
runner.setProperty(AbstractElasticsearchTransportClientProcessor.SAMPLER_INTERVAL, "5s");
runner.setProperty(FetchElasticsearch.INDEX, "doc");
runner.setProperty(FetchElasticsearch.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(FetchElasticsearch.DOC_ID, "${doc_id}");
runner.enqueue(docExample, new HashMap<String, String>() {{
@ -171,7 +168,6 @@ public class TestFetchElasticsearch {
runner.setProperty(AbstractElasticsearchTransportClientProcessor.SAMPLER_INTERVAL, "5s");
runner.setProperty(FetchElasticsearch.INDEX, "doc");
runner.setProperty(FetchElasticsearch.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(FetchElasticsearch.DOC_ID, "${doc_id}");
runner.assertNotValid();
@ -187,7 +183,6 @@ public class TestFetchElasticsearch {
runner.setProperty(AbstractElasticsearchTransportClientProcessor.SAMPLER_INTERVAL, "5s");
runner.setProperty(FetchElasticsearch.INDEX, "doc");
runner.setProperty(FetchElasticsearch.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(FetchElasticsearch.DOC_ID, "${doc_id}");
// No Node Available exception
@ -272,7 +267,6 @@ public class TestFetchElasticsearch {
runner.setProperty(AbstractElasticsearchTransportClientProcessor.SAMPLER_INTERVAL, "5s");
runner.setProperty(FetchElasticsearch.INDEX, "doc");
runner.setProperty(FetchElasticsearch.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(FetchElasticsearch.DOC_ID, "${doc_id}");
// Allow time for the controller service to fully initialize
@ -380,7 +374,6 @@ public class TestFetchElasticsearch {
System.out.println("Starting test " + new Object() {
}.getClass().getEnclosingMethod().getName());
final TestRunner runner = TestRunners.newTestRunner(new FetchElasticsearch());
runner.setValidateExpressionUsage(true);
//Local Cluster - Mac pulled from brew
runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch_brew");
@ -411,7 +404,6 @@ public class TestFetchElasticsearch {
System.out.println("Starting test " + new Object() {
}.getClass().getEnclosingMethod().getName());
final TestRunner runner = TestRunners.newTestRunner(new FetchElasticsearch());
runner.setValidateExpressionUsage(true);
//Local Cluster - Mac pulled from brew
runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch_brew");

View File

@ -16,14 +16,18 @@
*/
package org.apache.nifi.processors.elasticsearch;
import okhttp3.Call;
import okhttp3.MediaType;
import okhttp3.OkHttpClient;
import okhttp3.Protocol;
import okhttp3.Request;
import okhttp3.RequestBody;
import okhttp3.Response;
import okhttp3.ResponseBody;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.io.InputStream;
import java.net.URL;
import java.util.HashMap;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.ssl.SSLContextService;
@ -37,17 +41,14 @@ import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.io.IOException;
import java.io.InputStream;
import java.net.URL;
import java.util.HashMap;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import okhttp3.Call;
import okhttp3.MediaType;
import okhttp3.OkHttpClient;
import okhttp3.Protocol;
import okhttp3.Request;
import okhttp3.RequestBody;
import okhttp3.Response;
import okhttp3.ResponseBody;
public class TestFetchElasticsearchHttp {
@ -68,7 +69,6 @@ public class TestFetchElasticsearchHttp {
@Test
public void testFetchElasticsearchOnTriggerEL() throws IOException {
runner = TestRunners.newTestRunner(new FetchElasticsearchHttpTestProcessor(true)); // all docs are found
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "${es.url}");
runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
@ -97,7 +97,6 @@ public class TestFetchElasticsearchHttp {
@Test
public void testFetchElasticsearchOnTrigger() throws IOException {
runner = TestRunners.newTestRunner(new FetchElasticsearchHttpTestProcessor(true)); // all docs are found
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
@ -124,7 +123,6 @@ public class TestFetchElasticsearchHttp {
final String DOC_ID = "28039652140";
FetchElasticsearchHttpTestProcessor processor = new FetchElasticsearchHttpTestProcessor(true);
runner = TestRunners.newTestRunner(processor); // all docs are found
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, ES_URL);
runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
@ -149,7 +147,6 @@ public class TestFetchElasticsearchHttp {
@Test
public void testFetchElasticsearchOnTriggerWithFields() throws IOException {
runner = TestRunners.newTestRunner(new FetchElasticsearchHttpTestProcessor(true)); // all docs are found
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
@ -177,7 +174,6 @@ public class TestFetchElasticsearchHttp {
runner = TestRunners.newTestRunner(new FetchElasticsearchHttpTestProcessor(false)); // simulate doc not found
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
runner.setValidateExpressionUsage(true);
runner.setProperty(FetchElasticsearchHttp.DOC_ID, "${doc_id}");
runner.setIncomingConnection(true);
@ -201,7 +197,6 @@ public class TestFetchElasticsearchHttp {
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
runner.setProperty(FetchElasticsearchHttp.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(FetchElasticsearchHttp.DOC_ID, "${doc_id}");
runner.enqueue(docExample, new HashMap<String, String>() {{
@ -224,7 +219,6 @@ public class TestFetchElasticsearchHttp {
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
runner.setProperty(FetchElasticsearchHttp.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(FetchElasticsearchHttp.DOC_ID, "${doc_id}");
runner.enqueue(docExample, new HashMap<String, String>() {{
@ -247,7 +241,6 @@ public class TestFetchElasticsearchHttp {
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
runner.setProperty(FetchElasticsearchHttp.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(FetchElasticsearchHttp.DOC_ID, "${doc_id}");
runner.setIncomingConnection(false);
@ -264,7 +257,6 @@ public class TestFetchElasticsearchHttp {
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "127.0.0.1:9200");
runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
runner.setProperty(FetchElasticsearchHttp.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(FetchElasticsearchHttp.DOC_ID, "${doc_id}");
runner.assertNotValid();
@ -281,7 +273,6 @@ public class TestFetchElasticsearchHttp {
runner.setProperty(FetchElasticsearchHttp.PROP_SSL_CONTEXT_SERVICE, "ssl-context");
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
runner.setValidateExpressionUsage(true);
runner.setProperty(FetchElasticsearchHttp.DOC_ID, "${doc_id}");
// Allow time for the controller service to fully initialize
@ -299,7 +290,6 @@ public class TestFetchElasticsearchHttp {
FetchElasticsearchHttpTestProcessor p = new FetchElasticsearchHttpTestProcessor(true); // all docs are found
p.setExpectedUrl("http://127.0.0.1:9200/doc/status/28039652140?_source_include=id&myparam=myvalue");
runner = TestRunners.newTestRunner(p);
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
@ -391,6 +381,7 @@ public class TestFetchElasticsearchHttp {
return url;
}
@Override
protected OkHttpClient getClient() {
return client;
}
@ -413,7 +404,6 @@ public class TestFetchElasticsearchHttp {
System.out.println("Starting test " + new Object() {
}.getClass().getEnclosingMethod().getName());
final TestRunner runner = TestRunners.newTestRunner(new FetchElasticsearchHttp());
runner.setValidateExpressionUsage(true);
//Local Cluster - Mac pulled from brew
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
@ -437,7 +427,6 @@ public class TestFetchElasticsearchHttp {
System.out.println("Starting test " + new Object() {
}.getClass().getEnclosingMethod().getName());
final TestRunner runner = TestRunners.newTestRunner(new FetchElasticsearchHttp());
runner.setValidateExpressionUsage(true);
//Local Cluster - Mac pulled from brew
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");

View File

@ -16,6 +16,21 @@
*/
package org.apache.nifi.processors.elasticsearch;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.io.InputStream;
import java.util.HashMap;
import java.util.concurrent.ExecutionException;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.provenance.ProvenanceEventType;
@ -45,22 +60,6 @@ import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.io.IOException;
import java.io.InputStream;
import java.util.HashMap;
import java.util.concurrent.ExecutionException;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
public class TestPutElasticsearch {
private InputStream docExample;
@ -80,7 +79,6 @@ public class TestPutElasticsearch {
@Test
public void testPutElasticSearchOnTrigger() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch");
runner.setProperty(AbstractElasticsearchTransportClientProcessor.HOSTS, "127.0.0.1:9300");
runner.setProperty(AbstractElasticsearchTransportClientProcessor.PING_TIMEOUT, "5s");
@ -110,7 +108,6 @@ public class TestPutElasticsearch {
@Test
public void testPutElasticSearchOnTriggerEL() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "${cluster.name}");
runner.setProperty(AbstractElasticsearchTransportClientProcessor.HOSTS, "${hosts}");
runner.setProperty(AbstractElasticsearchTransportClientProcessor.PING_TIMEOUT, "${ping.timeout}");
@ -143,7 +140,6 @@ public class TestPutElasticsearch {
@Test
public void testPutElasticSearchOnTriggerWithFailures() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(true)); // simulate failures
runner.setValidateExpressionUsage(false);
runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch");
runner.setProperty(AbstractElasticsearchTransportClientProcessor.HOSTS, "127.0.0.1:9300");
runner.setProperty(AbstractElasticsearchTransportClientProcessor.PING_TIMEOUT, "5s");
@ -178,7 +174,6 @@ public class TestPutElasticsearch {
runner.setProperty(AbstractElasticsearchTransportClientProcessor.SAMPLER_INTERVAL, "5s");
runner.setProperty(PutElasticsearch.INDEX, "doc");
runner.setProperty(PutElasticsearch.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(PutElasticsearch.ID_ATTRIBUTE, "doc_id");
// No Node Available exception
@ -235,7 +230,6 @@ public class TestPutElasticsearch {
@Test
public void testPutElasticsearchOnTriggerWithNoIdAttribute() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(true)); // simulate failures
runner.setValidateExpressionUsage(false);
runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch");
runner.setProperty(AbstractElasticsearchTransportClientProcessor.HOSTS, "127.0.0.1:9300");
runner.setProperty(AbstractElasticsearchTransportClientProcessor.PING_TIMEOUT, "5s");
@ -256,7 +250,6 @@ public class TestPutElasticsearch {
@Test
public void testPutElasticsearchOnTriggerWithIndexFromAttribute() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false));
runner.setValidateExpressionUsage(false);
runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch");
runner.setProperty(AbstractElasticsearchTransportClientProcessor.HOSTS, "127.0.0.1:9300");
runner.setProperty(AbstractElasticsearchTransportClientProcessor.PING_TIMEOUT, "5s");
@ -293,7 +286,6 @@ public class TestPutElasticsearch {
@Test
public void testPutElasticSearchOnTriggerWithInvalidIndexOp() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch");
runner.setProperty(AbstractElasticsearchTransportClientProcessor.HOSTS, "127.0.0.1:9300");
runner.setProperty(AbstractElasticsearchTransportClientProcessor.PING_TIMEOUT, "5s");
@ -425,7 +417,6 @@ public class TestPutElasticsearch {
System.out.println("Starting test " + new Object() {
}.getClass().getEnclosingMethod().getName());
final TestRunner runner = TestRunners.newTestRunner(new PutElasticsearch());
runner.setValidateExpressionUsage(false);
//Local Cluster - Mac pulled from brew
runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch_brew");
@ -457,7 +448,6 @@ public class TestPutElasticsearch {
System.out.println("Starting test " + new Object() {
}.getClass().getEnclosingMethod().getName());
final TestRunner runner = TestRunners.newTestRunner(new PutElasticsearch());
runner.setValidateExpressionUsage(false);
//Local Cluster - Mac pulled from brew
runner.setProperty(AbstractElasticsearchTransportClientProcessor.CLUSTER_NAME, "elasticsearch_brew");

View File

@ -16,13 +16,17 @@
*/
package org.apache.nifi.processors.elasticsearch;
import okhttp3.Call;
import okhttp3.MediaType;
import okhttp3.OkHttpClient;
import okhttp3.Protocol;
import okhttp3.Request;
import okhttp3.Response;
import okhttp3.ResponseBody;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.net.ConnectException;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import org.apache.commons.io.IOUtils;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
@ -36,16 +40,13 @@ import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.io.IOException;
import java.net.ConnectException;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import okhttp3.Call;
import okhttp3.MediaType;
import okhttp3.OkHttpClient;
import okhttp3.Protocol;
import okhttp3.Request;
import okhttp3.Response;
import okhttp3.ResponseBody;
public class TestPutElasticsearchHttp {
@ -66,7 +67,6 @@ public class TestPutElasticsearchHttp {
@Test
public void testPutElasticSearchOnTriggerIndex() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@ -88,7 +88,6 @@ public class TestPutElasticsearchHttp {
@Test
public void testPutElasticSearchOnTriggerUpdate() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@ -110,7 +109,6 @@ public class TestPutElasticsearchHttp {
@Test
public void testPutElasticSearchOnTriggerDelete() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@ -132,7 +130,6 @@ public class TestPutElasticsearchHttp {
@Test
public void testPutElasticSearchOnTriggerEL() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "${es.url}");
runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@ -159,7 +156,6 @@ public class TestPutElasticsearchHttp {
@Test
public void testPutElasticSearchOnTriggerBadIndexOp() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@ -181,7 +177,6 @@ public class TestPutElasticsearchHttp {
@Test
public void testPutElasticSearchInvalidConfig() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@ -202,7 +197,6 @@ public class TestPutElasticsearchHttp {
PutElasticsearchTestProcessor processor = new PutElasticsearchTestProcessor(true);
processor.setStatus(100, "Should fail");
runner = TestRunners.newTestRunner(processor); // simulate failures
runner.setValidateExpressionUsage(false);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
runner.setProperty(PutElasticsearchHttp.TYPE, "status");
@ -229,7 +223,6 @@ public class TestPutElasticsearchHttp {
PutElasticsearchTestProcessor processor = new PutElasticsearchTestProcessor(true);
processor.setStatus(-1, "Connection Exception");
runner = TestRunners.newTestRunner(processor); // simulate failures
runner.setValidateExpressionUsage(false);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
runner.setProperty(PutElasticsearchHttp.TYPE, "status");
@ -246,7 +239,6 @@ public class TestPutElasticsearchHttp {
@Test
public void testPutElasticsearchOnTriggerWithNoIdAttribute() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(true)); // simulate failures
runner.setValidateExpressionUsage(false);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
runner.setProperty(PutElasticsearchHttp.TYPE, "status");
@ -266,7 +258,6 @@ public class TestPutElasticsearchHttp {
@Test
public void testPutElasticsearchOnTriggerWithIndexFromAttribute() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false));
runner.setValidateExpressionUsage(false);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttp.INDEX, "${i}");
runner.setProperty(PutElasticsearchHttp.TYPE, "${type}");
@ -300,7 +291,6 @@ public class TestPutElasticsearchHttp {
@Test
public void testPutElasticSearchOnTriggerWithInvalidIndexOp() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchTestProcessor(false)); // no failures
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@ -330,7 +320,6 @@ public class TestPutElasticsearchHttp {
p.setExpectedUrl("http://127.0.0.1:9200/_bulk?pipeline=my-pipeline");
runner = TestRunners.newTestRunner(p);
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
runner.setProperty(PutElasticsearchHttp.TYPE, "status");
@ -355,7 +344,6 @@ public class TestPutElasticsearchHttp {
PutElasticsearchTestProcessor processor = new PutElasticsearchTestProcessor(true);
processor.setResultField("not_found");
runner = TestRunners.newTestRunner(processor); // simulate failures
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttp.INDEX_OP, "delete");
runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@ -447,6 +435,7 @@ public class TestPutElasticsearchHttp {
});
}
@Override
protected OkHttpClient getClient() {
return client;
}
@ -469,7 +458,6 @@ public class TestPutElasticsearchHttp {
System.out.println("Starting test " + new Object() {
}.getClass().getEnclosingMethod().getName());
final TestRunner runner = TestRunners.newTestRunner(new PutElasticsearchHttp());
runner.setValidateExpressionUsage(false);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttp.INDEX, "doc");
@ -493,7 +481,6 @@ public class TestPutElasticsearchHttp {
System.out.println("Starting test " + new Object() {
}.getClass().getEnclosingMethod().getName());
final TestRunner runner = TestRunners.newTestRunner(new PutElasticsearchHttp());
runner.setValidateExpressionUsage(false);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttp.INDEX, "doc");

View File

@ -16,13 +16,18 @@
*/
package org.apache.nifi.processors.elasticsearch;
import okhttp3.Call;
import okhttp3.MediaType;
import okhttp3.OkHttpClient;
import okhttp3.Protocol;
import okhttp3.Request;
import okhttp3.Response;
import okhttp3.ResponseBody;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.net.ConnectException;
import java.util.HashMap;
import java.util.List;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.provenance.ProvenanceEventRecord;
@ -37,17 +42,13 @@ import org.junit.After;
import org.junit.Ignore;
import org.junit.Test;
import java.io.IOException;
import java.net.ConnectException;
import java.util.HashMap;
import java.util.List;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import okhttp3.Call;
import okhttp3.MediaType;
import okhttp3.OkHttpClient;
import okhttp3.Protocol;
import okhttp3.Request;
import okhttp3.Response;
import okhttp3.ResponseBody;
public class TestPutElasticsearchHttpRecord {
@ -62,7 +63,6 @@ public class TestPutElasticsearchHttpRecord {
public void testPutElasticSearchOnTriggerIndex() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecordTestProcessor(false)); // no failures
generateTestData();
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@ -88,7 +88,6 @@ public class TestPutElasticsearchHttpRecord {
public void testPutElasticSearchOnTriggerUpdate() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecordTestProcessor(false)); // no failures
generateTestData();
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@ -110,7 +109,6 @@ public class TestPutElasticsearchHttpRecord {
public void testPutElasticSearchOnTriggerDelete() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecordTestProcessor(false)); // no failures
generateTestData();
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@ -132,7 +130,6 @@ public class TestPutElasticsearchHttpRecord {
public void testPutElasticSearchOnTriggerEL() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecordTestProcessor(false)); // no failures
generateTestData();
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "${es.url}");
runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@ -159,7 +156,6 @@ public class TestPutElasticsearchHttpRecord {
public void testPutElasticSearchOnTriggerBadIndexOp() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecordTestProcessor(false)); // no failures
generateTestData();
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@ -181,7 +177,6 @@ public class TestPutElasticsearchHttpRecord {
public void testPutElasticSearchInvalidConfig() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecordTestProcessor(false)); // no failures
generateTestData();
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@ -276,7 +271,6 @@ public class TestPutElasticsearchHttpRecord {
public void testPutElasticsearchOnTriggerWithIndexFromAttribute() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecordTestProcessor(false));
generateTestData();
runner.setValidateExpressionUsage(false);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttpRecord.INDEX, "${i}");
runner.setProperty(PutElasticsearchHttpRecord.TYPE, "${type}");
@ -310,7 +304,6 @@ public class TestPutElasticsearchHttpRecord {
public void testPutElasticSearchOnTriggerWithInvalidIndexOp() throws IOException {
runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecordTestProcessor(false)); // no failures
generateTestData();
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@ -339,7 +332,6 @@ public class TestPutElasticsearchHttpRecord {
p.setExpectedUrl("http://127.0.0.1:9200/_bulk?pipeline=my-pipeline");
runner = TestRunners.newTestRunner(p);
generateTestData();
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@ -427,6 +419,7 @@ public class TestPutElasticsearchHttpRecord {
});
}
@Override
protected OkHttpClient getClient() {
return client;
}
@ -449,7 +442,6 @@ public class TestPutElasticsearchHttpRecord {
System.out.println("Starting test " + new Object() {
}.getClass().getEnclosingMethod().getName());
final TestRunner runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecord());
runner.setValidateExpressionUsage(false);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");
@ -476,7 +468,6 @@ public class TestPutElasticsearchHttpRecord {
System.out.println("Starting test " + new Object() {
}.getClass().getEnclosingMethod().getName());
final TestRunner runner = TestRunners.newTestRunner(new PutElasticsearchHttpRecord());
runner.setValidateExpressionUsage(false);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(PutElasticsearchHttpRecord.INDEX, "doc");

View File

@ -62,7 +62,6 @@ public class TestQueryElasticsearchHttp {
@Test
public void testQueryElasticsearchOnTrigger_withInput() throws IOException {
runner = TestRunners.newTestRunner(new QueryElasticsearchHttpTestProcessor());
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
@ -81,7 +80,6 @@ public class TestQueryElasticsearchHttp {
@Test
public void testQueryElasticsearchOnTrigger_withInput_EL() throws IOException {
runner = TestRunners.newTestRunner(new QueryElasticsearchHttpTestProcessor());
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "${es.url}");
runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
@ -104,7 +102,6 @@ public class TestQueryElasticsearchHttp {
@Test
public void testQueryElasticsearchOnTrigger_withInput_attributeTarget() throws IOException {
runner = TestRunners.newTestRunner(new QueryElasticsearchHttpTestProcessor());
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
@ -130,7 +127,6 @@ public class TestQueryElasticsearchHttp {
@Test
public void testQueryElasticsearchOnTrigger_withNoInput() throws IOException {
runner = TestRunners.newTestRunner(new QueryElasticsearchHttpTestProcessor());
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
@ -175,7 +171,6 @@ public class TestQueryElasticsearchHttp {
@Test
public void testQueryElasticsearchOnTriggerWithFields() throws IOException {
runner = TestRunners.newTestRunner(new QueryElasticsearchHttpTestProcessor());
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
@ -195,7 +190,6 @@ public class TestQueryElasticsearchHttp {
@Test
public void testQueryElasticsearchOnTriggerWithLimit() throws IOException {
runner = TestRunners.newTestRunner(new QueryElasticsearchHttpTestProcessor());
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
@ -221,7 +215,6 @@ public class TestQueryElasticsearchHttp {
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
runner.setProperty(QueryElasticsearchHttp.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(QueryElasticsearchHttp.QUERY, "${doc_id}");
runner.enqueue("".getBytes(), new HashMap<String, String>() {
@ -247,7 +240,6 @@ public class TestQueryElasticsearchHttp {
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
runner.setProperty(QueryElasticsearchHttp.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(QueryElasticsearchHttp.QUERY, "${doc_id}");
runner.enqueue("".getBytes(), new HashMap<String, String>() {
@ -273,7 +265,6 @@ public class TestQueryElasticsearchHttp {
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
runner.setProperty(QueryElasticsearchHttp.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(QueryElasticsearchHttp.QUERY, "${doc_id}");
runner.enqueue("".getBytes(), new HashMap<String, String>() {
@ -299,7 +290,6 @@ public class TestQueryElasticsearchHttp {
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
runner.setProperty(QueryElasticsearchHttp.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(QueryElasticsearchHttp.QUERY, "${doc_id}");
runner.enqueue("".getBytes(), new HashMap<String, String>() {
@ -326,7 +316,6 @@ public class TestQueryElasticsearchHttp {
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
runner.setProperty(QueryElasticsearchHttp.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(QueryElasticsearchHttp.QUERY, "${doc_id}");
runner.setIncomingConnection(false);
@ -348,7 +337,6 @@ public class TestQueryElasticsearchHttp {
runner.setProperty(QueryElasticsearchHttp.PROP_SSL_CONTEXT_SERVICE, "ssl-context");
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
runner.setValidateExpressionUsage(true);
runner.setProperty(QueryElasticsearchHttp.QUERY, "${doc_id}");
// Allow time for the controller service to fully initialize
@ -367,7 +355,6 @@ public class TestQueryElasticsearchHttp {
QueryElasticsearchHttpTestProcessor p = new QueryElasticsearchHttpTestProcessor();
p.setExpectedParam("myparam=myvalue");
runner = TestRunners.newTestRunner(p);
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(QueryElasticsearchHttp.INDEX, "doc");
@ -480,6 +467,7 @@ public class TestQueryElasticsearchHttp {
});
}
@Override
protected OkHttpClient getClient() {
return client;
}

View File

@ -63,7 +63,6 @@ public class TestScrollElasticsearchHttp {
@Test
public void testScrollElasticsearchOnTrigger_withNoInput() throws IOException {
runner = TestRunners.newTestRunner(new ScrollElasticsearchHttpTestProcessor());
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
@ -83,7 +82,6 @@ public class TestScrollElasticsearchHttp {
@Test
public void testScrollElasticsearchOnTrigger_withNoInput_EL() throws IOException {
runner = TestRunners.newTestRunner(new ScrollElasticsearchHttpTestProcessor());
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "${es.url}");
runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
@ -131,7 +129,6 @@ public class TestScrollElasticsearchHttp {
@Test
public void testScrollElasticsearchOnTriggerWithFields() throws IOException {
runner = TestRunners.newTestRunner(new ScrollElasticsearchHttpTestProcessor());
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
@ -157,7 +154,6 @@ public class TestScrollElasticsearchHttp {
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
runner.setProperty(ScrollElasticsearchHttp.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(ScrollElasticsearchHttp.QUERY, "${doc_id}");
runner.setIncomingConnection(false);
@ -182,7 +178,6 @@ public class TestScrollElasticsearchHttp {
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
runner.setProperty(ScrollElasticsearchHttp.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(ScrollElasticsearchHttp.QUERY, "${doc_id}");
runner.setIncomingConnection(false);
@ -207,7 +202,6 @@ public class TestScrollElasticsearchHttp {
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
runner.setProperty(ScrollElasticsearchHttp.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(ScrollElasticsearchHttp.QUERY, "${doc_id}");
runner.enqueue("".getBytes(), new HashMap<String, String>() {
@ -232,7 +226,6 @@ public class TestScrollElasticsearchHttp {
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
runner.setProperty(ScrollElasticsearchHttp.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(ScrollElasticsearchHttp.QUERY, "${doc_id}");
runner.setIncomingConnection(false);
@ -254,7 +247,6 @@ public class TestScrollElasticsearchHttp {
runner.setProperty(ScrollElasticsearchHttp.PROP_SSL_CONTEXT_SERVICE, "ssl-context");
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
runner.setValidateExpressionUsage(true);
runner.setProperty(ScrollElasticsearchHttp.QUERY, "${doc_id}");
runner.setIncomingConnection(false);
@ -278,7 +270,6 @@ public class TestScrollElasticsearchHttp {
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
runner.setProperty(ScrollElasticsearchHttp.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(ScrollElasticsearchHttp.QUERY, "${doc_id}");
runner.enqueue("".getBytes(), new HashMap<String, String>() {
@ -302,7 +293,6 @@ public class TestScrollElasticsearchHttp {
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
runner.setProperty(ScrollElasticsearchHttp.TYPE, "status");
runner.setValidateExpressionUsage(true);
runner.setProperty(ScrollElasticsearchHttp.QUERY, "${doc_id}");
runner.enqueue("".getBytes(), new HashMap<String, String>() {
@ -323,7 +313,6 @@ public class TestScrollElasticsearchHttp {
ScrollElasticsearchHttpTestProcessor p = new ScrollElasticsearchHttpTestProcessor();
p.setExpectedParam("myparam=myvalue");
runner = TestRunners.newTestRunner(p);
runner.setValidateExpressionUsage(true);
runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "http://127.0.0.1:9200");
runner.setProperty(ScrollElasticsearchHttp.INDEX, "doc");
@ -438,6 +427,7 @@ public class TestScrollElasticsearchHttp {
});
}
@Override
protected OkHttpClient getClient() {
return client;
}

View File

@ -36,6 +36,7 @@ import javax.mail.MessagingException;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
@ -62,7 +63,7 @@ abstract class AbstractEmailProcessor<T extends AbstractMailReceiver> extends Ab
.displayName("Host Name")
.description("Network address of Email server (e.g., pop.gmail.com, imap.gmail.com . . .)")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
@ -70,7 +71,7 @@ abstract class AbstractEmailProcessor<T extends AbstractMailReceiver> extends Ab
.displayName("Port")
.description("Numeric value identifying Port of Email server (e.g., 993)")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.PORT_VALIDATOR)
.build();
public static final PropertyDescriptor USER = new PropertyDescriptor.Builder()
@ -78,7 +79,7 @@ abstract class AbstractEmailProcessor<T extends AbstractMailReceiver> extends Ab
.displayName("User Name")
.description("User Name used for authentication and authorization with Email server.")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
@ -86,7 +87,7 @@ abstract class AbstractEmailProcessor<T extends AbstractMailReceiver> extends Ab
.displayName("Password")
.description("Password used for authentication and authorization with Email server.")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.sensitive(true)
.build();
@ -95,7 +96,7 @@ abstract class AbstractEmailProcessor<T extends AbstractMailReceiver> extends Ab
.displayName("Folder")
.description("Email folder to retrieve messages from (e.g., INBOX)")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.defaultValue("INBOX")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -104,7 +105,7 @@ abstract class AbstractEmailProcessor<T extends AbstractMailReceiver> extends Ab
.displayName("Fetch Size")
.description("Specify the maximum number of Messages to fetch per call to Email Server.")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.defaultValue("10")
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.build();
@ -123,7 +124,7 @@ abstract class AbstractEmailProcessor<T extends AbstractMailReceiver> extends Ab
.description("The amount of time to wait to connect to Email server")
.required(true)
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.defaultValue("30 sec")
.build();

View File

@ -52,6 +52,7 @@ import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.Validator;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
@ -92,7 +93,7 @@ public class ConsumeEWS extends AbstractProcessor {
.displayName("User Name")
.description("User Name used for authentication and authorization with Email server.")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
@ -100,7 +101,7 @@ public class ConsumeEWS extends AbstractProcessor {
.displayName("Password")
.description("Password used for authentication and authorization with Email server.")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.sensitive(true)
.build();
@ -109,7 +110,7 @@ public class ConsumeEWS extends AbstractProcessor {
.displayName("Folder")
.description("Email folder to retrieve messages from (e.g., INBOX)")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.defaultValue("INBOX")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -118,7 +119,7 @@ public class ConsumeEWS extends AbstractProcessor {
.displayName("Fetch Size")
.description("Specify the maximum number of Messages to fetch per call to Email Server.")
.required(true)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.defaultValue("10")
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.build();
@ -137,7 +138,7 @@ public class ConsumeEWS extends AbstractProcessor {
.description("The amount of time to wait to connect to Email server")
.required(true)
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.defaultValue("30 sec")
.build();
public static final PropertyDescriptor EXCHANGE_VERSION = new PropertyDescriptor.Builder()
@ -256,7 +257,7 @@ public class ConsumeEWS extends AbstractProcessor {
this.messageQueue = new ArrayBlockingQueue<>(fetchSize);
}
this.folderName = context.getProperty(FOLDER).getValue();
this.folderName = context.getProperty(FOLDER).evaluateAttributeExpressions().getValue();
Message emailMessage = this.receiveMessage(context);
if (emailMessage != null) {
@ -274,8 +275,8 @@ public class ConsumeEWS extends AbstractProcessor {
final String timeoutInMillis = String.valueOf(context.getProperty(CONNECTION_TIMEOUT).evaluateAttributeExpressions().asTimePeriod(TimeUnit.MILLISECONDS));
service.setTimeout(Integer.parseInt(timeoutInMillis));
String userEmail = context.getProperty(USER).getValue();
String password = context.getProperty(PASSWORD).getValue();
String userEmail = context.getProperty(USER).evaluateAttributeExpressions().getValue();
String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
ExchangeCredentials credentials = new WebCredentials(userEmail, password);
service.setCredentials(credentials);

View File

@ -30,6 +30,7 @@ import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
@ -98,7 +99,7 @@ public class ExtractEmailHeaders extends AbstractProcessor {
"NOTE the header key is case insensitive and will be matched as lower-case." +
" Values will respect email contents.")
.required(false)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.defaultValue("x-mailer")
.build();

View File

@ -41,6 +41,7 @@ import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
import org.apache.nifi.processor.DataUnit;
@ -85,7 +86,7 @@ public class ListenSMTP extends AbstractSessionFactoryProcessor {
+ "NOTE that on Unix derivative operating systems this port must "
+ "be higher than 1024 unless NiFi is running as with root user permissions.")
.required(true)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.addValidator(StandardValidators.PORT_VALIDATOR)
.build();
@ -95,7 +96,7 @@ public class ListenSMTP extends AbstractSessionFactoryProcessor {
.description("The maximum number of simultaneous SMTP connections.")
.required(true)
.defaultValue("1")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.addValidator(StandardValidators.INTEGER_VALIDATOR)
.build();
@ -105,7 +106,7 @@ public class ListenSMTP extends AbstractSessionFactoryProcessor {
.description("The maximum time to wait for an action of SMTP client.")
.defaultValue("60 seconds")
.required(true)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.build();
@ -115,7 +116,7 @@ public class ListenSMTP extends AbstractSessionFactoryProcessor {
.description("The maximum number of bytes the server will accept.")
.required(true)
.defaultValue("20 MB")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.addValidator(StandardValidators.createDataSizeBoundsValidator(1, Integer.MAX_VALUE))
.build();
@ -141,7 +142,7 @@ public class ListenSMTP extends AbstractSessionFactoryProcessor {
.displayName("SMTP hostname")
.description("The hostname to be embedded into the banner displayed when an "
+ "SMTP client connects to the processor TCP port .")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();

View File

@ -31,6 +31,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.AttributeExpression;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessorInitializationContext;
@ -56,7 +57,7 @@ public abstract class AbstractEnrichIP extends AbstractProcessor {
.displayName("IP Address Attribute")
.required(true)
.description("The name of an attribute whose value is a dotted decimal IP address for which enrichment should occur")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
.build();

View File

@ -26,6 +26,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.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.util.StandardValidators;
@ -44,7 +45,7 @@ public abstract class AbstractEnrichProcessor extends AbstractProcessor {
.displayName("Lookup value")
.required(true)
.description("The value that should be used to populate the query")
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
@ -70,7 +71,7 @@ public abstract class AbstractEnrichProcessor extends AbstractProcessor {
.description("Choice between a splitter and regex matcher used to parse the results of the query into attribute groups.\n" +
"NOTE: This is a multiline regular expression, therefore, the DFM should decide how to handle trailing new line " +
"characters.")
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(false)
.addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
.build();

View File

@ -21,6 +21,7 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.util.StandardValidators;
@ -85,7 +86,7 @@ public class KerberosProperties {
.description("Kerberos principal to authenticate as. Requires nifi.kerberos.krb5.file to be set in your nifi.properties")
.addValidator(kerberosConfigValidator)
.addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
this.kerberosKeytab = new PropertyDescriptor.Builder()
@ -94,7 +95,7 @@ public class KerberosProperties {
.addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
.addValidator(kerberosConfigValidator)
.addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
}

View File

@ -30,6 +30,7 @@ import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.hadoop.KerberosProperties;
import org.apache.nifi.hadoop.SecurityUtil;
import org.apache.nifi.kerberos.KerberosCredentialsService;
@ -78,7 +79,7 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
+ "To use swebhdfs, see 'Additional Details' section of PutHDFS's documentation.")
.required(false)
.addValidator(HadoopValidators.ONE_OR_MORE_FILE_EXISTS_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder()
@ -86,7 +87,7 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
.description("The HDFS directory from which files should be read")
.required(true)
.addValidator(StandardValidators.ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.build();
public static final PropertyDescriptor COMPRESSION_CODEC = new PropertyDescriptor.Builder()
@ -103,7 +104,7 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
.defaultValue("4 hours")
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
public static final PropertyDescriptor ADDITIONAL_CLASSPATH_RESOURCES = new PropertyDescriptor.Builder()

View File

@ -20,6 +20,7 @@ import static org.apache.nifi.processor.util.listen.ListenerProperties.NETWORK_I
import org.apache.nifi.annotation.lifecycle.OnScheduled;
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.ProcessSession;
@ -54,7 +55,7 @@ public abstract class AbstractListenEventBatchingProcessor<E extends Event> exte
"The maximum number of messages to add to a single FlowFile. If multiple messages are available, they will be concatenated along with "
+ "the <Message Delimiter> up to this configured maximum number of messages")
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.expressionLanguageSupported(false)
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.defaultValue("1")
.required(true)
.build();

View File

@ -21,6 +21,7 @@ import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator;
import org.apache.nifi.expression.AttributeExpression;
import org.apache.nifi.expression.ExpressionLanguageScope;
import java.net.NetworkInterface;
import java.net.SocketException;
@ -81,7 +82,7 @@ public class ListenerProperties {
return result;
}
})
.expressionLanguageSupported(true)
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
.build();
}

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