mirror of https://github.com/apache/nifi.git
NIFI-2208 This closes #754. refactored as per comments on JIRA. Reduced API expsosure and tightened lifecycle management.
This commit is contained in:
parent
7f45251bbd
commit
05a99a93cb
|
@ -26,7 +26,7 @@ import java.util.Set;
|
|||
import org.apache.nifi.controller.ControllerService;
|
||||
|
||||
/**
|
||||
* An immutable object for holding information about a type of processor
|
||||
* An immutable object for holding information about a type of component
|
||||
* property.
|
||||
*
|
||||
*/
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.nifi.expression.AttributeValueDecorator;
|
|||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
|
@ -121,144 +122,183 @@ public interface PropertyValue {
|
|||
|
||||
/**
|
||||
* <p>
|
||||
* Replaces values in the Property Value using the NiFi Expression
|
||||
* Language; a PropertyValue with the new value is then returned, supporting
|
||||
* call chaining.
|
||||
* Replaces values in the Property Value using the NiFi Expression Language;
|
||||
* a PropertyValue with the new value is then returned, supporting call
|
||||
* chaining. Before executing the expression language statement any
|
||||
* variables names found within any underlying {@link VariableRegistry} will
|
||||
* be substituted with their values.
|
||||
* </p>
|
||||
*
|
||||
* @return a PropertyValue with the new value is returned, supporting call
|
||||
* chaining
|
||||
*
|
||||
* @throws ProcessException if the Expression cannot be compiled or evaluating
|
||||
* the Expression against the given attributes causes an Exception to be thrown
|
||||
* @throws ProcessException if the Expression cannot be compiled or
|
||||
* evaluating the Expression against the given attributes causes an
|
||||
* Exception to be thrown
|
||||
*/
|
||||
PropertyValue evaluateAttributeExpressions() throws ProcessException;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Replaces values in the Property Value using the NiFi Expression Language;
|
||||
* a PropertyValue with the new value is then returned, supporting call chaining.
|
||||
* a PropertyValue with the new value is then returned, supporting call
|
||||
* chaining. Before executing the expression language statement any
|
||||
* variables names found within any underlying {@link VariableRegistry} will
|
||||
* be substituted with their values.
|
||||
* </p>
|
||||
*
|
||||
* @param attributes a Map of attributes that the Expression can reference, in addition
|
||||
* to JVM System Properties and Environmental Properties.
|
||||
* @param attributes a Map of attributes that the Expression can reference.
|
||||
* These will take precedence over any underlying variable registry values.
|
||||
*
|
||||
* @return a PropertyValue with the new value
|
||||
*
|
||||
* @throws ProcessException if the Expression cannot be compiled or evaluating the Expression against
|
||||
* the given attributes causes an Exception to be thrown
|
||||
* @throws ProcessException if the Expression cannot be compiled or
|
||||
* evaluating the Expression against the given attributes causes an
|
||||
* Exception to be thrown
|
||||
*/
|
||||
PropertyValue evaluateAttributeExpressions(Map<String, String> attributes) throws ProcessException;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Replaces values in the Property Value using the NiFi Expression Language.
|
||||
* The supplied decorator is then given a chance to decorate the
|
||||
* value, and a PropertyValue with the new value is then returned,
|
||||
* supporting call chaining.
|
||||
* The supplied decorator is then given a chance to decorate the value, and
|
||||
* a PropertyValue with the new value is then returned, supporting call
|
||||
* chaining. Before executing the expression language statement any
|
||||
* variables names found within any underlying {@link VariableRegistry} will
|
||||
* be substituted with their values.
|
||||
* </p>
|
||||
*
|
||||
* @param attributes a Map of attributes that the Expression can reference, in addition
|
||||
* to JVM System Properties and Environmental Properties.
|
||||
* @param decorator the decorator to use in order to update the values returned by the Expression Language
|
||||
* @param attributes a Map of attributes that the Expression can reference.
|
||||
* These will take precedence over any variables in any underlying variable
|
||||
* registry.
|
||||
* @param decorator the decorator to use in order to update the values
|
||||
* returned after variable substitution and expression language evaluation.
|
||||
*
|
||||
* @return a PropertyValue with the new value
|
||||
*
|
||||
* @throws ProcessException if the Expression cannot be compiled or evaluating the Expression against
|
||||
* the given attributes causes an Exception to be thrown
|
||||
* @throws ProcessException if the Expression cannot be compiled or
|
||||
* evaluating the Expression against the given attributes causes an
|
||||
* Exception to be thrown
|
||||
*/
|
||||
PropertyValue evaluateAttributeExpressions(Map<String, String> attributes, AttributeValueDecorator decorator) throws ProcessException;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Replaces values in the Property Value using the NiFi Expression
|
||||
* Language; a PropertyValue with the new value is then returned, supporting
|
||||
* call chaining.
|
||||
* Replaces values in the Property Value using the NiFi Expression Language;
|
||||
* a PropertyValue with the new value is then returned, supporting call
|
||||
* chaining. Before executing the expression language statement any
|
||||
* variables names found within any underlying {@link VariableRegistry} will
|
||||
* be substituted with their values.
|
||||
* </p>
|
||||
*
|
||||
* @param flowFile to evaluate attributes of
|
||||
* @param flowFile to evaluate attributes of. It's flow file properties and
|
||||
* then flow file attributes will take precedence over any underlying
|
||||
* variable registry.
|
||||
* @return a PropertyValue with the new value is returned, supporting call
|
||||
* chaining
|
||||
*
|
||||
* @throws ProcessException if the Expression cannot be compiled or evaluating
|
||||
* the Expression against the given attributes causes an Exception to be thrown
|
||||
* @throws ProcessException if the Expression cannot be compiled or
|
||||
* evaluating the Expression against the given attributes causes an
|
||||
* Exception to be thrown
|
||||
*/
|
||||
PropertyValue evaluateAttributeExpressions(FlowFile flowFile) throws ProcessException;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Replaces values in the Property Value using the NiFi Expression
|
||||
* Language; a PropertyValue with the new value is then returned, supporting
|
||||
* call chaining.
|
||||
* Replaces values in the Property Value using the NiFi Expression Language;
|
||||
* a PropertyValue with the new value is then returned, supporting call
|
||||
* chaining. Before executing the expression language statement any
|
||||
* variables names found within any underlying {@link VariableRegistry} will
|
||||
* be substituted with their values.
|
||||
* </p>
|
||||
*
|
||||
* @param flowFile to evaluate attributes of
|
||||
* @param additionalAttributes a Map of additional attributes that the Expression can reference. If entries in
|
||||
* this Map conflict with entries in the FlowFile's attributes, the entries in this Map are given a higher priority.
|
||||
* @param flowFile to evaluate attributes of. It's flow file properties and
|
||||
* then flow file attributes will take precedence over any underlying
|
||||
* variable registry.
|
||||
* @param additionalAttributes a Map of additional attributes that the
|
||||
* Expression can reference. These attributes will take precedence over any
|
||||
* conflicting attributes in the provided flowfile or any underlying
|
||||
* variable registry.
|
||||
*
|
||||
* @return a PropertyValue with the new value is returned, supporting call
|
||||
* chaining
|
||||
*
|
||||
* @throws ProcessException if the Expression cannot be compiled or evaluating
|
||||
* the Expression against the given attributes causes an Exception to be thrown
|
||||
* @throws ProcessException if the Expression cannot be compiled or
|
||||
* evaluating the Expression against the given attributes causes an
|
||||
* Exception to be thrown
|
||||
*/
|
||||
PropertyValue evaluateAttributeExpressions(FlowFile flowFile, Map<String, String> additionalAttributes) throws ProcessException;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Replaces values in the Property Value using the NiFi Expression
|
||||
* Language; a PropertyValue with the new value is then returned, supporting
|
||||
* call chaining.
|
||||
* Replaces values in the Property Value using the NiFi Expression Language;
|
||||
* a PropertyValue with the new value is then returned, supporting call
|
||||
* chaining. Before executing the expression language statement any
|
||||
* variables names found within any underlying {@link VariableRegistry} will
|
||||
* be substituted with their values.
|
||||
* </p>
|
||||
*
|
||||
* @param flowFile to evaluate attributes of
|
||||
* @param additionalAttributes a Map of additional attributes that the Expression can reference. If entries in
|
||||
* this Map conflict with entries in the FlowFile's attributes, the entries in this Map are given a higher priority.
|
||||
* @param decorator the decorator to use in order to update the values returned by the Expression Language
|
||||
* @param flowFile to evaluate attributes of. It's flow file properties and
|
||||
* then flow file attributes will take precedence over any underlying
|
||||
* variable registry.
|
||||
* @param additionalAttributes a Map of additional attributes that the
|
||||
* Expression can reference. These attributes will take precedence over any
|
||||
* conflicting attributes in the provided flowfile or any underlying
|
||||
* variable registry.
|
||||
* @param decorator the decorator to use in order to update the values
|
||||
* returned after variable substitution and expression language evaluation.
|
||||
*
|
||||
* @return a PropertyValue with the new value is returned, supporting call
|
||||
* chaining
|
||||
*
|
||||
* @throws ProcessException if the Expression cannot be compiled or evaluating
|
||||
* the Expression against the given attributes causes an Exception to be thrown
|
||||
* @throws ProcessException if the Expression cannot be compiled or
|
||||
* evaluating the Expression against the given attributes causes an
|
||||
* Exception to be thrown
|
||||
*/
|
||||
PropertyValue evaluateAttributeExpressions(FlowFile flowFile, Map<String, String> additionalAttributes, AttributeValueDecorator decorator) throws ProcessException;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Replaces values in the Property Value using the NiFi Expression
|
||||
* Language. The supplied decorator is then given a chance to decorate the
|
||||
* value, and a PropertyValue with the new value is then returned,
|
||||
* supporting call chaining.
|
||||
* Replaces values in the Property Value using the NiFi Expression Language.
|
||||
* The supplied decorator is then given a chance to decorate the value, and
|
||||
* a PropertyValue with the new value is then returned, supporting call
|
||||
* chaining. Before executing the expression language statement any
|
||||
* variables names found within any underlying {@link VariableRegistry} will
|
||||
* be substituted with their values.
|
||||
* </p>
|
||||
*
|
||||
* @param decorator The supplied decorator is then given a chance to
|
||||
* decorate the value
|
||||
* @param decorator the decorator to use in order to update the values
|
||||
* returned after variable substitution and expression language evaluation.
|
||||
* @return a PropertyValue with the new value is then returned, supporting
|
||||
* call chaining
|
||||
*
|
||||
* @throws ProcessException if the Expression cannot be compiled or evaluating
|
||||
* the Expression against the given attributes causes an Exception to be thrown
|
||||
* @throws ProcessException if the Expression cannot be compiled or
|
||||
* evaluating the Expression against the given attributes causes an
|
||||
* Exception to be thrown
|
||||
*/
|
||||
PropertyValue evaluateAttributeExpressions(AttributeValueDecorator decorator) throws ProcessException;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Replaces values in the Property Value using the NiFi Expression
|
||||
* Language. The supplied decorator is then given a chance to decorate the
|
||||
* value, and a PropertyValue with the new value is then returned,
|
||||
* supporting call chaining.
|
||||
* Replaces values in the Property Value using the NiFi Expression Language.
|
||||
* The supplied decorator is then given a chance to decorate the value, and
|
||||
* a PropertyValue with the new value is then returned, supporting call
|
||||
* chaining. Before executing the expression language statement any
|
||||
* variables names found within any underlying {@link VariableRegistry} will
|
||||
* be substituted with their values.
|
||||
* </p>
|
||||
*
|
||||
* @param flowFile to evaluate expressions against
|
||||
* @param decorator The supplied decorator is then given a chance to
|
||||
* decorate the value
|
||||
* @param decorator the decorator to use in order to update the values
|
||||
* returned after variable substitution and expression language evaluation.
|
||||
*
|
||||
*
|
||||
* @return a PropertyValue with the new value is then returned, supporting
|
||||
* call chaining
|
||||
*
|
||||
* @throws ProcessException if the Expression cannot be compiled or evaluating
|
||||
* the Expression against the given attributes causes an Exception to be thrown
|
||||
* @throws ProcessException if the Expression cannot be compiled or
|
||||
* evaluating the Expression against the given attributes causes an
|
||||
* Exception to be thrown
|
||||
*/
|
||||
PropertyValue evaluateAttributeExpressions(FlowFile flowFile, AttributeValueDecorator decorator) throws ProcessException;
|
||||
}
|
||||
|
|
|
@ -18,32 +18,37 @@ package org.apache.nifi.expression;
|
|||
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
|
||||
/**
|
||||
* Defines a type of expression language statement that can be applied
|
||||
* parameterized by various attributes and properties as specified in each of
|
||||
* the method calls. AttributeExpression evaluations may be also backed by a
|
||||
* {@link VariableRegistry} used to substitute attributes and variables found in
|
||||
* the expression for which the registry has a value.
|
||||
*/
|
||||
public interface AttributeExpression {
|
||||
|
||||
/**
|
||||
* @return Evaluates the expression without providing any FlowFile Attributes. This
|
||||
* will evaluate the expression based only on System Properties and JVM
|
||||
* Environment properties
|
||||
* @return Evaluates the expression without any additional attributes.
|
||||
* @throws ProcessException if unable to evaluate
|
||||
*/
|
||||
String evaluate() throws ProcessException;
|
||||
|
||||
/**
|
||||
* Evaluates the expression without providing any FlowFile Attributes. This
|
||||
* will evaluate the expression based only on System Properties and JVM
|
||||
* Environment properties but allows the values to be decorated
|
||||
* Evaluates the expression without additional attributes but enables the
|
||||
* expression result to be decorated before returning.
|
||||
*
|
||||
* @param decorator for attribute value
|
||||
* @param decorator to execute on the resulting expression value
|
||||
* @return evaluated value
|
||||
* @throws ProcessException if failure in evaluation
|
||||
*/
|
||||
String evaluate(AttributeValueDecorator decorator) throws ProcessException;
|
||||
|
||||
/**
|
||||
* Evaluates the expression, providing access to the attributes, file size,
|
||||
* id, etc. of the given FlowFile, as well as System Properties and JVM
|
||||
* Environment properties
|
||||
* Evaluates the expression providing access to additional variables
|
||||
* including the flow file properties such as file size, identifier, etc..
|
||||
* and also all of the flow file attributes.
|
||||
*
|
||||
* @param flowFile to evaluate
|
||||
* @return evaluated value
|
||||
|
@ -52,9 +57,11 @@ public interface AttributeExpression {
|
|||
String evaluate(FlowFile flowFile) throws ProcessException;
|
||||
|
||||
/**
|
||||
* Evaluates the expression, providing access to the attributes, file size,
|
||||
* id, etc. of the given FlowFile, as well as System Properties and JVM
|
||||
* Environment properties and allows the values to be decorated
|
||||
* Evaluates the expression providing access to additional variables
|
||||
* including the flow file properties such as file size, identifier, etc..
|
||||
* and also all of the flow file attributes. The resulting value after
|
||||
* executing any variable substitution and expression evaluation is run
|
||||
* through the given decorator and returned.
|
||||
*
|
||||
* @param flowFile to evaluate
|
||||
* @param decorator for evaluation
|
||||
|
|
|
@ -16,12 +16,11 @@
|
|||
*/
|
||||
package org.apache.nifi.expression;
|
||||
|
||||
@FunctionalInterface
|
||||
public interface AttributeValueDecorator {
|
||||
|
||||
/**
|
||||
* Decorates the value of a FlowFile Attribute or System/JVM property in
|
||||
* some way
|
||||
*
|
||||
* Decorates the given value
|
||||
* @param attributeValue to decorate
|
||||
* @return decorated value
|
||||
*/
|
||||
|
|
|
@ -1,67 +0,0 @@
|
|||
/*
|
||||
* 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.registry;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Map;
|
||||
|
||||
|
||||
abstract class FileVariableRegistry extends MultiMapVariableRegistry {
|
||||
|
||||
FileVariableRegistry() {
|
||||
super();
|
||||
}
|
||||
|
||||
FileVariableRegistry(File... files) throws IOException{
|
||||
super();
|
||||
addVariables(files);
|
||||
}
|
||||
|
||||
FileVariableRegistry(Path... paths) throws IOException{
|
||||
super();
|
||||
addVariables(paths);
|
||||
}
|
||||
|
||||
private void addVariables(File ...files) throws IOException{
|
||||
if(files != null) {
|
||||
for (final File file : files) {
|
||||
Map<String,String> map = convertFile(file);
|
||||
if(map != null) {
|
||||
registry.addMap(convertFile(file));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
private void addVariables(Path ...paths) throws IOException{
|
||||
if(paths != null) {
|
||||
for (final Path path : paths) {
|
||||
Map<String,String> map = convertFile(path.toFile());
|
||||
if(map != null) {
|
||||
registry.addMap(map);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected abstract Map<String,String> convertFile(File file) throws IOException;
|
||||
|
||||
}
|
|
@ -1,145 +0,0 @@
|
|||
/*
|
||||
* 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.registry;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class ImmutableMultiMap<V> implements Map<String,V> {
|
||||
|
||||
private final List<Map<String,V>> maps;
|
||||
|
||||
ImmutableMultiMap() {
|
||||
this.maps = new ArrayList<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
return keySet().size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEmpty() {
|
||||
for (final Map<String,V> map : maps) {
|
||||
if (!map.isEmpty()) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean containsKey(final Object key) {
|
||||
if (key == null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
for (final Map<String,V> map : maps) {
|
||||
if (map.containsKey(key)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean containsValue(final Object value) {
|
||||
for (final Map<String,V> map : maps) {
|
||||
if (map.containsValue(value)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("rawtypes")
|
||||
public V get(final Object key) {
|
||||
if (key == null) {
|
||||
throw new IllegalArgumentException("Null Keys are not allowed");
|
||||
}
|
||||
|
||||
for (final Map<String,V> map : maps) {
|
||||
final V val = map.get(key);
|
||||
if (val != null) {
|
||||
return val;
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public V put(String key, V value) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public V remove(Object key) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void putAll(Map<? extends String, ? extends V> m) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
public Set<String> keySet() {
|
||||
final Set<String> keySet = new HashSet<>();
|
||||
for (final Map map : maps) {
|
||||
keySet.addAll(map.keySet());
|
||||
}
|
||||
return keySet;
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
public Collection<V> values() {
|
||||
final Set<V> values = new HashSet<>();
|
||||
for (final Map map : maps) {
|
||||
values.addAll(map.values());
|
||||
}
|
||||
return values;
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
public Set<java.util.Map.Entry<String, V>> entrySet() {
|
||||
final Set<java.util.Map.Entry<String, V>> entrySet = new HashSet<>();
|
||||
for (final Map map : maps) {
|
||||
entrySet.addAll(map.entrySet());
|
||||
}
|
||||
return entrySet;
|
||||
}
|
||||
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
void addMap(Map<String,V> map){
|
||||
this.maps.add(map);
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -1,70 +0,0 @@
|
|||
/*
|
||||
* 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.registry;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/***
|
||||
* This implementation of variable registry uses the ImmutableMultiMap which stores one or more
|
||||
* registries that can be searched, accessed and appended. NOTE: Duplicate values within
|
||||
* or between added registries will be stored however on retrieval the first value encountered will be returned.
|
||||
* */
|
||||
public class MultiMapVariableRegistry implements VariableRegistry {
|
||||
|
||||
protected final ImmutableMultiMap<String> registry;
|
||||
|
||||
MultiMapVariableRegistry() {
|
||||
this.registry = new ImmutableMultiMap<>();
|
||||
}
|
||||
|
||||
@SafeVarargs
|
||||
MultiMapVariableRegistry(Map<String,String>...maps){
|
||||
this();
|
||||
if(maps != null) {
|
||||
for (Map<String, String> map : maps) {
|
||||
addVariables(map);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void addVariables(Map<String, String> map) {
|
||||
this.registry.addMap(map);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addRegistry(VariableRegistry variableRegistry) {
|
||||
if(variableRegistry != null && !variableRegistry.getVariables().isEmpty()) {
|
||||
this.registry.addMap(variableRegistry.getVariables());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, String> getVariables() {
|
||||
return registry;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getVariableValue(String variable) {
|
||||
return registry.get(variable);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getVariableNames() {
|
||||
return this.registry.keySet();
|
||||
}
|
||||
}
|
|
@ -1,82 +0,0 @@
|
|||
/*
|
||||
* 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.registry;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.file.Path;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
class PropertiesVariableRegistry extends FileVariableRegistry {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(PropertiesVariableRegistry.class);
|
||||
|
||||
PropertiesVariableRegistry(File... files) throws IOException{
|
||||
super(files);
|
||||
}
|
||||
|
||||
PropertiesVariableRegistry(Path... paths) throws IOException {
|
||||
super(paths);
|
||||
}
|
||||
|
||||
PropertiesVariableRegistry(Properties...properties){
|
||||
super();
|
||||
addVariables(properties);
|
||||
}
|
||||
|
||||
private void addVariables(Properties... properties){
|
||||
if(properties != null) {
|
||||
for (Properties props : properties) {
|
||||
addVariables(convertToMap(props));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<String,String> convertFile(File file) throws IOException{
|
||||
|
||||
if(file.exists()) {
|
||||
try (final InputStream inStream = new BufferedInputStream(new FileInputStream(file))) {
|
||||
Properties properties = new Properties();
|
||||
properties.load(inStream);
|
||||
return convertToMap(properties);
|
||||
}
|
||||
}else{
|
||||
LOG.warn("Could not add file " + file.getName() + ". file did not exist.");
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private Map<String,String> convertToMap(Properties properties){
|
||||
HashMap<String,String> propertiesMap = new HashMap<>(properties.keySet().size());
|
||||
for(Object key: properties.keySet()){
|
||||
propertiesMap.put((String)key,(String) properties.get(key));
|
||||
}
|
||||
return propertiesMap;
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -0,0 +1,154 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.registry;
|
||||
|
||||
/**
|
||||
* An immutable object for holding information about a type of processor
|
||||
* property.
|
||||
*
|
||||
*/
|
||||
public final class VariableDescriptor implements Comparable<VariableDescriptor> {
|
||||
|
||||
/**
|
||||
* The name (or key) of the variable by which all access/lookups to the
|
||||
* value will occur. This is the mechanism of establishing identity and
|
||||
* comparing equality.
|
||||
*/
|
||||
private final String name;
|
||||
|
||||
/**
|
||||
* A brief description of the variable. This description is meant to be
|
||||
* displayed to a user or simply provide a mechanism of documenting intent.
|
||||
*/
|
||||
private final String description;
|
||||
|
||||
/**
|
||||
* indicates that the value for this variable should be considered sensitive
|
||||
* and protected whenever stored or represented
|
||||
*/
|
||||
private final boolean sensitive;
|
||||
|
||||
/**
|
||||
* Convenience constructor to create a descriptor based on name alone. To
|
||||
* include additional parameters use Builder instead.
|
||||
*
|
||||
* @param name name used as unique identifier for this descriptor
|
||||
*/
|
||||
public VariableDescriptor(final String name) {
|
||||
this(new Builder(name));
|
||||
}
|
||||
|
||||
protected VariableDescriptor(final Builder builder) {
|
||||
this.name = builder.name;
|
||||
this.description = builder.description;
|
||||
this.sensitive = builder.sensitive;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(final VariableDescriptor o) {
|
||||
if (o == null) {
|
||||
return -1;
|
||||
}
|
||||
return getName().compareTo(o.getName());
|
||||
}
|
||||
|
||||
public static final class Builder {
|
||||
|
||||
private String name = null;
|
||||
private String description = "";
|
||||
private boolean sensitive = false;
|
||||
|
||||
/**
|
||||
* Establishes the unique identifier or key name of the variable.
|
||||
*
|
||||
* @param name of the property
|
||||
*/
|
||||
public Builder(final String name) {
|
||||
if (null == name || name.trim().isEmpty()) {
|
||||
throw new IllegalArgumentException("Name must not be null or empty");
|
||||
}
|
||||
this.name = name.trim();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param description of the variable
|
||||
* @return the builder
|
||||
*/
|
||||
public Builder description(final String description) {
|
||||
if (null != description) {
|
||||
this.description = description;
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param sensitive true if sensitive; false otherwise
|
||||
* @return the builder
|
||||
*/
|
||||
public Builder sensitive(final boolean sensitive) {
|
||||
this.sensitive = sensitive;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return a VariableDescriptor as configured
|
||||
*
|
||||
*/
|
||||
public VariableDescriptor build() {
|
||||
return new VariableDescriptor(this);
|
||||
}
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
public String getDescription() {
|
||||
return description;
|
||||
}
|
||||
|
||||
public boolean isSensitive() {
|
||||
return sensitive;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(final Object other) {
|
||||
if (other == null) {
|
||||
return false;
|
||||
}
|
||||
if (!(other instanceof VariableDescriptor)) {
|
||||
return false;
|
||||
}
|
||||
if (this == other) {
|
||||
return true;
|
||||
}
|
||||
|
||||
final VariableDescriptor desc = (VariableDescriptor) other;
|
||||
return this.name.equals(desc.name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 797 + this.name.hashCode() * 97;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getClass().getSimpleName() + "[" + name + "]";
|
||||
}
|
||||
|
||||
}
|
|
@ -16,45 +16,115 @@
|
|||
*/
|
||||
package org.apache.nifi.registry;
|
||||
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Access key/value pairs throughout the application.
|
||||
* Provides a registry of variables available for use by various components and
|
||||
* extension points. This enables components to reference variable names rather
|
||||
* than explicit values which can make configurations of those components more
|
||||
* portable.
|
||||
*/
|
||||
public interface VariableRegistry {
|
||||
|
||||
/**
|
||||
* Returns a map of key/value pairs stored in the registry
|
||||
* @return variables
|
||||
**/
|
||||
Map<String, String> getVariables();
|
||||
* Returns an empty registry which can be used as a more intentional null
|
||||
* value.
|
||||
*/
|
||||
public static final VariableRegistry EMPTY_REGISTRY = () -> Collections.EMPTY_MAP;
|
||||
|
||||
/**
|
||||
* Return a value for a given variable
|
||||
* @param variable variable
|
||||
* @return value
|
||||
**/
|
||||
String getVariableValue(String variable);
|
||||
* Provides a registry containing all environment variables and system
|
||||
* properties. System properties receive precedence.
|
||||
*/
|
||||
public static final VariableRegistry ENVIRONMENT_SYSTEM_REGISTRY = new VariableRegistry() {
|
||||
final Map<VariableDescriptor, String> map = new HashMap<>();
|
||||
|
||||
{
|
||||
System.getenv().entrySet().stream().forEach((entry) -> {
|
||||
final VariableDescriptor desc = new VariableDescriptor.Builder(entry.getKey())
|
||||
.description("Env Var")
|
||||
.sensitive(false)
|
||||
.build();
|
||||
map.put(desc, entry.getValue());
|
||||
});
|
||||
System.getProperties().entrySet().stream().forEach((entry) -> {
|
||||
final VariableDescriptor desc = new VariableDescriptor.Builder(entry.getKey().toString())
|
||||
.description("System Property")
|
||||
.sensitive(false)
|
||||
.build();
|
||||
map.put(desc, entry.getValue().toString());
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<VariableDescriptor, String> getVariableMap() {
|
||||
return Collections.unmodifiableMap(map);
|
||||
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
/**
|
||||
* Concatenate a variable registry
|
||||
* @param variableRegistry variableRegistry
|
||||
* */
|
||||
void addRegistry(VariableRegistry variableRegistry);
|
||||
* Provides access to a map of variable key/value pairs. For variables
|
||||
* considered to be sensitive care must be taken to ensure their values are
|
||||
* protected whenever stored or exposed.
|
||||
*
|
||||
* @return An immutable map of all variables in the registry
|
||||
*/
|
||||
|
||||
Map<VariableDescriptor, String> getVariableMap();
|
||||
|
||||
/**
|
||||
* Returns a set variable names in the registry
|
||||
* @return variableNames
|
||||
**/
|
||||
Set<String> getVariableNames();
|
||||
* Returns the VariableDescriptor for the given key name if it exists.
|
||||
*
|
||||
* @param name the string name of the VariableDescriptor to lookup.
|
||||
* @return the variable descriptor registered for this name if it exists;
|
||||
* null otherwise
|
||||
*/
|
||||
default VariableDescriptor getVariableKey(final String name) {
|
||||
if (name == null) {
|
||||
return null;
|
||||
}
|
||||
final VariableDescriptor spec = new VariableDescriptor(name);
|
||||
for (final Map.Entry<VariableDescriptor, String> entry : getVariableMap().entrySet()) {
|
||||
if (entry.getKey().equals(spec)) {
|
||||
return entry.getKey();
|
||||
}
|
||||
}
|
||||
return null;
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Concatenate variable key value pair to registry
|
||||
* @param variables variable Map
|
||||
* */
|
||||
void addVariables(Map<String,String> variables);
|
||||
* Gets the variable value
|
||||
*
|
||||
* @param name the string name of the VariableDescriptor that is the key of
|
||||
* the value to lookup.
|
||||
* @return the value associated with the given variable name if found; null
|
||||
* otherwise
|
||||
*/
|
||||
default String getVariableValue(final String name) {
|
||||
if (name == null) {
|
||||
return null;
|
||||
}
|
||||
return getVariableMap().get(new VariableDescriptor(name));
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the variable value
|
||||
*
|
||||
* @param descriptor the descriptor for which to lookup the variable value.
|
||||
* @return the variable value if the given descriptor is equivalent to one
|
||||
* of the entries in the registry; null otherwise
|
||||
*/
|
||||
default String getVariableValue(final VariableDescriptor descriptor) {
|
||||
if (descriptor == null) {
|
||||
return null;
|
||||
}
|
||||
return getVariableMap().get(descriptor);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1,48 +0,0 @@
|
|||
/*
|
||||
* 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.registry;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
||||
public class VariableRegistryFactory {
|
||||
|
||||
public static VariableRegistry getPropertiesInstance(final Properties...properties){
|
||||
return new PropertiesVariableRegistry(properties);
|
||||
}
|
||||
|
||||
public static VariableRegistry getPropertiesInstance(final Path... paths) throws IOException{
|
||||
return new PropertiesVariableRegistry(paths);
|
||||
}
|
||||
|
||||
public static VariableRegistry getPropertiesInstance(final File ...files) throws IOException{
|
||||
return new PropertiesVariableRegistry(files);
|
||||
}
|
||||
|
||||
@SafeVarargs
|
||||
public static VariableRegistry getInstance(final Map<String,String> ...maps){
|
||||
return new MultiMapVariableRegistry(maps);
|
||||
}
|
||||
|
||||
public static VariableRegistry getInstance(){
|
||||
return new MultiMapVariableRegistry();
|
||||
}
|
||||
|
||||
}
|
|
@ -1,23 +0,0 @@
|
|||
/*
|
||||
* 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.registry;
|
||||
|
||||
public interface VariableRegistryProvider {
|
||||
|
||||
VariableRegistry getVariableRegistry();
|
||||
|
||||
}
|
|
@ -1,87 +0,0 @@
|
|||
/*
|
||||
* 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.registry;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Path;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public class VariableRegistryUtils {
|
||||
|
||||
private final static Logger LOG = LoggerFactory.getLogger(VariableRegistryUtils.class);
|
||||
|
||||
public static VariableRegistry createSystemVariableRegistry(){
|
||||
VariableRegistry variableRegistry = VariableRegistryFactory.getInstance();
|
||||
VariableRegistry propRegistry = VariableRegistryFactory.getPropertiesInstance(System.getProperties());
|
||||
VariableRegistry envRegistry = VariableRegistryFactory.getInstance(System.getenv());
|
||||
variableRegistry.addRegistry(propRegistry);
|
||||
variableRegistry.addRegistry(envRegistry);
|
||||
return variableRegistry;
|
||||
}
|
||||
|
||||
public static VariableRegistry createCustomVariableRegistry(Path[] properties){
|
||||
|
||||
VariableRegistry customRegistry = null;
|
||||
try {
|
||||
customRegistry = VariableRegistryFactory.getPropertiesInstance(properties);
|
||||
customRegistry.addRegistry(createSystemVariableRegistry());
|
||||
} catch (IOException ioe){
|
||||
LOG.error("Exception thrown while attempting to add properties to registry",ioe);
|
||||
}
|
||||
return customRegistry;
|
||||
}
|
||||
|
||||
public static VariableRegistry createFlowVariableRegistry(VariableRegistry variableRegistry, final FlowFile flowFile, final Map<String, String> additionalAttributes){
|
||||
final Map<String, String> flowFileAttributes = flowFile == null ? null : flowFile.getAttributes();
|
||||
final Map<String, String> additionalMap = additionalAttributes == null ? null : additionalAttributes;
|
||||
|
||||
Map<String, String> flowFileProps = null;
|
||||
if (flowFile != null) {
|
||||
flowFileProps = new HashMap<>();
|
||||
flowFileProps.put("flowFileId", String.valueOf(flowFile.getId()));
|
||||
flowFileProps.put("fileSize", String.valueOf(flowFile.getSize()));
|
||||
flowFileProps.put("entryDate", String.valueOf(flowFile.getEntryDate()));
|
||||
flowFileProps.put("lineageStartDate", String.valueOf(flowFile.getLineageStartDate()));
|
||||
flowFileProps.put("lastQueueDate",String.valueOf(flowFile.getLastQueueDate()));
|
||||
flowFileProps.put("queueDateIndex",String.valueOf(flowFile.getQueueDateIndex()));
|
||||
}
|
||||
|
||||
VariableRegistry newRegistry = VariableRegistryFactory.getInstance();
|
||||
|
||||
if(flowFileAttributes != null) {
|
||||
newRegistry.addVariables(flowFileAttributes);
|
||||
}
|
||||
if(additionalMap != null) {
|
||||
newRegistry.addVariables(additionalMap);
|
||||
}
|
||||
if(flowFileProps != null) {
|
||||
newRegistry.addVariables(flowFileProps);
|
||||
}
|
||||
|
||||
if(variableRegistry != null) {
|
||||
newRegistry.addRegistry(variableRegistry);
|
||||
}
|
||||
|
||||
return newRegistry;
|
||||
}
|
||||
|
||||
}
|
|
@ -14,157 +14,39 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.registry;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNull;
|
||||
|
||||
public class TestVariableRegistry {
|
||||
|
||||
@Test
|
||||
public void testReadMap(){
|
||||
Map<String,String> variables1 = new HashMap<>();
|
||||
variables1.put("fake.property.1","fake test value");
|
||||
public void testSystemProp() {
|
||||
assertNull(System.getProperty("ALKJAFLKJDFLSKJSDFLKJSDF"));
|
||||
final VariableRegistry sysEvnReg = VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY;
|
||||
System.setProperty("ALKJAFLKJDFLSKJSDFLKJSDF", "here now");
|
||||
//should not be in Variable Registry
|
||||
assertNull(sysEvnReg.getVariableValue("ALKJAFLKJDFLSKJSDFLKJSDF"));
|
||||
//should be in System properties now though...
|
||||
assertEquals("here now", System.getProperty("ALKJAFLKJDFLSKJSDFLKJSDF"));
|
||||
|
||||
Map<String,String> variables2 = new HashMap<>();
|
||||
variables1.put("fake.property.2","fake test value");
|
||||
|
||||
VariableRegistry registry = VariableRegistryFactory.getInstance(variables1,variables2);
|
||||
|
||||
Map<String,String> variables = registry.getVariables();
|
||||
assertTrue(variables.size() == 2);
|
||||
assertTrue(variables.get("fake.property.1").equals("fake test value"));
|
||||
assertTrue(registry.getVariableValue("fake.property.2").equals("fake test value"));
|
||||
//Test should be stable but a security manager could block it. The following assertions are optional and based on access to the following property.
|
||||
//It was chosen from this list https://docs.oracle.com/javase/tutorial/essential/environment/sysprop.html
|
||||
final String vendorUrl = System.getProperty("java.vendor.url");
|
||||
if (vendorUrl != null) { // we can run this extra test
|
||||
//var reg value matches system property
|
||||
assertEquals(vendorUrl, sysEvnReg.getVariableValue("java.vendor.url"));
|
||||
//change system property
|
||||
System.setProperty("java.vendor.url", "http://fake.vendor.url/");
|
||||
//changed in system properties
|
||||
assertEquals("http://fake.vendor.url/", System.getProperty("java.vendor.url"));
|
||||
//var reg value matches system property still
|
||||
assertEquals(vendorUrl, sysEvnReg.getVariableValue("java.vendor.url"));
|
||||
//restore to its old value
|
||||
System.setProperty("java.vendor.url", vendorUrl);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadProperties(){
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty("fake.property.1","fake test value");
|
||||
VariableRegistry registry = VariableRegistryFactory.getPropertiesInstance(properties);
|
||||
Map<String,String> variables = registry.getVariables();
|
||||
assertTrue(variables.get("fake.property.1").equals("fake test value"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadFiles() throws IOException{
|
||||
final Path fooPath = Paths.get("src/test/resources/TestVariableRegistry/foobar.properties");
|
||||
final Path testPath = Paths.get("src/test/resources/TestVariableRegistry/test.properties");
|
||||
VariableRegistry registry = VariableRegistryFactory.getPropertiesInstance(fooPath.toFile(),testPath.toFile());
|
||||
Map<String,String> variables = registry.getVariables();
|
||||
assertTrue(variables.size() == 3);
|
||||
assertTrue(variables.get("fake.property.1").equals("test me out 1"));
|
||||
assertTrue(variables.get("fake.property.3").equals("test me out 3, test me out 4"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExcludeInvalidFiles() throws IOException{
|
||||
final Path fooPath = Paths.get("src/test/resources/TestVariableRegistry/foobar.properties");
|
||||
final Path testPath = Paths.get("src/test/resources/TestVariableRegistry/test.properties");
|
||||
final Path fakePath = Paths.get("src/test/resources/TestVariableRegistry/fake.properties");
|
||||
VariableRegistry registry = VariableRegistryFactory.getPropertiesInstance(fooPath.toFile(),testPath.toFile(),fakePath.toFile());
|
||||
Map<String,String> variables = registry.getVariables();
|
||||
assertTrue(variables.size() == 3);
|
||||
assertTrue(variables.get("fake.property.1").equals("test me out 1"));
|
||||
assertTrue(variables.get("fake.property.3").equals("test me out 3, test me out 4"));
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testReadPaths() throws IOException{
|
||||
final Path fooPath = Paths.get("src/test/resources/TestVariableRegistry/foobar.properties");
|
||||
final Path testPath = Paths.get("src/test/resources/TestVariableRegistry/test.properties");
|
||||
VariableRegistry registry = VariableRegistryFactory.getPropertiesInstance(fooPath,testPath);
|
||||
Map<String,String> variables = registry.getVariables();
|
||||
assertTrue(variables.size() == 3);
|
||||
assertTrue(variables.get("fake.property.1").equals("test me out 1"));
|
||||
assertTrue(variables.get("fake.property.3").equals("test me out 3, test me out 4"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExcludeInvalidPaths() throws IOException{
|
||||
final Path fooPath = Paths.get("src/test/resources/TestVariableRegistry/foobar.properties");
|
||||
final Path testPath = Paths.get("src/test/resources/TestVariableRegistry/test.properties");
|
||||
final Path fakePath = Paths.get("src/test/resources/TestVariableRegistry/fake.properties");
|
||||
VariableRegistry registry = VariableRegistryFactory.getPropertiesInstance(fooPath,testPath,fakePath);
|
||||
Map<String,String> variables = registry.getVariables();
|
||||
assertTrue(variables.size() == 3);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAddRegistry() throws IOException{
|
||||
|
||||
final Path fooPath = Paths.get("src/test/resources/TestVariableRegistry/foobar.properties");
|
||||
VariableRegistry pathRegistry = VariableRegistryFactory.getPropertiesInstance(fooPath);
|
||||
|
||||
final Path testPath = Paths.get("src/test/resources/TestVariableRegistry/test.properties");
|
||||
VariableRegistry fileRegistry = VariableRegistryFactory.getPropertiesInstance(testPath.toFile());
|
||||
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty("fake.property.5","test me out 5");
|
||||
VariableRegistry propRegistry = VariableRegistryFactory.getPropertiesInstance(properties);
|
||||
|
||||
propRegistry.addRegistry(pathRegistry);
|
||||
propRegistry.addRegistry(fileRegistry);
|
||||
|
||||
Map<String,String> variables = propRegistry.getVariables();
|
||||
assertTrue(variables.size() == 4);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAttemptToAddNullRegistry() throws IOException{
|
||||
|
||||
final Path fooPath = Paths.get("src/test/resources/TestVariableRegistry/foobar.properties");
|
||||
VariableRegistry pathRegistry = VariableRegistryFactory.getPropertiesInstance(fooPath);
|
||||
VariableRegistry nullRegistry = null;
|
||||
pathRegistry.addRegistry(nullRegistry);
|
||||
assertTrue(pathRegistry.getVariables().size() == 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNoOverwriteRegistry()throws IOException{
|
||||
final Path fooPath = Paths.get("src/test/resources/TestVariableRegistry/foobar.properties");
|
||||
VariableRegistry pathRegistry = VariableRegistryFactory.getPropertiesInstance(fooPath);
|
||||
|
||||
final Path testPath = Paths.get("src/test/resources/TestVariableRegistry/test.properties");
|
||||
VariableRegistry fileRegistry = VariableRegistryFactory.getPropertiesInstance(testPath.toFile());
|
||||
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty("fake.property.3","test me out 5");
|
||||
VariableRegistry propRegistry = VariableRegistryFactory.getPropertiesInstance(properties);
|
||||
|
||||
propRegistry.addRegistry(pathRegistry);
|
||||
propRegistry.addRegistry(fileRegistry);
|
||||
|
||||
Map<String,String> variables = propRegistry.getVariables();
|
||||
String testDuplicate = propRegistry.getVariableValue("fake.property.3");
|
||||
assertTrue(variables.size() == 3);
|
||||
assertTrue(testDuplicate.equals("test me out 5"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetVariableNames() throws IOException{
|
||||
final Path fooPath = Paths.get("src/test/resources/TestVariableRegistry/foobar.properties");
|
||||
final Path testPath = Paths.get("src/test/resources/TestVariableRegistry/test.properties");
|
||||
VariableRegistry registry = VariableRegistryFactory.getPropertiesInstance(fooPath,testPath);
|
||||
Set<String> variableNames= registry.getVariableNames();
|
||||
assertTrue(variableNames.size() == 3);
|
||||
assertTrue(variableNames.contains("fake.property.1"));
|
||||
assertTrue(variableNames.contains("fake.property.2"));
|
||||
assertTrue(variableNames.contains("fake.property.3"));
|
||||
assertTrue(!variableNames.contains("fake.property.4"));
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -1,143 +0,0 @@
|
|||
/*
|
||||
* 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.registry;
|
||||
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
public class TestVariableRegistryUtils {
|
||||
|
||||
@Test
|
||||
public void testCreateSystemVariableRegistry(){
|
||||
System.setProperty("fake","test");
|
||||
VariableRegistry variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
Map<String,String> variables = variableRegistry.getVariables();
|
||||
assertTrue(variables.containsKey("PATH"));
|
||||
assertTrue(variables.get("fake").equals("test"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateCustomVariableRegistry(){
|
||||
final Path fooPath = Paths.get("src/test/resources/TestVariableRegistry/foobar.properties");
|
||||
final Path testPath = Paths.get("src/test/resources/TestVariableRegistry/test.properties");
|
||||
Path[] paths = {fooPath,testPath};
|
||||
System.setProperty("fake","test");
|
||||
VariableRegistry variableRegistry = VariableRegistryUtils.createCustomVariableRegistry(paths);
|
||||
Map<String,String> variables = variableRegistry.getVariables();
|
||||
assertTrue(variables.containsKey("PATH"));
|
||||
assertTrue(variables.containsKey("fake.property.3"));
|
||||
assertTrue(variables.get("fake").equals("test"));
|
||||
assertTrue(variables.get("fake.property.3").equals("test me out 3, test me out 4"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateFlowVariableRegistry(){
|
||||
System.setProperty("fake","test");
|
||||
FlowFile flowFile = createFlowFile();
|
||||
|
||||
VariableRegistry variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
VariableRegistry populatedRegistry = VariableRegistryUtils.createFlowVariableRegistry(variableRegistry,flowFile,null);
|
||||
Map<String,String> variables = populatedRegistry.getVariables();
|
||||
assertTrue(variables.containsKey("PATH"));
|
||||
assertTrue(variables.get("fake").equals("test"));
|
||||
assertTrue(variables.get("flowFileId").equals("1"));
|
||||
assertTrue(variables.get("fileSize").equals("50"));
|
||||
assertTrue(variables.get("entryDate").equals("1000"));
|
||||
assertTrue(variables.get("lineageStartDate").equals("10000"));
|
||||
assertTrue(variables.get("filename").equals("fakefile.txt"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPopulateRegistryWithEmptyFlowFileAndAttributes(){
|
||||
System.setProperty("fake","test");
|
||||
VariableRegistry variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
VariableRegistry populatedRegistry = VariableRegistryUtils.createFlowVariableRegistry(variableRegistry,null,null);
|
||||
Map<String,String> variables = populatedRegistry.getVariables();
|
||||
assertTrue( variables.containsKey("PATH"));
|
||||
assertTrue( variables.get("fake").equals("test"));
|
||||
}
|
||||
|
||||
|
||||
private FlowFile createFlowFile(){
|
||||
return new FlowFile() {
|
||||
@Override
|
||||
public long getId() {
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getEntryDate() {
|
||||
return 1000;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLineageStartDate() {
|
||||
return 10000;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Long getLastQueueDate() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isPenalized() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getAttribute(String key) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSize() {
|
||||
return 50;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLineageStartIndex() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getQueueDateIndex() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, String> getAttributes() {
|
||||
Map<String,String> attributes = new HashMap<>();
|
||||
attributes.put("filename","fakefile.txt");
|
||||
return attributes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(FlowFile o) {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
|
@ -1,50 +1,45 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!-- 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. -->
|
||||
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. -->
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<parent>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi</artifactId>
|
||||
<version>1.0.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
<artifactId>nifi-bootstrap</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<parent>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi</artifactId>
|
||||
<version>1.0.0-SNAPSHOT</version>
|
||||
</parent>
|
||||
<artifactId>nifi-bootstrap</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>slf4j-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-api</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-processor-utils</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>javax.mail</groupId>
|
||||
<artifactId>mail</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-expression-language</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.spockframework</groupId>
|
||||
<artifactId>spock-core</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>slf4j-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-api</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-processor-utils</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>javax.mail</groupId>
|
||||
<artifactId>mail</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-expression-language</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
|
|
|
@ -47,7 +47,6 @@ import org.apache.nifi.components.PropertyValue;
|
|||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.w3c.dom.Document;
|
||||
|
@ -68,10 +67,10 @@ public class NotificationServiceManager {
|
|||
|
||||
|
||||
public NotificationServiceManager() {
|
||||
this(VariableRegistryUtils.createSystemVariableRegistry());
|
||||
this(VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY);
|
||||
}
|
||||
|
||||
NotificationServiceManager(VariableRegistry variableRegistry){
|
||||
NotificationServiceManager(final VariableRegistry variableRegistry){
|
||||
this.variableRegistry = variableRegistry;
|
||||
notificationExecutor = Executors.newScheduledThreadPool(1, new ThreadFactory() {
|
||||
@Override
|
||||
|
|
|
@ -17,9 +17,9 @@
|
|||
package org.apache.nifi.attribute.expression.language;
|
||||
|
||||
|
||||
import java.util.Map;
|
||||
import org.apache.nifi.expression.AttributeValueDecorator;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
|
||||
public class EmptyPreparedQuery implements PreparedQuery {
|
||||
|
||||
|
@ -30,7 +30,7 @@ public class EmptyPreparedQuery implements PreparedQuery {
|
|||
}
|
||||
|
||||
@Override
|
||||
public String evaluateExpressions(VariableRegistry variableRegistry, AttributeValueDecorator decorator) throws ProcessException {
|
||||
public String evaluateExpressions(Map<String, String> valueLookup, AttributeValueDecorator decorator) throws ProcessException {
|
||||
return value;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,10 +17,10 @@
|
|||
package org.apache.nifi.attribute.expression.language;
|
||||
|
||||
|
||||
import java.util.Map;
|
||||
import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
|
||||
import org.apache.nifi.expression.AttributeValueDecorator;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
|
||||
/**
|
||||
* An implementation of PreparedQuery that throws an
|
||||
|
@ -39,7 +39,7 @@ public class InvalidPreparedQuery implements PreparedQuery {
|
|||
}
|
||||
|
||||
@Override
|
||||
public String evaluateExpressions(final VariableRegistry variableRegistry, final AttributeValueDecorator decorator) throws ProcessException {
|
||||
public String evaluateExpressions(final Map<String,String> valueLookup, final AttributeValueDecorator decorator) throws ProcessException {
|
||||
throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation);
|
||||
}
|
||||
|
||||
|
|
|
@ -17,12 +17,12 @@
|
|||
package org.apache.nifi.attribute.expression.language;
|
||||
|
||||
|
||||
import java.util.Map;
|
||||
import org.apache.nifi.expression.AttributeValueDecorator;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
|
||||
public interface PreparedQuery {
|
||||
|
||||
String evaluateExpressions(VariableRegistry registry, AttributeValueDecorator decorator) throws ProcessException;
|
||||
String evaluateExpressions(Map<String, String> valueLookup, AttributeValueDecorator decorator) throws ProcessException;
|
||||
|
||||
}
|
||||
|
|
|
@ -194,7 +194,6 @@ import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpre
|
|||
import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UUID;
|
||||
|
||||
import org.apache.nifi.attribute.expression.language.evaluation.selection.MappingEvaluator;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
|
||||
/**
|
||||
* Class used for creating and evaluating NiFi Expression Language. Once a Query
|
||||
|
@ -364,8 +363,8 @@ public class Query {
|
|||
return -1;
|
||||
}
|
||||
|
||||
static String evaluateExpression(final Tree tree, final String queryText, final VariableRegistry registry, final AttributeValueDecorator decorator) throws ProcessException {
|
||||
final Object evaluated = Query.fromTree(tree, queryText).evaluate(registry).getValue();
|
||||
static String evaluateExpression(final Tree tree, final String queryText, final Map<String, String> valueMap, final AttributeValueDecorator decorator) throws ProcessException {
|
||||
final Object evaluated = Query.fromTree(tree, queryText).evaluate(valueMap).getValue();
|
||||
if (evaluated == null) {
|
||||
return null;
|
||||
}
|
||||
|
@ -375,12 +374,12 @@ public class Query {
|
|||
return decorator == null ? escaped : decorator.decorate(escaped);
|
||||
}
|
||||
|
||||
static String evaluateExpressions(final String rawValue, VariableRegistry registry) throws ProcessException {
|
||||
return evaluateExpressions(rawValue, registry, null);
|
||||
static String evaluateExpressions(final String rawValue, final Map<String, String> valueLookup) throws ProcessException {
|
||||
return evaluateExpressions(rawValue, valueLookup, null);
|
||||
}
|
||||
|
||||
static String evaluateExpressions(final String rawValue, VariableRegistry registry, final AttributeValueDecorator decorator) throws ProcessException {
|
||||
return Query.prepare(rawValue).evaluateExpressions(registry, decorator);
|
||||
static String evaluateExpressions(final String rawValue, final Map<String, String> valueLookup, final AttributeValueDecorator decorator) throws ProcessException {
|
||||
return Query.prepare(rawValue).evaluateExpressions(valueLookup, decorator);
|
||||
}
|
||||
|
||||
private static Evaluator<?> getRootSubjectEvaluator(final Evaluator<?> evaluator) {
|
||||
|
@ -542,12 +541,12 @@ public class Query {
|
|||
return evaluator.getResultType();
|
||||
}
|
||||
|
||||
QueryResult<?> evaluate(final VariableRegistry registry) {
|
||||
QueryResult<?> evaluate(final Map<String, String> map) {
|
||||
if (evaluated.getAndSet(true)) {
|
||||
throw new IllegalStateException("A Query cannot be evaluated more than once");
|
||||
}
|
||||
|
||||
return evaluator.evaluate(registry.getVariables());
|
||||
return evaluator.evaluate(map);
|
||||
}
|
||||
|
||||
Tree getTree() {
|
||||
|
|
|
@ -21,13 +21,17 @@ import org.apache.nifi.expression.AttributeValueDecorator;
|
|||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
|
||||
public class StandardAttributeExpression implements AttributeExpression {
|
||||
|
||||
private final Query query;
|
||||
private final VariableRegistry variableRegistry;
|
||||
|
||||
public StandardAttributeExpression(final Query query) {
|
||||
this.query = query;
|
||||
this.variableRegistry = null;
|
||||
}
|
||||
|
||||
public StandardAttributeExpression(final Query query, final VariableRegistry variableRegistry) {
|
||||
this.query = query;
|
||||
this.variableRegistry = variableRegistry;
|
||||
|
@ -40,7 +44,7 @@ public class StandardAttributeExpression implements AttributeExpression {
|
|||
|
||||
@Override
|
||||
public String evaluate() throws ProcessException {
|
||||
return evaluate((AttributeValueDecorator) null);
|
||||
return evaluate(null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -55,8 +59,8 @@ public class StandardAttributeExpression implements AttributeExpression {
|
|||
|
||||
@Override
|
||||
public String evaluate(final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException {
|
||||
VariableRegistry flowFileRegistry = VariableRegistryUtils.createFlowVariableRegistry(variableRegistry,flowFile,null);
|
||||
final Object evaluationResult = query.evaluate(flowFileRegistry).getValue();
|
||||
final ValueLookup lookup = new ValueLookup(variableRegistry, flowFile);
|
||||
final Object evaluationResult = query.evaluate(lookup).getValue();
|
||||
if (evaluationResult == null) {
|
||||
return "";
|
||||
}
|
||||
|
|
|
@ -26,6 +26,10 @@ public class StandardExpressionLanguageCompiler implements ExpressionLanguageCom
|
|||
|
||||
private final VariableRegistry variableRegistry;
|
||||
|
||||
public StandardExpressionLanguageCompiler() {
|
||||
this.variableRegistry = VariableRegistry.EMPTY_REGISTRY;
|
||||
}
|
||||
|
||||
public StandardExpressionLanguageCompiler(final VariableRegistry variableRegistry) {
|
||||
this.variableRegistry = variableRegistry;
|
||||
}
|
||||
|
|
|
@ -25,7 +25,6 @@ import org.apache.nifi.expression.AttributeValueDecorator;
|
|||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
|
||||
import org.antlr.runtime.tree.Tree;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
|
||||
public class StandardPreparedQuery implements PreparedQuery {
|
||||
|
||||
|
@ -39,14 +38,14 @@ public class StandardPreparedQuery implements PreparedQuery {
|
|||
|
||||
|
||||
@Override
|
||||
public String evaluateExpressions(final VariableRegistry registry, final AttributeValueDecorator decorator) throws ProcessException {
|
||||
public String evaluateExpressions(final Map<String, String> valueMap, final AttributeValueDecorator decorator) throws ProcessException {
|
||||
final StringBuilder sb = new StringBuilder();
|
||||
for (final String val : queryStrings) {
|
||||
final Tree tree = trees.get(val);
|
||||
if (tree == null) {
|
||||
sb.append(val);
|
||||
} else {
|
||||
final String evaluated = Query.evaluateExpression(tree, val, registry, decorator);
|
||||
final String evaluated = Query.evaluateExpression(tree, val, valueMap, decorator);
|
||||
if (evaluated != null) {
|
||||
sb.append(evaluated);
|
||||
}
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.apache.nifi.flowfile.FlowFile;
|
|||
import org.apache.nifi.processor.DataUnit;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.apache.nifi.util.FormatUtils;
|
||||
|
||||
public class StandardPropertyValue implements PropertyValue {
|
||||
|
@ -37,21 +36,32 @@ public class StandardPropertyValue implements PropertyValue {
|
|||
private final PreparedQuery preparedQuery;
|
||||
private final VariableRegistry variableRegistry;
|
||||
|
||||
public StandardPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup) {
|
||||
this(rawValue, serviceLookup, Query.prepare(rawValue), VariableRegistry.EMPTY_REGISTRY);
|
||||
}
|
||||
|
||||
public StandardPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup, final VariableRegistry variableRegistry) {
|
||||
this(rawValue, serviceLookup, Query.prepare(rawValue), variableRegistry);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs a new StandardPropertyValue with the given value & service lookup and indicates whether or not the rawValue contains any NiFi Expressions. If it is unknown whether or not the value
|
||||
* contains any NiFi Expressions, the {@link #StandardPropertyValue(String, ControllerServiceLookup, VariableRegistry)} constructor should be used or <code>true</code> should be passed.
|
||||
* However, if it is known that the value contains no NiFi Expression, that information should be provided so that calls to {@link #evaluateAttributeExpressions()} are much more efficient
|
||||
* Constructs a new StandardPropertyValue with the given value & service
|
||||
* lookup and indicates whether or not the rawValue contains any NiFi
|
||||
* Expressions. If it is unknown whether or not the value contains any NiFi
|
||||
* Expressions, the
|
||||
* {@link #StandardPropertyValue(String, ControllerServiceLookup, VariableRegistry)}
|
||||
* constructor should be used or <code>true</code> should be passed.
|
||||
* However, if it is known that the value contains no NiFi Expression, that
|
||||
* information should be provided so that calls to
|
||||
* {@link #evaluateAttributeExpressions()} are much more efficient
|
||||
*
|
||||
* @param rawValue value
|
||||
* @param serviceLookup lookup
|
||||
* @param preparedQuery query
|
||||
* @param variableRegistry variableRegistry
|
||||
*/
|
||||
public StandardPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup, final PreparedQuery preparedQuery,
|
||||
final VariableRegistry variableRegistry) {
|
||||
final VariableRegistry variableRegistry) {
|
||||
this.rawValue = rawValue;
|
||||
this.serviceLookup = serviceLookup;
|
||||
this.preparedQuery = preparedQuery;
|
||||
|
@ -138,12 +148,10 @@ public class StandardPropertyValue implements PropertyValue {
|
|||
if (rawValue == null || preparedQuery == null) {
|
||||
return this;
|
||||
}
|
||||
VariableRegistry registry = VariableRegistryUtils.createFlowVariableRegistry(variableRegistry,flowFile,additionalAttributes);
|
||||
return new StandardPropertyValue(preparedQuery.evaluateExpressions(registry,decorator), serviceLookup,null,registry);
|
||||
final ValueLookup lookup = new ValueLookup(variableRegistry, flowFile, additionalAttributes);
|
||||
return new StandardPropertyValue(preparedQuery.evaluateExpressions(lookup, decorator), serviceLookup, null, variableRegistry);
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return rawValue;
|
||||
|
|
|
@ -0,0 +1,184 @@
|
|||
/*
|
||||
* 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.attribute.expression.language;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.registry.VariableDescriptor;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
|
||||
/**
|
||||
* A convenience class to encapsulate the logic of variable substitution
|
||||
* based first on any additional variable maps, then flow file properties,
|
||||
* then flow file attributes, and finally the provided variable registry.
|
||||
*/
|
||||
final class ValueLookup implements Map<String, String> {
|
||||
|
||||
final List<Map<String, String>> maps = new ArrayList<>();
|
||||
final VariableRegistry registry;
|
||||
|
||||
/**
|
||||
* Constructs a ValueLookup where values are looked up first based any
|
||||
* provided additional maps, then flowfile properties, then flowfile
|
||||
* attributes, then based on the provided variable registry. The lookup is
|
||||
* immutable and operations which attempt to alter state will throw
|
||||
* UnsupportedOperationException
|
||||
*
|
||||
* @param registry the variable registry to lookup from; may be null
|
||||
* @param flowFile the flowFile to pull attributes from; may be null
|
||||
* @param additionalMaps the maps to pull values from; may be null or empty
|
||||
*/
|
||||
ValueLookup(final VariableRegistry registry, final FlowFile flowFile, final Map<String, String>... additionalMaps) {
|
||||
for (final Map<String, String> map : additionalMaps) {
|
||||
if (map != null && !map.isEmpty()) {
|
||||
maps.add(map);
|
||||
}
|
||||
}
|
||||
if (flowFile != null) {
|
||||
maps.add(ValueLookup.extractFlowFileProperties(flowFile));
|
||||
maps.add(flowFile.getAttributes());
|
||||
}
|
||||
|
||||
this.registry = registry == null ? VariableRegistry.EMPTY_REGISTRY : registry;
|
||||
}
|
||||
|
||||
static final Map<String, String> extractFlowFileProperties(final FlowFile flowFile) {
|
||||
final Map<String, String> flowFileProps = new HashMap<>();
|
||||
flowFileProps.put("flowFileId", String.valueOf(flowFile.getId()));
|
||||
flowFileProps.put("fileSize", String.valueOf(flowFile.getSize()));
|
||||
flowFileProps.put("entryDate", String.valueOf(flowFile.getEntryDate()));
|
||||
flowFileProps.put("lineageStartDate", String.valueOf(flowFile.getLineageStartDate()));
|
||||
flowFileProps.put("lastQueueDate", String.valueOf(flowFile.getLastQueueDate()));
|
||||
flowFileProps.put("queueDateIndex", String.valueOf(flowFile.getQueueDateIndex()));
|
||||
return flowFileProps;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
return keySet().size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEmpty() {
|
||||
for (final Map<String, String> map : maps) {
|
||||
if (!map.isEmpty()) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return registry.getVariableMap().isEmpty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean containsKey(Object key) {
|
||||
if (key == null) {
|
||||
return false;
|
||||
}
|
||||
if (maps.stream().anyMatch((map) -> (map.containsKey(key)))) {
|
||||
return true;
|
||||
}
|
||||
return registry.getVariableKey(key.toString()) != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean containsValue(Object value) {
|
||||
if (value == null) {
|
||||
return false;
|
||||
}
|
||||
//check entrySet then iterate through values (otherwise might find a value that was hidden/overriden
|
||||
final Collection<String> values = values();
|
||||
return values.contains(value.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String get(Object key) {
|
||||
if (key == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
for (final Map<String, String> map : maps) {
|
||||
final String val = map.get(key.toString());
|
||||
if (val != null) {
|
||||
return val;
|
||||
}
|
||||
}
|
||||
return registry.getVariableValue(key.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String put(String key, String value) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String remove(Object key) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void putAll(Map<? extends String, ? extends String> m) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> keySet() {
|
||||
final Set<String> keySet = new HashSet<>();
|
||||
entrySet().stream().forEach((entry) -> {
|
||||
keySet.add(entry.getKey());
|
||||
});
|
||||
return keySet;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> values() {
|
||||
final Set<String> values = new HashSet<>();
|
||||
entrySet().stream().forEach((entry) -> {
|
||||
values.add(entry.getValue());
|
||||
});
|
||||
return values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<Entry<String, String>> entrySet() {
|
||||
final Map<String, String> newMap = new HashMap<>();
|
||||
//put variable registry entries first
|
||||
for (final Map.Entry<VariableDescriptor, String> entry : registry.getVariableMap().entrySet()) {
|
||||
newMap.put(entry.getKey().getName(), entry.getValue());
|
||||
}
|
||||
//put attribute maps in reverse order
|
||||
final List<Map<String,String>> listOfMaps = new ArrayList<>(maps);
|
||||
Collections.reverse(listOfMaps);
|
||||
for(final Map<String,String> map : listOfMaps){
|
||||
for(final Map.Entry<String, String> entry : map.entrySet()){
|
||||
newMap.put(entry.getKey(), entry.getValue());
|
||||
}
|
||||
}
|
||||
return newMap.entrySet();
|
||||
}
|
||||
|
||||
}
|
|
@ -18,8 +18,6 @@ package org.apache.nifi.attribute.expression.language
|
|||
|
||||
import org.apache.nifi.attribute.expression.language.evaluation.QueryResult
|
||||
import org.apache.nifi.expression.AttributeExpression
|
||||
import org.apache.nifi.registry.VariableRegistry
|
||||
import org.apache.nifi.registry.VariableRegistryFactory
|
||||
import org.junit.After
|
||||
import org.junit.Before
|
||||
import org.junit.BeforeClass
|
||||
|
@ -59,7 +57,6 @@ public class QueryGroovyTest extends GroovyTestCase {
|
|||
final Map<String, String> attributes = [
|
||||
single : ORIGINAL_VALUE,
|
||||
repeating: [ORIGINAL_VALUE].multiply(n).join(" ")]
|
||||
final VariableRegistry variableRegistry = VariableRegistryFactory.getInstance(attributes)
|
||||
logger.info("Attributes: ${attributes}")
|
||||
|
||||
final String REPLACEMENT_VALUE = "Goodbye Planet"
|
||||
|
@ -78,10 +75,10 @@ public class QueryGroovyTest extends GroovyTestCase {
|
|||
Query replaceRepeatingQuery = Query.compile(REPLACE_REPEATING_EXPRESSION)
|
||||
|
||||
// Act
|
||||
QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(variableRegistry)
|
||||
QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(attributes)
|
||||
logger.info("Replace single result: ${replaceSingleResult.value}")
|
||||
|
||||
QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(variableRegistry)
|
||||
QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(attributes)
|
||||
logger.info("Replace repeating result: ${replaceRepeatingResult.value}")
|
||||
|
||||
// Assert
|
||||
|
@ -100,7 +97,6 @@ public class QueryGroovyTest extends GroovyTestCase {
|
|||
final Map<String, String> attributes = [
|
||||
single : ORIGINAL_VALUE,
|
||||
repeating: [ORIGINAL_VALUE].multiply(n).join(" ")]
|
||||
final VariableRegistry variableRegistry = VariableRegistryFactory.getInstance(attributes)
|
||||
logger.info("Attributes: ${attributes}")
|
||||
|
||||
final String REPLACEMENT_VALUE = "Goodbye Planet"
|
||||
|
@ -119,10 +115,10 @@ public class QueryGroovyTest extends GroovyTestCase {
|
|||
Query replaceRepeatingQuery = Query.compile(REPLACE_REPEATING_EXPRESSION)
|
||||
|
||||
// Act
|
||||
QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(variableRegistry)
|
||||
QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(attributes)
|
||||
logger.info("Replace single result: ${replaceSingleResult.value}")
|
||||
|
||||
QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(variableRegistry)
|
||||
QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(attributes)
|
||||
logger.info("Replace repeating result: ${replaceRepeatingResult.value}")
|
||||
|
||||
// Assert
|
||||
|
@ -141,7 +137,6 @@ public class QueryGroovyTest extends GroovyTestCase {
|
|||
final Map<String, String> attributes = [
|
||||
single : ORIGINAL_VALUE,
|
||||
repeating: [ORIGINAL_VALUE].multiply(n).join(" ")]
|
||||
final VariableRegistry variableRegistry = VariableRegistryFactory.getInstance(attributes)
|
||||
logger.info("Attributes: ${attributes}")
|
||||
|
||||
final String REPLACEMENT_VALUE = "Goodbye Planet"
|
||||
|
@ -160,10 +155,10 @@ public class QueryGroovyTest extends GroovyTestCase {
|
|||
Query replaceRepeatingQuery = Query.compile(REPLACE_REPEATING_EXPRESSION)
|
||||
|
||||
// Act
|
||||
QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(variableRegistry)
|
||||
QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(attributes)
|
||||
logger.info("Replace single result: ${replaceSingleResult.value}")
|
||||
|
||||
QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(variableRegistry)
|
||||
QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(attributes)
|
||||
logger.info("Replace repeating result: ${replaceRepeatingResult.value}")
|
||||
|
||||
// Assert
|
||||
|
|
|
@ -35,7 +35,6 @@ import java.util.HashSet;
|
|||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
||||
import org.apache.nifi.attribute.expression.language.Query.Range;
|
||||
import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
|
||||
|
@ -46,8 +45,6 @@ import org.apache.nifi.flowfile.FlowFile;
|
|||
import org.antlr.runtime.tree.Tree;
|
||||
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryFactory;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.junit.Assert;
|
||||
|
||||
import org.junit.Ignore;
|
||||
|
@ -57,7 +54,6 @@ import org.mockito.Mockito;
|
|||
|
||||
public class TestQuery {
|
||||
|
||||
|
||||
@Test
|
||||
public void testCompilation() {
|
||||
assertInvalid("${attr:uuid()}");
|
||||
|
@ -106,9 +102,9 @@ public class TestQuery {
|
|||
Query.validateExpression("$${attr}", true);
|
||||
|
||||
Query.validateExpression("${filename:startsWith('T8MTXBC')\n"
|
||||
+ ":or( ${filename:startsWith('C4QXABC')} )\n"
|
||||
+ ":or( ${filename:startsWith('U6CXEBC')} )"
|
||||
+ ":or( ${filename:startsWith('KYM3ABC')} )}", false);
|
||||
+ ":or( ${filename:startsWith('C4QXABC')} )\n"
|
||||
+ ":or( ${filename:startsWith('U6CXEBC')} )"
|
||||
+ ":or( ${filename:startsWith('KYM3ABC')} )}", false);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -121,7 +117,7 @@ public class TestQuery {
|
|||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("x", "x");
|
||||
attributes.put("y", "x");
|
||||
final String result = Query.evaluateExpressions(expression,VariableRegistryFactory.getInstance(attributes), null);
|
||||
final String result = Query.evaluateExpressions(expression, attributes, null);
|
||||
assertEquals("true", result);
|
||||
|
||||
Query.validateExpression(expression, false);
|
||||
|
@ -181,14 +177,13 @@ public class TestQuery {
|
|||
public void testWithTicksOutside() {
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("attr", "My Value");
|
||||
VariableRegistry registry = VariableRegistryFactory.getInstance(attributes);
|
||||
assertEquals(1, Query.extractExpressionRanges("\"${attr}").size());
|
||||
assertEquals(1, Query.extractExpressionRanges("'${attr}").size());
|
||||
assertEquals(1, Query.extractExpressionRanges("'${attr}'").size());
|
||||
assertEquals(1, Query.extractExpressionRanges("${attr}").size());
|
||||
|
||||
assertEquals("'My Value'", Query.evaluateExpressions("'${attr}'", registry, null));
|
||||
assertEquals("'My Value", Query.evaluateExpressions("'${attr}", registry, null));
|
||||
assertEquals("'My Value'", Query.evaluateExpressions("'${attr}'", attributes, null));
|
||||
assertEquals("'My Value", Query.evaluateExpressions("'${attr}", attributes, null));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -198,7 +193,7 @@ public class TestQuery {
|
|||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("dateTime", "2013/11/18 10:22:27.678");
|
||||
|
||||
final QueryResult<?> result = query.evaluate(VariableRegistryFactory.getInstance(attributes));
|
||||
final QueryResult<?> result = query.evaluate(attributes);
|
||||
assertEquals(ResultType.NUMBER, result.getResultType());
|
||||
assertEquals(1384788147678L, result.getValue());
|
||||
}
|
||||
|
@ -227,7 +222,7 @@ public class TestQuery {
|
|||
final Date roundedToNearestSecond = new Date(date.getTime() - millis);
|
||||
final String formatted = sdf.format(roundedToNearestSecond);
|
||||
|
||||
final QueryResult<?> result = query.evaluate(VariableRegistryFactory.getInstance(attributes));
|
||||
final QueryResult<?> result = query.evaluate(attributes);
|
||||
assertEquals(ResultType.STRING, result.getResultType());
|
||||
assertEquals(formatted, result.getValue());
|
||||
}
|
||||
|
@ -237,15 +232,14 @@ public class TestQuery {
|
|||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("x", "abc");
|
||||
attributes.put("a", "abc");
|
||||
VariableRegistry registry = VariableRegistryFactory.getInstance(attributes);
|
||||
|
||||
verifyEquals("${x:equals(${a})}", attributes, true);
|
||||
|
||||
Query.validateExpression("${x:equals('${a}')}", false);
|
||||
assertEquals("true", Query.evaluateExpressions("${x:equals('${a}')}", registry, null));
|
||||
assertEquals("true", Query.evaluateExpressions("${x:equals('${a}')}", attributes, null));
|
||||
|
||||
Query.validateExpression("${x:equals(\"${a}\")}", false);
|
||||
assertEquals("true", Query.evaluateExpressions("${x:equals(\"${a}\")}", registry, null));
|
||||
assertEquals("true", Query.evaluateExpressions("${x:equals(\"${a}\")}", attributes, null));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -278,18 +272,17 @@ public class TestQuery {
|
|||
|
||||
@Test
|
||||
public void testEmbeddedExpressionsAndQuotesWithProperties() {
|
||||
final Properties attributes = new Properties();
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("x", "abc");
|
||||
attributes.put("a", "abc");
|
||||
VariableRegistry registry = VariableRegistryFactory.getPropertiesInstance(attributes);
|
||||
|
||||
verifyEquals("${x:equals(${a})}",registry,true);
|
||||
verifyEquals("${x:equals(${a})}", attributes, true);
|
||||
|
||||
Query.validateExpression("${x:equals('${a}')}", false);
|
||||
assertEquals("true", Query.evaluateExpressions("${x:equals('${a}')}", registry, null));
|
||||
assertEquals("true", Query.evaluateExpressions("${x:equals('${a}')}", attributes, null));
|
||||
|
||||
Query.validateExpression("${x:equals(\"${a}\")}", false);
|
||||
assertEquals("true", Query.evaluateExpressions("${x:equals(\"${a}\")}", registry, null));
|
||||
assertEquals("true", Query.evaluateExpressions("${x:equals(\"${a}\")}", attributes, null));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -367,8 +360,8 @@ public class TestQuery {
|
|||
Mockito.when(mockFlowFile.getSize()).thenReturn(1L);
|
||||
Mockito.when(mockFlowFile.getLineageStartDate()).thenReturn(System.currentTimeMillis());
|
||||
|
||||
final VariableRegistry variableRegistry = VariableRegistryUtils.createFlowVariableRegistry(VariableRegistryUtils.createSystemVariableRegistry(),mockFlowFile,null);
|
||||
return Query.evaluateExpressions(queryString,variableRegistry);
|
||||
final ValueLookup lookup = new ValueLookup(VariableRegistry.EMPTY_REGISTRY, mockFlowFile);
|
||||
return Query.evaluateExpressions(queryString, lookup);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -524,7 +517,7 @@ public class TestQuery {
|
|||
verifyEquals("${x:toNumber():gt( ${y:toNumber():plus( ${z:toNumber()} )} )}", attributes, true);
|
||||
|
||||
attributes.put("y", "88");
|
||||
assertEquals("true", Query.evaluateExpressions("${x:equals( '${y}' )}", VariableRegistryFactory.getInstance(attributes), null));
|
||||
assertEquals("true", Query.evaluateExpressions("${x:equals( '${y}' )}", attributes, null));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -572,7 +565,7 @@ public class TestQuery {
|
|||
final String format = "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'";
|
||||
|
||||
final String query = "startDateTime=\"${date:toNumber():toDate():format(\"" + format + "\")}\"";
|
||||
final String result = Query.evaluateExpressions(query, VariableRegistryFactory.getInstance(attributes), null);
|
||||
final String result = Query.evaluateExpressions(query, attributes, null);
|
||||
|
||||
final String expectedTime = new SimpleDateFormat(format, Locale.US).format(timestamp);
|
||||
assertEquals("startDateTime=\"" + expectedTime + "\"", result);
|
||||
|
@ -641,7 +634,7 @@ public class TestQuery {
|
|||
final String query = "${ abc:equals('abc'):or( \n\t${xx:isNull()}\n) }";
|
||||
assertEquals(ResultType.BOOLEAN, Query.getResultType(query));
|
||||
Query.validateExpression(query, false);
|
||||
assertEquals("true", Query.evaluateExpressions(query,VariableRegistryUtils.createSystemVariableRegistry()));
|
||||
assertEquals("true", Query.evaluateExpressions(query, Collections.EMPTY_MAP));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -657,23 +650,22 @@ public class TestQuery {
|
|||
public void testComments() {
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("abc", "xyz");
|
||||
VariableRegistry registry = VariableRegistryFactory.getInstance(attributes);
|
||||
final String expression
|
||||
= "# hello, world\n"
|
||||
+ "${# ref attr\n"
|
||||
+ "\t"
|
||||
+ "abc"
|
||||
+ "\t"
|
||||
+ "#end ref attr\n"
|
||||
+ "}";
|
||||
= "# hello, world\n"
|
||||
+ "${# ref attr\n"
|
||||
+ "\t"
|
||||
+ "abc"
|
||||
+ "\t"
|
||||
+ "#end ref attr\n"
|
||||
+ "}";
|
||||
|
||||
Query query = Query.compile(expression);
|
||||
QueryResult<?> result = query.evaluate(registry);
|
||||
QueryResult<?> result = query.evaluate(attributes);
|
||||
assertEquals(ResultType.STRING, result.getResultType());
|
||||
assertEquals("xyz", result.getValue());
|
||||
|
||||
query = Query.compile("${abc:append('# hello') #good-bye \n}");
|
||||
result = query.evaluate(registry);
|
||||
result = query.evaluate(attributes);
|
||||
assertEquals(ResultType.STRING, result.getResultType());
|
||||
assertEquals("xyz# hello", result.getValue());
|
||||
}
|
||||
|
@ -802,27 +794,18 @@ public class TestQuery {
|
|||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("entryDate", String.valueOf(now.getTimeInMillis()));
|
||||
|
||||
VariableRegistry registry = VariableRegistryFactory.getInstance(attributes);
|
||||
verifyEquals("${entryDate:toNumber():toDate():format('yyyy')}", attributes, String.valueOf(year));
|
||||
|
||||
attributes.clear();
|
||||
attributes.put("month", "3");
|
||||
attributes.put("day", "4");
|
||||
attributes.put("year", "2013");
|
||||
assertEquals("63", Query.evaluateExpressions("${year:append('/'):append(${month}):append('/'):append(${day}):toDate('yyyy/MM/dd'):format('D')}", registry, null));
|
||||
assertEquals("63", Query.evaluateExpressions("${year:append('/'):append('${month}'):append('/'):append('${day}'):toDate('yyyy/MM/dd'):format('D')}", registry, null));
|
||||
assertEquals("63", Query.evaluateExpressions("${year:append('/'):append(${month}):append('/'):append(${day}):toDate('yyyy/MM/dd'):format('D')}", attributes, null));
|
||||
assertEquals("63", Query.evaluateExpressions("${year:append('/'):append('${month}'):append('/'):append('${day}'):toDate('yyyy/MM/dd'):format('D')}", attributes, null));
|
||||
|
||||
verifyEquals("${year:append('/'):append(${month}):append('/'):append(${day}):toDate('yyyy/MM/dd'):format('D')}", attributes, "63");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSystemProperty() {
|
||||
System.setProperty("hello", "good-bye");
|
||||
VariableRegistry variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
assertEquals("good-bye", Query.evaluateExpressions("${hello}",variableRegistry));
|
||||
assertEquals("good-bye", Query.compile("${hello}").evaluate(variableRegistry).getValue());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAnyAttribute() {
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
|
@ -860,15 +843,14 @@ public class TestQuery {
|
|||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("abc", "a,b,c");
|
||||
attributes.put("xyz", "abc");
|
||||
VariableRegistry registry = VariableRegistryFactory.getInstance(attributes);
|
||||
|
||||
final String query = "${anyDelineatedValue('${abc}', ','):equals('b')}";
|
||||
assertEquals(ResultType.BOOLEAN, Query.getResultType(query));
|
||||
|
||||
assertEquals("true", Query.evaluateExpressions(query, registry, null));
|
||||
assertEquals("true", Query.evaluateExpressions("${anyDelineatedValue('${abc}', ','):equals('a')}", registry, null));
|
||||
assertEquals("true", Query.evaluateExpressions("${anyDelineatedValue('${abc}', ','):equals('c')}", registry, null));
|
||||
assertEquals("false", Query.evaluateExpressions("${anyDelineatedValue('${abc}', ','):equals('d')}", registry, null));
|
||||
assertEquals("true", Query.evaluateExpressions(query, attributes, null));
|
||||
assertEquals("true", Query.evaluateExpressions("${anyDelineatedValue('${abc}', ','):equals('a')}", attributes, null));
|
||||
assertEquals("true", Query.evaluateExpressions("${anyDelineatedValue('${abc}', ','):equals('c')}", attributes, null));
|
||||
assertEquals("false", Query.evaluateExpressions("${anyDelineatedValue('${abc}', ','):equals('d')}", attributes, null));
|
||||
|
||||
verifyEquals("${anyDelineatedValue(${abc}, ','):equals('b')}", attributes, true);
|
||||
verifyEquals("${anyDelineatedValue(${abc}, ','):equals('a')}", attributes, true);
|
||||
|
@ -882,15 +864,13 @@ public class TestQuery {
|
|||
attributes.put("abc", "a,b,c");
|
||||
attributes.put("xyz", "abc");
|
||||
|
||||
VariableRegistry registry = VariableRegistryFactory.getInstance(attributes);
|
||||
|
||||
final String query = "${allDelineatedValues('${abc}', ','):matches('[abc]')}";
|
||||
|
||||
assertEquals(ResultType.BOOLEAN, Query.getResultType(query));
|
||||
assertEquals("true", Query.evaluateExpressions(query, registry, null));
|
||||
assertEquals("true", Query.evaluateExpressions(query, registry, null));
|
||||
assertEquals("false", Query.evaluateExpressions("${allDelineatedValues('${abc}', ','):matches('[abd]')}",registry, null));
|
||||
assertEquals("false", Query.evaluateExpressions("${allDelineatedValues('${abc}', ','):equals('a'):not()}", registry, null));
|
||||
assertEquals("true", Query.evaluateExpressions(query, attributes, null));
|
||||
assertEquals("true", Query.evaluateExpressions(query, attributes, null));
|
||||
assertEquals("false", Query.evaluateExpressions("${allDelineatedValues('${abc}', ','):matches('[abd]')}", attributes, null));
|
||||
assertEquals("false", Query.evaluateExpressions("${allDelineatedValues('${abc}', ','):equals('a'):not()}", attributes, null));
|
||||
|
||||
verifyEquals("${allDelineatedValues(${abc}, ','):matches('[abc]')}", attributes, true);
|
||||
verifyEquals("${allDelineatedValues(${abc}, ','):matches('[abd]')}", attributes, false);
|
||||
|
@ -956,13 +936,12 @@ public class TestQuery {
|
|||
attributes.put("xyz", "4132");
|
||||
attributes.put("hello", "world!");
|
||||
attributes.put("dotted", "abc.xyz");
|
||||
VariableRegistry registry = VariableRegistryFactory.getInstance(attributes);
|
||||
|
||||
final String evaluated = Query.evaluateExpressions("${abc:matches('1234${end}4321')}", registry, null);
|
||||
final String evaluated = Query.evaluateExpressions("${abc:matches('1234${end}4321')}", attributes, null);
|
||||
assertEquals("true", evaluated);
|
||||
|
||||
attributes.put("end", "888");
|
||||
final String secondEvaluation = Query.evaluateExpressions("${abc:matches('1234${end}4321')}", registry, null);
|
||||
final String secondEvaluation = Query.evaluateExpressions("${abc:matches('1234${end}4321')}", attributes, null);
|
||||
assertEquals("false", secondEvaluation);
|
||||
|
||||
verifyEquals("${dotted:matches('abc\\.xyz')}", attributes, true);
|
||||
|
@ -977,13 +956,12 @@ public class TestQuery {
|
|||
attributes.put("hello", "world!");
|
||||
attributes.put("dotted", "abc.xyz");
|
||||
|
||||
final String evaluated = Query.evaluateExpressions("${abc:find('1234${end}4321')}", VariableRegistryFactory.getInstance(attributes), null);
|
||||
final String evaluated = Query.evaluateExpressions("${abc:find('1234${end}4321')}", attributes, null);
|
||||
assertEquals("true", evaluated);
|
||||
|
||||
attributes.put("end", "888");
|
||||
|
||||
|
||||
final String secondEvaluation = Query.evaluateExpressions("${abc:find('${end}4321')}",VariableRegistryFactory.getInstance(attributes), null);
|
||||
final String secondEvaluation = Query.evaluateExpressions("${abc:find('${end}4321')}", attributes, null);
|
||||
assertEquals("false", secondEvaluation);
|
||||
|
||||
verifyEquals("${dotted:find('\\.')}", attributes, true);
|
||||
|
@ -1105,16 +1083,22 @@ public class TestQuery {
|
|||
attributes.put("filename 3", "abcxy");
|
||||
|
||||
final String query
|
||||
= "${"
|
||||
+ " 'non-existing':notNull():not():and(" + // true AND (
|
||||
" ${filename1:startsWith('y')" + // false
|
||||
" :or(" + // or
|
||||
" ${ filename1:startsWith('x'):and(false) }" + // false
|
||||
" ):or(" + // or
|
||||
" ${ filename2:endsWith('xxxx'):or( ${'filename 3':length():gt(1)} ) }" + // true )
|
||||
" )}"
|
||||
+ " )"
|
||||
+ "}";
|
||||
= "${"
|
||||
+ " 'non-existing':notNull():not():and("
|
||||
+ // true AND (
|
||||
" ${filename1:startsWith('y')"
|
||||
+ // false
|
||||
" :or("
|
||||
+ // or
|
||||
" ${ filename1:startsWith('x'):and(false) }"
|
||||
+ // false
|
||||
" ):or("
|
||||
+ // or
|
||||
" ${ filename2:endsWith('xxxx'):or( ${'filename 3':length():gt(1)} ) }"
|
||||
+ // true )
|
||||
" )}"
|
||||
+ " )"
|
||||
+ "}";
|
||||
|
||||
System.out.println(query);
|
||||
verifyEquals(query, attributes, true);
|
||||
|
@ -1164,7 +1148,7 @@ public class TestQuery {
|
|||
attributes.put("b", "x");
|
||||
attributes.put("abcxcba", "hello");
|
||||
|
||||
final String evaluated = Query.evaluateExpressions("${ 'abc${b}cba':substring(0, 1) }", VariableRegistryFactory.getInstance(attributes), null);
|
||||
final String evaluated = Query.evaluateExpressions("${ 'abc${b}cba':substring(0, 1) }", attributes, null);
|
||||
assertEquals("h", evaluated);
|
||||
}
|
||||
|
||||
|
@ -1198,21 +1182,21 @@ public class TestQuery {
|
|||
final List<String> expressions = Query.extractExpressions(query);
|
||||
assertEquals(1, expressions.size());
|
||||
assertEquals("${abc}", expressions.get(0));
|
||||
assertEquals("{ xyz }", Query.evaluateExpressions(query, VariableRegistryFactory.getInstance(attributes)));
|
||||
assertEquals("{ xyz }", Query.evaluateExpressions(query, attributes));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLiteralFunction() {
|
||||
final Map<String, String> attrs = Collections.<String, String> emptyMap();
|
||||
final Map<String, String> attrs = Collections.<String, String>emptyMap();
|
||||
verifyEquals("${literal(2):gt(1)}", attrs, true);
|
||||
verifyEquals("${literal('hello'):substring(0, 1):equals('h')}", attrs, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRandomFunction() {
|
||||
final Map<String, String> attrs = Collections.<String, String> emptyMap();
|
||||
final Map<String, String> attrs = Collections.<String, String>emptyMap();
|
||||
final Long negOne = Long.valueOf(-1L);
|
||||
final HashSet<Long> results = new HashSet<Long>(100);
|
||||
final HashSet<Long> results = new HashSet<>(100);
|
||||
for (int i = 0; i < results.size(); i++) {
|
||||
long result = (Long) getResult("${random()}", attrs).getValue();
|
||||
assertThat("random", result, greaterThan(negOne));
|
||||
|
@ -1222,7 +1206,7 @@ public class TestQuery {
|
|||
|
||||
QueryResult<?> getResult(String expr, Map<String, String> attrs) {
|
||||
final Query query = Query.compile(expr);
|
||||
final QueryResult<?> result = query.evaluate(VariableRegistryFactory.getInstance(attrs));
|
||||
final QueryResult<?> result = query.evaluate(attrs);
|
||||
return result;
|
||||
}
|
||||
|
||||
|
@ -1331,17 +1315,11 @@ public class TestQuery {
|
|||
}
|
||||
|
||||
private void verifyEquals(final String expression, final Map<String, String> attributes, final Object expectedResult) {
|
||||
|
||||
VariableRegistry registry = VariableRegistryFactory.getInstance(attributes);
|
||||
verifyEquals(expression,registry,expectedResult);
|
||||
}
|
||||
|
||||
private void verifyEquals(final String expression, final VariableRegistry registry, final Object expectedResult) {
|
||||
Query.validateExpression(expression, false);
|
||||
assertEquals(String.valueOf(expectedResult), Query.evaluateExpressions(expression, registry, null));
|
||||
assertEquals(String.valueOf(expectedResult), Query.evaluateExpressions(expression, attributes, null));
|
||||
|
||||
final Query query = Query.compile(expression);
|
||||
final QueryResult<?> result = query.evaluate(registry);
|
||||
final QueryResult<?> result = query.evaluate(attributes);
|
||||
|
||||
if (expectedResult instanceof Number) {
|
||||
assertEquals(ResultType.NUMBER, result.getResultType());
|
||||
|
|
|
@ -22,7 +22,6 @@ import java.util.HashMap;
|
|||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.nifi.registry.VariableRegistryFactory;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -54,7 +53,7 @@ public class TestStandardPreparedQuery {
|
|||
final StandardPreparedQuery prepared = (StandardPreparedQuery) Query.prepare("${xx}");
|
||||
final long start = System.nanoTime();
|
||||
for (int i = 0; i < 10000000; i++) {
|
||||
assertEquals("world", prepared.evaluateExpressions(VariableRegistryFactory.getInstance(attrs), null));
|
||||
assertEquals("world", prepared.evaluateExpressions(attrs, null));
|
||||
}
|
||||
final long nanos = System.nanoTime() - start;
|
||||
System.out.println(TimeUnit.NANOSECONDS.toMillis(nanos));
|
||||
|
@ -68,7 +67,7 @@ public class TestStandardPreparedQuery {
|
|||
|
||||
final long start = System.nanoTime();
|
||||
for (int i = 0; i < 10000000; i++) {
|
||||
assertEquals("world", Query.evaluateExpressions("${xx}", VariableRegistryFactory.getInstance(attrs)));
|
||||
assertEquals("world", Query.evaluateExpressions("${xx}", attrs));
|
||||
}
|
||||
final long nanos = System.nanoTime() - start;
|
||||
System.out.println(TimeUnit.NANOSECONDS.toMillis(nanos));
|
||||
|
@ -86,7 +85,7 @@ public class TestStandardPreparedQuery {
|
|||
}
|
||||
|
||||
private String evaluate(final String query, final Map<String, String> attrs) {
|
||||
final String evaluated = ((StandardPreparedQuery) Query.prepare(query)).evaluateExpressions(VariableRegistryFactory.getInstance(attrs), null);
|
||||
final String evaluated = ((StandardPreparedQuery) Query.prepare(query)).evaluateExpressions(attrs, null);
|
||||
return evaluated;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,150 @@
|
|||
/*
|
||||
* 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.attribute.expression.language;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.junit.Test;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class TestValueLookup {
|
||||
|
||||
@Test
|
||||
public void testCreateCustomVariableRegistry() {
|
||||
|
||||
final VariableRegistry variableRegistry = VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY;
|
||||
final ValueLookup initialLookup = new ValueLookup(variableRegistry, null);
|
||||
assertTrue(initialLookup.containsKey("PATH"));
|
||||
assertFalse(initialLookup.containsKey("fake.property.3"));
|
||||
assertFalse(initialLookup.containsKey("fake"));
|
||||
|
||||
final Map<String, String> otherAttrs = new HashMap<>();
|
||||
otherAttrs.put("fake", "test");
|
||||
otherAttrs.put("fake.property.3", "test me out 3, test me out 4");
|
||||
final ValueLookup newLookup = new ValueLookup(variableRegistry, null, otherAttrs);
|
||||
assertTrue(newLookup.containsKey("PATH"));
|
||||
assertTrue(newLookup.containsKey("fake.property.3"));
|
||||
assertEquals("test me out 3, test me out 4", newLookup.get("fake.property.3"));
|
||||
assertEquals("test", newLookup.get("fake"));
|
||||
assertFalse(newLookup.containsKey("filename"));
|
||||
|
||||
final FlowFile fakeFile = createFlowFile();
|
||||
final ValueLookup ffLookup = new ValueLookup(variableRegistry, fakeFile, otherAttrs);
|
||||
assertTrue(ffLookup.containsKey("filename"));
|
||||
assertEquals("test", ffLookup.get("fake"));
|
||||
assertEquals("1", ffLookup.get("flowFileId"));
|
||||
assertEquals("50", ffLookup.get("fileSize"));
|
||||
assertEquals("1000", ffLookup.get("entryDate"));
|
||||
assertEquals("10000", ffLookup.get("lineageStartDate"));
|
||||
assertEquals("fakefile.txt", ffLookup.get("filename"));
|
||||
|
||||
final Map<String, String> overrides = new HashMap<>();
|
||||
overrides.put("fake", "the real deal");
|
||||
final ValueLookup overriddenLookup = new ValueLookup(variableRegistry, fakeFile, overrides, otherAttrs);
|
||||
assertTrue(overriddenLookup.containsKey("filename"));
|
||||
assertEquals("the real deal", overriddenLookup.get("fake"));
|
||||
assertEquals("1", overriddenLookup.get("flowFileId"));
|
||||
assertEquals("50", overriddenLookup.get("fileSize"));
|
||||
assertEquals("1000", overriddenLookup.get("entryDate"));
|
||||
assertEquals("10000", overriddenLookup.get("lineageStartDate"));
|
||||
assertEquals("fakefile.txt", overriddenLookup.get("filename"));
|
||||
assertEquals("original", overriddenLookup.get("override me"));
|
||||
|
||||
final Map<String, String> newOverrides = new HashMap<>();
|
||||
newOverrides.put("fake", "the real deal");
|
||||
newOverrides.put("override me", "done you are now overridden");
|
||||
final ValueLookup newOverriddenLookup = new ValueLookup(variableRegistry, fakeFile, newOverrides, otherAttrs);
|
||||
assertTrue(newOverriddenLookup.containsKey("filename"));
|
||||
assertEquals("the real deal", newOverriddenLookup.get("fake"));
|
||||
assertEquals("1", newOverriddenLookup.get("flowFileId"));
|
||||
assertEquals("50", newOverriddenLookup.get("fileSize"));
|
||||
assertEquals("1000", newOverriddenLookup.get("entryDate"));
|
||||
assertEquals("10000", newOverriddenLookup.get("lineageStartDate"));
|
||||
assertEquals("fakefile.txt", newOverriddenLookup.get("filename"));
|
||||
assertEquals("done you are now overridden", newOverriddenLookup.get("override me"));
|
||||
}
|
||||
|
||||
private FlowFile createFlowFile() {
|
||||
return new FlowFile() {
|
||||
@Override
|
||||
public long getId() {
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getEntryDate() {
|
||||
return 1000;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLineageStartDate() {
|
||||
return 10000;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Long getLastQueueDate() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isPenalized() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getAttribute(String key) {
|
||||
return getAttributes().get(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSize() {
|
||||
return 50;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLineageStartIndex() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getQueueDateIndex() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, String> getAttributes() {
|
||||
Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("filename", "fakefile.txt");
|
||||
attributes.put("override me", "original");
|
||||
return attributes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(FlowFile o) {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
|
@ -986,7 +986,7 @@ public class NiFiProperties extends Properties {
|
|||
|
||||
return vrPropertiesPaths.toArray( new Path[vrPropertiesPaths.size()]);
|
||||
} else {
|
||||
return null;
|
||||
return new Path[]{};
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -20,13 +20,12 @@ import java.util.Collection;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.nifi.controller.ControllerService;
|
||||
import org.apache.nifi.registry.VariableRegistryProvider;
|
||||
|
||||
/**
|
||||
* NiFi web context providing limited access to dataflow configuration for
|
||||
* component custom UIs.
|
||||
*/
|
||||
public interface NiFiWebConfigurationContext extends VariableRegistryProvider {
|
||||
public interface NiFiWebConfigurationContext {
|
||||
|
||||
/**
|
||||
* @param serviceIdentifier of the controller service
|
||||
|
|
|
@ -34,13 +34,21 @@ public class MockConfigurationContext implements ConfigurationContext {
|
|||
private final ControllerService service;
|
||||
private final VariableRegistry variableRegistry;
|
||||
|
||||
public MockConfigurationContext(final Map<PropertyDescriptor, String> properties, final ControllerServiceLookup serviceLookup,
|
||||
final VariableRegistry variableRegistry) {
|
||||
public MockConfigurationContext(final Map<PropertyDescriptor, String> properties,
|
||||
final ControllerServiceLookup serviceLookup) {
|
||||
this(null, properties, serviceLookup, VariableRegistry.EMPTY_REGISTRY);
|
||||
}
|
||||
|
||||
public MockConfigurationContext(final Map<PropertyDescriptor, String> properties,
|
||||
final ControllerServiceLookup serviceLookup,
|
||||
final VariableRegistry variableRegistry) {
|
||||
this(null, properties, serviceLookup, variableRegistry);
|
||||
}
|
||||
|
||||
public MockConfigurationContext(final ControllerService service, final Map<PropertyDescriptor, String> properties, final ControllerServiceLookup serviceLookup,
|
||||
final VariableRegistry variableRegistry) {
|
||||
public MockConfigurationContext(final ControllerService service,
|
||||
final Map<PropertyDescriptor, String> properties,
|
||||
final ControllerServiceLookup serviceLookup,
|
||||
final VariableRegistry variableRegistry) {
|
||||
this.service = service;
|
||||
this.properties = properties;
|
||||
this.serviceLookup = serviceLookup;
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.nifi.processor.SchedulingContext;
|
|||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.state.MockStateManager;
|
||||
import org.junit.Assert;
|
||||
import static java.util.Objects.requireNonNull;
|
||||
|
||||
public class MockProcessContext extends MockControllerServiceLookup implements SchedulingContext, ControllerServiceLookup, NodeTypeProvider {
|
||||
|
||||
|
@ -65,14 +66,15 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
|
|||
private volatile boolean isClustered;
|
||||
private volatile boolean isPrimaryNode;
|
||||
|
||||
public MockProcessContext(final ConfigurableComponent component, final VariableRegistry variableRegistry) {
|
||||
this(component, new MockStateManager(component),variableRegistry);
|
||||
public MockProcessContext(final ConfigurableComponent component) {
|
||||
this(component, new MockStateManager(component),VariableRegistry.EMPTY_REGISTRY);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new MockProcessContext for the given Processor
|
||||
*
|
||||
* @param component being mocked
|
||||
* @param stateManager state manager
|
||||
* @param variableRegistry variableRegistry
|
||||
*/
|
||||
public MockProcessContext(final ConfigurableComponent component, final StateManager stateManager, final VariableRegistry variableRegistry) {
|
||||
|
|
|
@ -39,6 +39,14 @@ public class MockPropertyValue implements PropertyValue {
|
|||
private final VariableRegistry variableRegistry;
|
||||
private boolean expressionsEvaluated = false;
|
||||
|
||||
public MockPropertyValue(final String rawValue) {
|
||||
this(rawValue, null);
|
||||
}
|
||||
|
||||
public MockPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup) {
|
||||
this(rawValue, serviceLookup, VariableRegistry.EMPTY_REGISTRY, null);
|
||||
}
|
||||
|
||||
public MockPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup, final VariableRegistry variableRegistry) {
|
||||
this(rawValue, serviceLookup, variableRegistry, null);
|
||||
}
|
||||
|
|
|
@ -41,6 +41,10 @@ public class MockValidationContext implements ValidationContext, ControllerServi
|
|||
private final StateManager stateManager;
|
||||
private final VariableRegistry variableRegistry;
|
||||
|
||||
public MockValidationContext(final MockProcessContext processContext) {
|
||||
this(processContext, null, VariableRegistry.EMPTY_REGISTRY);
|
||||
}
|
||||
|
||||
public MockValidationContext(final MockProcessContext processContext, final StateManager stateManager, final VariableRegistry variableRegistry) {
|
||||
this.context = processContext;
|
||||
this.stateManager = stateManager;
|
||||
|
|
|
@ -18,12 +18,11 @@ package org.apache.nifi.util;
|
|||
|
||||
import org.apache.nifi.processor.Processor;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
|
||||
public class TestRunners {
|
||||
|
||||
public static TestRunner newTestRunner(final Processor processor) {
|
||||
return newTestRunner(processor,VariableRegistryUtils.createSystemVariableRegistry());
|
||||
return newTestRunner(processor,VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY);
|
||||
}
|
||||
|
||||
public static TestRunner newTestRunner(final Processor processor, VariableRegistry variableRegistry){
|
||||
|
|
|
@ -20,7 +20,7 @@ import org.apache.nifi.processor.AbstractProcessor;
|
|||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -32,7 +32,7 @@ public class CurrentTestStandardProcessorTestRunner {
|
|||
@Test
|
||||
public void testOnScheduledCalledAfterRunFinished() {
|
||||
SlowRunProcessor processor = new SlowRunProcessor();
|
||||
StandardProcessorTestRunner runner = new StandardProcessorTestRunner(processor, VariableRegistryUtils.createSystemVariableRegistry());
|
||||
StandardProcessorTestRunner runner = new StandardProcessorTestRunner(processor, VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY);
|
||||
final int iterations = 5;
|
||||
runner.run(iterations);
|
||||
// if the counter is not equal to iterations, the the processor must have been unscheduled
|
||||
|
|
|
@ -32,7 +32,6 @@ import org.apache.nifi.processor.ProcessContext;
|
|||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestMockProcessContext {
|
||||
|
@ -40,7 +39,7 @@ public class TestMockProcessContext {
|
|||
@Test
|
||||
public void testRemoveProperty() {
|
||||
final DummyProcessor proc = new DummyProcessor();
|
||||
final MockProcessContext context = new MockProcessContext(proc, VariableRegistryUtils.createSystemVariableRegistry());
|
||||
final MockProcessContext context = new MockProcessContext(proc);
|
||||
context.setProperty(DummyProcessor.REQUIRED_PROP, "req-value");
|
||||
context.setProperty(DummyProcessor.OPTIONAL_PROP, "opt-value");
|
||||
context.setProperty(DummyProcessor.DEFAULTED_PROP, "custom-value");
|
||||
|
|
|
@ -20,8 +20,6 @@ import org.apache.nifi.controller.ConfigurationContext;
|
|||
import org.apache.nifi.controller.status.ProcessGroupStatus;
|
||||
import org.apache.nifi.controller.status.ProcessorStatus;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.apache.nifi.reporting.EventAccess;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.apache.nifi.reporting.ReportingContext;
|
||||
|
@ -45,7 +43,6 @@ import java.util.UUID;
|
|||
public class TestAmbariReportingTask {
|
||||
|
||||
private ProcessGroupStatus status;
|
||||
private VariableRegistry variableRegistry;
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
|
@ -76,7 +73,6 @@ public class TestAmbariReportingTask {
|
|||
Collection<ProcessGroupStatus> groupStatuses = new ArrayList<>();
|
||||
groupStatuses.add(groupStatus);
|
||||
status.setProcessGroupStatus(groupStatuses);
|
||||
variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -109,11 +105,11 @@ public class TestAmbariReportingTask {
|
|||
// mock the ReportingContext for onTrigger(...)
|
||||
final ReportingContext context = Mockito.mock(ReportingContext.class);
|
||||
Mockito.when(context.getProperty(AmbariReportingTask.METRICS_COLLECTOR_URL))
|
||||
.thenReturn(new MockPropertyValue(metricsUrl, null, variableRegistry));
|
||||
.thenReturn(new MockPropertyValue(metricsUrl));
|
||||
Mockito.when(context.getProperty(AmbariReportingTask.APPLICATION_ID))
|
||||
.thenReturn(new MockPropertyValue(applicationId, null, variableRegistry));
|
||||
.thenReturn(new MockPropertyValue(applicationId));
|
||||
Mockito.when(context.getProperty(AmbariReportingTask.HOSTNAME))
|
||||
.thenReturn(new MockPropertyValue(hostName, null, variableRegistry));
|
||||
.thenReturn(new MockPropertyValue(hostName));
|
||||
|
||||
|
||||
final EventAccess eventAccess = Mockito.mock(EventAccess.class);
|
||||
|
|
|
@ -18,8 +18,6 @@ package org.apache.nifi.processors.elasticsearch;
|
|||
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.apache.nifi.ssl.SSLContextService;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.MockProcessContext;
|
||||
|
@ -64,13 +62,11 @@ public class TestFetchElasticsearch {
|
|||
|
||||
private InputStream docExample;
|
||||
private TestRunner runner;
|
||||
private VariableRegistry variableRegistry;
|
||||
|
||||
@Before
|
||||
public void setUp() throws IOException {
|
||||
ClassLoader classloader = Thread.currentThread().getContextClassLoader();
|
||||
docExample = classloader.getResourceAsStream("DocumentExample.json");
|
||||
variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
|
||||
}
|
||||
|
||||
|
@ -220,7 +216,7 @@ public class TestFetchElasticsearch {
|
|||
}
|
||||
};
|
||||
|
||||
MockProcessContext context = new MockProcessContext(processor, variableRegistry);
|
||||
MockProcessContext context = new MockProcessContext(processor);
|
||||
processor.initialize(new MockProcessorInitializationContext(processor, context));
|
||||
processor.callCreateElasticsearchClient(context);
|
||||
}
|
||||
|
|
|
@ -25,7 +25,6 @@ import org.apache.nifi.authorization.generated.Authorizers;
|
|||
import org.apache.nifi.authorization.generated.Property;
|
||||
import org.apache.nifi.nar.ExtensionManager;
|
||||
import org.apache.nifi.nar.NarCloseable;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -73,7 +72,6 @@ public class AuthorizerFactoryBean implements FactoryBean, DisposableBean, Autho
|
|||
|
||||
private Authorizer authorizer;
|
||||
private NiFiProperties properties;
|
||||
private VariableRegistry variableRegistry;
|
||||
private final Map<String, Authorizer> authorizers = new HashMap<>();
|
||||
|
||||
|
||||
|
@ -192,7 +190,7 @@ public class AuthorizerFactoryBean implements FactoryBean, DisposableBean, Autho
|
|||
for (final Property property : authorizer.getProperty()) {
|
||||
authorizerProperties.put(property.getName(), property.getValue());
|
||||
}
|
||||
return new StandardAuthorizerConfigurationContext(authorizer.getIdentifier(), authorizerProperties, variableRegistry);
|
||||
return new StandardAuthorizerConfigurationContext(authorizer.getIdentifier(), authorizerProperties);
|
||||
}
|
||||
|
||||
private void performMethodInjection(final Authorizer instance, final Class authorizerClass) throws IllegalAccessException, IllegalArgumentException, InvocationTargetException {
|
||||
|
@ -487,7 +485,4 @@ public class AuthorizerFactoryBean implements FactoryBean, DisposableBean, Autho
|
|||
this.properties = properties;
|
||||
}
|
||||
|
||||
public void setVariableRegistry(VariableRegistry variableRegistry) {
|
||||
this.variableRegistry = variableRegistry;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@
|
|||
<!-- user/entity authorizer -->
|
||||
<bean id="authorizer" class="org.apache.nifi.authorization.AuthorizerFactoryBean">
|
||||
<property name="properties" ref="nifiProperties"/>
|
||||
<property name="variableRegistry" ref="variableRegistry"/>
|
||||
</bean>
|
||||
|
||||
</beans>
|
||||
|
|
|
@ -21,8 +21,6 @@ import org.apache.nifi.authorization.AuthorizationResult.Result;
|
|||
import org.apache.nifi.authorization.exception.AuthorizerCreationException;
|
||||
import org.apache.nifi.authorization.resource.ResourceFactory;
|
||||
import org.apache.nifi.authorization.resource.ResourceType;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.nifi.util.file.FileUtils;
|
||||
import org.junit.After;
|
||||
|
@ -145,7 +143,6 @@ public class FileAuthorizerTest {
|
|||
private File flow;
|
||||
private File flowNoPorts;
|
||||
private File flowWithDns;
|
||||
private VariableRegistry variableRegistry;
|
||||
|
||||
private AuthorizerConfigurationContext configurationContext;
|
||||
|
||||
|
@ -181,8 +178,8 @@ public class FileAuthorizerTest {
|
|||
when(properties.getFlowConfigurationFile()).thenReturn(flow);
|
||||
|
||||
configurationContext = mock(AuthorizerConfigurationContext.class);
|
||||
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_AUTHORIZATIONS_FILE))).thenReturn(new StandardPropertyValue(primaryAuthorizations.getPath(), null,variableRegistry));
|
||||
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_TENANTS_FILE))).thenReturn(new StandardPropertyValue(primaryTenants.getPath(), null,variableRegistry));
|
||||
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_AUTHORIZATIONS_FILE))).thenReturn(new StandardPropertyValue(primaryAuthorizations.getPath(), null));
|
||||
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_TENANTS_FILE))).thenReturn(new StandardPropertyValue(primaryTenants.getPath(), null));
|
||||
|
||||
authorizer = new FileAuthorizer();
|
||||
authorizer.setNiFiProperties(properties);
|
||||
|
@ -200,7 +197,7 @@ public class FileAuthorizerTest {
|
|||
@Test
|
||||
public void testOnConfiguredWhenLegacyUsersFileProvidedWithOverlappingRoles() throws Exception {
|
||||
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
|
||||
.thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-multirole.xml", null,VariableRegistryUtils.createSystemVariableRegistry()));
|
||||
.thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-multirole.xml", null));
|
||||
|
||||
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
|
||||
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
|
||||
|
@ -225,7 +222,7 @@ public class FileAuthorizerTest {
|
|||
when(properties.getFlowConfigurationFile()).thenReturn(flowNoPorts);
|
||||
|
||||
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
|
||||
.thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null,variableRegistry));
|
||||
.thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null));
|
||||
|
||||
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
|
||||
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
|
||||
|
@ -245,7 +242,7 @@ public class FileAuthorizerTest {
|
|||
@Test
|
||||
public void testOnConfiguredWhenLegacyUsersFileProvided() throws Exception {
|
||||
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
|
||||
.thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null,VariableRegistryUtils.createSystemVariableRegistry()));
|
||||
.thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null));
|
||||
|
||||
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
|
||||
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
|
||||
|
@ -395,7 +392,7 @@ public class FileAuthorizerTest {
|
|||
authorizer.setNiFiProperties(properties);
|
||||
|
||||
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
|
||||
.thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-with-dns.xml", null,variableRegistry));
|
||||
.thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-with-dns.xml", null));
|
||||
|
||||
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
|
||||
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
|
||||
|
@ -443,7 +440,7 @@ public class FileAuthorizerTest {
|
|||
@Test(expected = AuthorizerCreationException.class)
|
||||
public void testOnConfiguredWhenBadLegacyUsersFileProvided() throws Exception {
|
||||
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
|
||||
.thenReturn(new StandardPropertyValue("src/test/resources/does-not-exist.xml", null,VariableRegistryUtils.createSystemVariableRegistry()));
|
||||
.thenReturn(new StandardPropertyValue("src/test/resources/does-not-exist.xml", null));
|
||||
|
||||
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
|
||||
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
|
||||
|
@ -453,12 +450,11 @@ public class FileAuthorizerTest {
|
|||
@Test(expected = AuthorizerCreationException.class)
|
||||
public void testOnConfiguredWhenInitialAdminAndLegacyUsersProvided() throws Exception {
|
||||
final String adminIdentity = "admin-user";
|
||||
final VariableRegistry variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_INITIAL_ADMIN_IDENTITY)))
|
||||
.thenReturn(new StandardPropertyValue(adminIdentity, null,variableRegistry));
|
||||
.thenReturn(new StandardPropertyValue(adminIdentity, null));
|
||||
|
||||
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
|
||||
.thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null,variableRegistry));
|
||||
.thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null));
|
||||
|
||||
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
|
||||
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
|
||||
|
@ -483,7 +479,7 @@ public class FileAuthorizerTest {
|
|||
final String adminIdentity = "admin-user";
|
||||
|
||||
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_INITIAL_ADMIN_IDENTITY)))
|
||||
.thenReturn(new StandardPropertyValue(adminIdentity, null,VariableRegistryUtils.createSystemVariableRegistry()));
|
||||
.thenReturn(new StandardPropertyValue(adminIdentity, null));
|
||||
|
||||
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
|
||||
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
|
||||
|
@ -513,7 +509,6 @@ public class FileAuthorizerTest {
|
|||
|
||||
@Test
|
||||
public void testOnConfiguredWhenInitialAdminProvidedAndNoFlowExists() throws Exception {
|
||||
VariableRegistry variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
// setup NiFi properties to return a file that does not exist
|
||||
properties = mock(NiFiProperties.class);
|
||||
when(properties.getRestoreDirectory()).thenReturn(restoreAuthorizations.getParentFile());
|
||||
|
@ -522,7 +517,7 @@ public class FileAuthorizerTest {
|
|||
|
||||
final String adminIdentity = "admin-user";
|
||||
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_INITIAL_ADMIN_IDENTITY)))
|
||||
.thenReturn(new StandardPropertyValue(adminIdentity, null, variableRegistry));
|
||||
.thenReturn(new StandardPropertyValue(adminIdentity, null));
|
||||
|
||||
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
|
||||
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
|
||||
|
@ -553,7 +548,6 @@ public class FileAuthorizerTest {
|
|||
@Test
|
||||
public void testOnConfiguredWhenInitialAdminProvidedAndFlowIsNull() throws Exception {
|
||||
// setup NiFi properties to return a file that does not exist
|
||||
VariableRegistry variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
properties = mock(NiFiProperties.class);
|
||||
when(properties.getRestoreDirectory()).thenReturn(restoreAuthorizations.getParentFile());
|
||||
when(properties.getFlowConfigurationFile()).thenReturn(null);
|
||||
|
@ -561,7 +555,7 @@ public class FileAuthorizerTest {
|
|||
|
||||
final String adminIdentity = "admin-user";
|
||||
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_INITIAL_ADMIN_IDENTITY)))
|
||||
.thenReturn(new StandardPropertyValue(adminIdentity, null, variableRegistry));
|
||||
.thenReturn(new StandardPropertyValue(adminIdentity, null));
|
||||
|
||||
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
|
||||
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
|
||||
|
@ -602,7 +596,7 @@ public class FileAuthorizerTest {
|
|||
|
||||
final String adminIdentity = "CN=localhost, OU=Apache NiFi, O=Apache, L=Santa Monica, ST=CA, C=US";
|
||||
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_INITIAL_ADMIN_IDENTITY)))
|
||||
.thenReturn(new StandardPropertyValue(adminIdentity, null,variableRegistry));
|
||||
.thenReturn(new StandardPropertyValue(adminIdentity, null));
|
||||
|
||||
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
|
||||
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
|
||||
|
@ -620,7 +614,7 @@ public class FileAuthorizerTest {
|
|||
final String adminIdentity = "admin-user";
|
||||
|
||||
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_INITIAL_ADMIN_IDENTITY)))
|
||||
.thenReturn(new StandardPropertyValue(adminIdentity, null,VariableRegistryUtils.createSystemVariableRegistry()));
|
||||
.thenReturn(new StandardPropertyValue(adminIdentity, null));
|
||||
|
||||
final String nodeIdentity1 = "node1";
|
||||
final String nodeIdentity2 = "node2";
|
||||
|
@ -669,7 +663,7 @@ public class FileAuthorizerTest {
|
|||
|
||||
final String adminIdentity = "CN=user1, OU=Apache NiFi, O=Apache, L=Santa Monica, ST=CA, C=US";
|
||||
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_INITIAL_ADMIN_IDENTITY)))
|
||||
.thenReturn(new StandardPropertyValue(adminIdentity, null,variableRegistry));
|
||||
.thenReturn(new StandardPropertyValue(adminIdentity, null));
|
||||
|
||||
final String nodeIdentity1 = "CN=node1, OU=Apache NiFi, O=Apache, L=Santa Monica, ST=CA, C=US";
|
||||
final String nodeIdentity2 = "CN=node2, OU=Apache NiFi, O=Apache, L=Santa Monica, ST=CA, C=US";
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.nifi.authorization;
|
|||
|
||||
import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
|
||||
import org.apache.nifi.components.PropertyValue;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
|
@ -31,12 +30,10 @@ public class StandardAuthorizerConfigurationContext implements AuthorizerConfigu
|
|||
|
||||
private final String identifier;
|
||||
private final Map<String, String> properties;
|
||||
final VariableRegistry variableRegistry;
|
||||
|
||||
public StandardAuthorizerConfigurationContext(String identifier, Map<String, String> properties, VariableRegistry variableRegistry) {
|
||||
public StandardAuthorizerConfigurationContext(String identifier, Map<String, String> properties) {
|
||||
this.identifier = identifier;
|
||||
this.properties = Collections.unmodifiableMap(new HashMap<String, String>(properties));
|
||||
this.variableRegistry = variableRegistry;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -51,7 +48,7 @@ public class StandardAuthorizerConfigurationContext implements AuthorizerConfigu
|
|||
|
||||
@Override
|
||||
public PropertyValue getProperty(String property) {
|
||||
return new StandardPropertyValue(properties.get(property), null,variableRegistry);
|
||||
return new StandardPropertyValue(properties.get(property), null);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -238,7 +238,7 @@ import java.util.concurrent.locks.Lock;
|
|||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import static java.util.Objects.requireNonNull;
|
||||
|
||||
import static java.util.Objects.requireNonNull;
|
||||
|
||||
public class FlowController implements EventAccess, ControllerServiceProvider, ReportingTaskProvider, QueueProvider, Authorizable, ProvenanceAuthorizableFactory, NodeTypeProvider {
|
||||
|
||||
|
@ -324,7 +324,6 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
*/
|
||||
private final StringEncryptor encryptor;
|
||||
|
||||
|
||||
private final ScheduledExecutorService clusterTaskExecutor = new FlowEngine(3, "Clustering Tasks", true);
|
||||
private final ResourceClaimManager resourceClaimManager = new StandardResourceClaimManager();
|
||||
|
||||
|
@ -376,19 +375,21 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
final Authorizer authorizer,
|
||||
final AuditService auditService,
|
||||
final StringEncryptor encryptor,
|
||||
final BulletinRepository bulletinRepo, VariableRegistry variableRegistry) {
|
||||
final BulletinRepository bulletinRepo,
|
||||
final VariableRegistry variableRegistry) {
|
||||
|
||||
return new FlowController(
|
||||
flowFileEventRepo,
|
||||
properties,
|
||||
authorizer,
|
||||
auditService,
|
||||
encryptor,
|
||||
/* configuredForClustering */ false,
|
||||
/* NodeProtocolSender */ null,
|
||||
bulletinRepo,
|
||||
/* cluster coordinator */ null,
|
||||
/* heartbeat monitor */ null, variableRegistry);
|
||||
flowFileEventRepo,
|
||||
properties,
|
||||
authorizer,
|
||||
auditService,
|
||||
encryptor,
|
||||
/* configuredForClustering */ false,
|
||||
/* NodeProtocolSender */ null,
|
||||
bulletinRepo,
|
||||
/* cluster coordinator */ null,
|
||||
/* heartbeat monitor */ null,
|
||||
/* variable registry */ variableRegistry);
|
||||
}
|
||||
|
||||
public static FlowController createClusteredInstance(
|
||||
|
@ -404,16 +405,16 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
VariableRegistry variableRegistry) {
|
||||
|
||||
final FlowController flowController = new FlowController(
|
||||
flowFileEventRepo,
|
||||
properties,
|
||||
authorizer,
|
||||
auditService,
|
||||
encryptor,
|
||||
/* configuredForClustering */ true,
|
||||
protocolSender,
|
||||
bulletinRepo,
|
||||
clusterCoordinator,
|
||||
heartbeatMonitor, variableRegistry);
|
||||
flowFileEventRepo,
|
||||
properties,
|
||||
authorizer,
|
||||
auditService,
|
||||
encryptor,
|
||||
/* configuredForClustering */ true,
|
||||
protocolSender,
|
||||
bulletinRepo,
|
||||
clusterCoordinator,
|
||||
heartbeatMonitor, variableRegistry);
|
||||
|
||||
return flowController;
|
||||
}
|
||||
|
@ -429,7 +430,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
final BulletinRepository bulletinRepo,
|
||||
final ClusterCoordinator clusterCoordinator,
|
||||
final HeartbeatMonitor heartbeatMonitor,
|
||||
VariableRegistry variableRegistry) {
|
||||
final VariableRegistry variableRegistry) {
|
||||
|
||||
maxTimerDrivenThreads = new AtomicInteger(10);
|
||||
maxEventDrivenThreads = new AtomicInteger(5);
|
||||
|
@ -447,11 +448,10 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
final FlowFileRepository flowFileRepo = createFlowFileRepository(properties, resourceClaimManager);
|
||||
flowFileRepository = flowFileRepo;
|
||||
flowFileEventRepository = flowFileEventRepo;
|
||||
counterRepositoryRef = new AtomicReference<CounterRepository>(new StandardCounterRepository());
|
||||
counterRepositoryRef = new AtomicReference<>(new StandardCounterRepository());
|
||||
|
||||
bulletinRepository = bulletinRepo;
|
||||
this.variableRegistry = variableRegistry;
|
||||
|
||||
this.variableRegistry = variableRegistry == null ? VariableRegistry.EMPTY_REGISTRY : variableRegistry;
|
||||
|
||||
try {
|
||||
this.provenanceRepository = createProvenanceRepository(properties);
|
||||
|
@ -477,7 +477,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
|
||||
final ProcessContextFactory contextFactory = new ProcessContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceRepository);
|
||||
processScheduler.setSchedulingAgent(SchedulingStrategy.EVENT_DRIVEN, new EventDrivenSchedulingAgent(
|
||||
eventDrivenEngineRef.get(), this, stateManagerProvider, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor, this.variableRegistry));
|
||||
eventDrivenEngineRef.get(), this, stateManagerProvider, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor, this.variableRegistry));
|
||||
|
||||
final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor, this.variableRegistry);
|
||||
final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor, this.variableRegistry);
|
||||
|
@ -595,7 +595,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
registerForClusterCoordinator();
|
||||
} else {
|
||||
LOG.info("The Elected Cluster Coordinator is {}. Will not register to be elected for this role until after connecting "
|
||||
+ "to the cluster and inheriting the cluster's flow.", electedCoordinatorNodeId);
|
||||
+ "to the cluster and inheriting the cluster's flow.", electedCoordinatorNodeId);
|
||||
}
|
||||
|
||||
leaderElectionManager.start();
|
||||
|
@ -623,7 +623,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
final String implementationClassName = properties.getProperty(NiFiProperties.FLOWFILE_REPOSITORY_IMPLEMENTATION, DEFAULT_FLOWFILE_REPO_IMPLEMENTATION);
|
||||
if (implementationClassName == null) {
|
||||
throw new RuntimeException("Cannot create FlowFile Repository because the NiFi Properties is missing the following property: "
|
||||
+ NiFiProperties.FLOWFILE_REPOSITORY_IMPLEMENTATION);
|
||||
+ NiFiProperties.FLOWFILE_REPOSITORY_IMPLEMENTATION);
|
||||
}
|
||||
|
||||
try {
|
||||
|
@ -750,7 +750,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
|
||||
/**
|
||||
* <p>
|
||||
* Causes any processors that were added to the flow with a 'delayStart' flag of true to now start
|
||||
* Causes any processors that were added to the flow with a 'delayStart'
|
||||
* flag of true to now start
|
||||
* </p>
|
||||
*
|
||||
* @param startDelayedComponents true if start
|
||||
|
@ -772,7 +773,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
startConnectable(connectable);
|
||||
}
|
||||
} catch (final Throwable t) {
|
||||
LOG.error("Unable to start {} due to {}", new Object[] {connectable, t.toString()});
|
||||
LOG.error("Unable to start {} due to {}", new Object[]{connectable, t.toString()});
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.error("", t);
|
||||
}
|
||||
|
@ -787,7 +788,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
remoteGroupPort.getRemoteProcessGroup().startTransmitting(remoteGroupPort);
|
||||
startedTransmitting++;
|
||||
} catch (final Throwable t) {
|
||||
LOG.error("Unable to start transmitting with {} due to {}", new Object[] {remoteGroupPort, t});
|
||||
LOG.error("Unable to start transmitting with {} due to {}", new Object[]{remoteGroupPort, t});
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -802,7 +803,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
startConnectable(connectable);
|
||||
}
|
||||
} catch (final Throwable t) {
|
||||
LOG.error("Unable to start {} due to {}", new Object[] {connectable, t});
|
||||
LOG.error("Unable to start {} due to {}", new Object[]{connectable, t});
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -818,7 +819,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
final String implementationClassName = properties.getProperty(NiFiProperties.CONTENT_REPOSITORY_IMPLEMENTATION, DEFAULT_CONTENT_REPO_IMPLEMENTATION);
|
||||
if (implementationClassName == null) {
|
||||
throw new RuntimeException("Cannot create Content Repository because the NiFi Properties is missing the following property: "
|
||||
+ NiFiProperties.CONTENT_REPOSITORY_IMPLEMENTATION);
|
||||
+ NiFiProperties.CONTENT_REPOSITORY_IMPLEMENTATION);
|
||||
}
|
||||
|
||||
try {
|
||||
|
@ -836,7 +837,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
final String implementationClassName = properties.getProperty(NiFiProperties.PROVENANCE_REPO_IMPLEMENTATION_CLASS, DEFAULT_PROVENANCE_REPO_IMPLEMENTATION);
|
||||
if (implementationClassName == null) {
|
||||
throw new RuntimeException("Cannot create Provenance Repository because the NiFi Properties is missing the following property: "
|
||||
+ NiFiProperties.PROVENANCE_REPO_IMPLEMENTATION_CLASS);
|
||||
+ NiFiProperties.PROVENANCE_REPO_IMPLEMENTATION_CLASS);
|
||||
}
|
||||
|
||||
try {
|
||||
|
@ -850,7 +851,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
final String implementationClassName = properties.getProperty(NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION, DEFAULT_COMPONENT_STATUS_REPO_IMPLEMENTATION);
|
||||
if (implementationClassName == null) {
|
||||
throw new RuntimeException("Cannot create Component Status Repository because the NiFi Properties is missing the following property: "
|
||||
+ NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION);
|
||||
+ NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION);
|
||||
}
|
||||
|
||||
try {
|
||||
|
@ -864,14 +865,17 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
* Creates a connection between two Connectable objects.
|
||||
*
|
||||
* @param id required ID of the connection
|
||||
* @param name the name of the connection, or <code>null</code> to leave the connection unnamed
|
||||
* @param name the name of the connection, or <code>null</code> to leave the
|
||||
* connection unnamed
|
||||
* @param source required source
|
||||
* @param destination required destination
|
||||
* @param relationshipNames required collection of relationship names
|
||||
* @return
|
||||
*
|
||||
* @throws NullPointerException if the ID, source, destination, or set of relationships is null.
|
||||
* @throws IllegalArgumentException if <code>relationships</code> is an empty collection
|
||||
* @throws NullPointerException if the ID, source, destination, or set of
|
||||
* relationships is null.
|
||||
* @throws IllegalArgumentException if <code>relationships</code> is an
|
||||
* empty collection
|
||||
*/
|
||||
public Connection createConnection(final String id, final String name, final Connectable source, final Connectable destination, final Collection<String> relationshipNames) {
|
||||
final StandardConnection.Builder builder = new StandardConnection.Builder(processScheduler);
|
||||
|
@ -948,7 +952,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
* @param name port name
|
||||
* @return new port
|
||||
* @throws NullPointerException if the ID or name is not unique
|
||||
* @throws IllegalStateException if an Input Port already exists with the same name or id.
|
||||
* @throws IllegalStateException if an Input Port already exists with the
|
||||
* same name or id.
|
||||
*/
|
||||
public Port createLocalInputPort(String id, String name) {
|
||||
id = requireNonNull(id).intern();
|
||||
|
@ -964,7 +969,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
* @param name port name
|
||||
* @return new port
|
||||
* @throws NullPointerException if the ID or name is not unique
|
||||
* @throws IllegalStateException if an Input Port already exists with the same name or id.
|
||||
* @throws IllegalStateException if an Input Port already exists with the
|
||||
* same name or id.
|
||||
*/
|
||||
public Port createLocalOutputPort(String id, String name) {
|
||||
id = requireNonNull(id).intern();
|
||||
|
@ -986,14 +992,16 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
|
||||
/**
|
||||
* <p>
|
||||
* Creates a new ProcessorNode with the given type and identifier and initializes it invoking the methods annotated with {@link OnAdded}.
|
||||
* Creates a new ProcessorNode with the given type and identifier and
|
||||
* initializes it invoking the methods annotated with {@link OnAdded}.
|
||||
* </p>
|
||||
*
|
||||
* @param type processor type
|
||||
* @param id processor id
|
||||
* @return new processor
|
||||
* @throws NullPointerException if either arg is null
|
||||
* @throws ProcessorInstantiationException if the processor cannot be instantiated for any reason
|
||||
* @throws ProcessorInstantiationException if the processor cannot be
|
||||
* instantiated for any reason
|
||||
*/
|
||||
public ProcessorNode createProcessor(final String type, final String id) throws ProcessorInstantiationException {
|
||||
return createProcessor(type, id, true);
|
||||
|
@ -1001,15 +1009,19 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
|
||||
/**
|
||||
* <p>
|
||||
* Creates a new ProcessorNode with the given type and identifier and optionally initializes it.
|
||||
* Creates a new ProcessorNode with the given type and identifier and
|
||||
* optionally initializes it.
|
||||
* </p>
|
||||
*
|
||||
* @param type the fully qualified Processor class name
|
||||
* @param id the unique ID of the Processor
|
||||
* @param firstTimeAdded whether or not this is the first time this Processor is added to the graph. If {@code true}, will invoke methods annotated with the {@link OnAdded} annotation.
|
||||
* @param firstTimeAdded whether or not this is the first time this
|
||||
* Processor is added to the graph. If {@code true}, will invoke methods
|
||||
* annotated with the {@link OnAdded} annotation.
|
||||
* @return new processor node
|
||||
* @throws NullPointerException if either arg is null
|
||||
* @throws ProcessorInstantiationException if the processor cannot be instantiated for any reason
|
||||
* @throws ProcessorInstantiationException if the processor cannot be
|
||||
* instantiated for any reason
|
||||
*/
|
||||
public ProcessorNode createProcessor(final String type, String id, final boolean firstTimeAdded) throws ProcessorInstantiationException {
|
||||
id = id.intern();
|
||||
|
@ -1093,7 +1105,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* @return the ExtensionManager used for instantiating Processors, Prioritizers, etc.
|
||||
* @return the ExtensionManager used for instantiating Processors,
|
||||
* Prioritizers, etc.
|
||||
*/
|
||||
public ExtensionManager getExtensionManager() {
|
||||
return extensionManager;
|
||||
|
@ -1132,41 +1145,47 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* Creates a Port to use as an Input Port for the root Process Group, which is used for Site-to-Site communications
|
||||
* Creates a Port to use as an Input Port for the root Process Group, which
|
||||
* is used for Site-to-Site communications
|
||||
*
|
||||
* @param id port id
|
||||
* @param name port name
|
||||
* @return new port
|
||||
* @throws NullPointerException if the ID or name is not unique
|
||||
* @throws IllegalStateException if an Input Port already exists with the same name or id.
|
||||
* @throws IllegalStateException if an Input Port already exists with the
|
||||
* same name or id.
|
||||
*/
|
||||
public Port createRemoteInputPort(String id, String name) {
|
||||
id = requireNonNull(id).intern();
|
||||
name = requireNonNull(name).intern();
|
||||
verifyPortIdDoesNotExist(id);
|
||||
return new StandardRootGroupPort(id, name, null, TransferDirection.RECEIVE, ConnectableType.INPUT_PORT,
|
||||
authorizer, getBulletinRepository(), processScheduler, Boolean.TRUE.equals(isSiteToSiteSecure));
|
||||
authorizer, getBulletinRepository(), processScheduler, Boolean.TRUE.equals(isSiteToSiteSecure));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a Port to use as an Output Port for the root Process Group, which is used for Site-to-Site communications and will queue flow files waiting to be delivered to remote instances
|
||||
* Creates a Port to use as an Output Port for the root Process Group, which
|
||||
* is used for Site-to-Site communications and will queue flow files waiting
|
||||
* to be delivered to remote instances
|
||||
*
|
||||
* @param id port id
|
||||
* @param name port name
|
||||
* @return new port
|
||||
* @throws NullPointerException if the ID or name is not unique
|
||||
* @throws IllegalStateException if an Input Port already exists with the same name or id.
|
||||
* @throws IllegalStateException if an Input Port already exists with the
|
||||
* same name or id.
|
||||
*/
|
||||
public Port createRemoteOutputPort(String id, String name) {
|
||||
id = requireNonNull(id).intern();
|
||||
name = requireNonNull(name).intern();
|
||||
verifyPortIdDoesNotExist(id);
|
||||
return new StandardRootGroupPort(id, name, null, TransferDirection.SEND, ConnectableType.OUTPUT_PORT,
|
||||
authorizer, getBulletinRepository(), processScheduler, Boolean.TRUE.equals(isSiteToSiteSecure));
|
||||
authorizer, getBulletinRepository(), processScheduler, Boolean.TRUE.equals(isSiteToSiteSecure));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new Remote Process Group with the given ID that points to the given URI
|
||||
* Creates a new Remote Process Group with the given ID that points to the
|
||||
* given URI
|
||||
*
|
||||
* @param id group id
|
||||
* @param uri group uri
|
||||
|
@ -1179,7 +1198,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* Verifies that no output port exists with the given id or name. If this does not hold true, throws an IllegalStateException
|
||||
* Verifies that no output port exists with the given id or name. If this
|
||||
* does not hold true, throws an IllegalStateException
|
||||
*
|
||||
* @param id port identifier
|
||||
* @throws IllegalStateException port already exists
|
||||
|
@ -1196,7 +1216,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* @return the name of this controller, which is also the name of the Root Group.
|
||||
* @return the name of this controller, which is also the name of the Root
|
||||
* Group.
|
||||
*/
|
||||
public String getName() {
|
||||
readLock.lock();
|
||||
|
@ -1208,7 +1229,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* Sets the name for the Root Group, which also changes the name for the controller.
|
||||
* Sets the name for the Root Group, which also changes the name for the
|
||||
* controller.
|
||||
*
|
||||
* @param name of root group
|
||||
*/
|
||||
|
@ -1222,7 +1244,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* @return the comments of this controller, which is also the comment of the Root Group
|
||||
* @return the comments of this controller, which is also the comment of the
|
||||
* Root Group
|
||||
*/
|
||||
public String getComments() {
|
||||
readLock.lock();
|
||||
|
@ -1236,7 +1259,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
/**
|
||||
* Sets the comments
|
||||
*
|
||||
* @param comments for the Root Group, which also changes the comment for the controller
|
||||
* @param comments for the Root Group, which also changes the comment for
|
||||
* the controller
|
||||
*/
|
||||
public void setComments(final String comments) {
|
||||
readLock.lock();
|
||||
|
@ -1248,7 +1272,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* @return <code>true</code> if the scheduling engine for this controller has been terminated.
|
||||
* @return <code>true</code> if the scheduling engine for this controller
|
||||
* has been terminated.
|
||||
*/
|
||||
public boolean isTerminated() {
|
||||
this.readLock.lock();
|
||||
|
@ -1260,12 +1285,16 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* Triggers the controller to begin shutdown, stopping all processors and terminating the scheduling engine. After calling this method, the {@link #isTerminated()} method will indicate whether or
|
||||
* not the shutdown has finished.
|
||||
* Triggers the controller to begin shutdown, stopping all processors and
|
||||
* terminating the scheduling engine. After calling this method, the
|
||||
* {@link #isTerminated()} method will indicate whether or not the shutdown
|
||||
* has finished.
|
||||
*
|
||||
* @param kill if <code>true</code>, attempts to stop all active threads, but makes no guarantee that this will happen
|
||||
* @param kill if <code>true</code>, attempts to stop all active threads,
|
||||
* but makes no guarantee that this will happen
|
||||
*
|
||||
* @throws IllegalStateException if the controller is already stopped or currently in the processor of stopping
|
||||
* @throws IllegalStateException if the controller is already stopped or
|
||||
* currently in the processor of stopping
|
||||
*/
|
||||
public void shutdown(final boolean kill) {
|
||||
this.shutdown = true;
|
||||
|
@ -1328,14 +1357,14 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
try {
|
||||
flowFileRepository.close();
|
||||
} catch (final Throwable t) {
|
||||
LOG.warn("Unable to shut down FlowFileRepository due to {}", new Object[] {t});
|
||||
LOG.warn("Unable to shut down FlowFileRepository due to {}", new Object[]{t});
|
||||
}
|
||||
|
||||
if (this.timerDrivenEngineRef.get().isTerminated() && eventDrivenEngineRef.get().isTerminated()) {
|
||||
LOG.info("Controller has been terminated successfully.");
|
||||
} else {
|
||||
LOG.warn("Controller hasn't terminated properly. There exists an uninterruptable thread that "
|
||||
+ "will take an indeterminate amount of time to stop. Might need to kill the program manually.");
|
||||
+ "will take an indeterminate amount of time to stop. Might need to kill the program manually.");
|
||||
}
|
||||
|
||||
for (final RemoteSiteListener listener : externalSiteListeners) {
|
||||
|
@ -1365,13 +1394,13 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Serializes the current state of the controller to the given OutputStream
|
||||
*
|
||||
* @param serializer serializer
|
||||
* @param os stream
|
||||
* @throws FlowSerializationException if serialization of the flow fails for any reason
|
||||
* @throws FlowSerializationException if serialization of the flow fails for
|
||||
* any reason
|
||||
*/
|
||||
public void serialize(final FlowSerializer serializer, final OutputStream os) throws FlowSerializationException {
|
||||
readLock.lock();
|
||||
|
@ -1385,17 +1414,24 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
/**
|
||||
* Synchronizes this controller with the proposed flow.
|
||||
*
|
||||
* For more details, see {@link FlowSynchronizer#sync(FlowController, DataFlow, StringEncryptor)}.
|
||||
* For more details, see
|
||||
* {@link FlowSynchronizer#sync(FlowController, DataFlow, StringEncryptor)}.
|
||||
*
|
||||
* @param synchronizer synchronizer
|
||||
* @param dataFlow the flow to load the controller with. If the flow is null or zero length, then the controller must not have a flow or else an UninheritableFlowException will be thrown.
|
||||
* @param dataFlow the flow to load the controller with. If the flow is null
|
||||
* or zero length, then the controller must not have a flow or else an
|
||||
* UninheritableFlowException will be thrown.
|
||||
*
|
||||
* @throws FlowSerializationException if proposed flow is not a valid flow configuration file
|
||||
* @throws UninheritableFlowException if the proposed flow cannot be loaded by the controller because in doing so would risk orphaning flow files
|
||||
* @throws FlowSynchronizationException if updates to the controller failed. If this exception is thrown, then the controller should be considered unsafe to be used
|
||||
* @throws FlowSerializationException if proposed flow is not a valid flow
|
||||
* configuration file
|
||||
* @throws UninheritableFlowException if the proposed flow cannot be loaded
|
||||
* by the controller because in doing so would risk orphaning flow files
|
||||
* @throws FlowSynchronizationException if updates to the controller failed.
|
||||
* If this exception is thrown, then the controller should be considered
|
||||
* unsafe to be used
|
||||
*/
|
||||
public void synchronize(final FlowSynchronizer synchronizer, final DataFlow dataFlow)
|
||||
throws FlowSerializationException, FlowSynchronizationException, UninheritableFlowException {
|
||||
throws FlowSerializationException, FlowSynchronizationException, UninheritableFlowException {
|
||||
writeLock.lock();
|
||||
try {
|
||||
LOG.debug("Synchronizing controller with proposed flow");
|
||||
|
@ -1407,7 +1443,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* @return the currently configured maximum number of threads that can be used for executing processors at any given time.
|
||||
* @return the currently configured maximum number of threads that can be
|
||||
* used for executing processors at any given time.
|
||||
*/
|
||||
public int getMaxTimerDrivenThreadCount() {
|
||||
return maxTimerDrivenThreads.get();
|
||||
|
@ -1437,9 +1474,11 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* Updates the number of threads that can be simultaneously used for executing processors.
|
||||
* Updates the number of threads that can be simultaneously used for
|
||||
* executing processors.
|
||||
*
|
||||
* @param maxThreadCount This method must be called while holding the write lock!
|
||||
* @param maxThreadCount This method must be called while holding the write
|
||||
* lock!
|
||||
*/
|
||||
private void setMaxThreadCount(final int maxThreadCount, final FlowEngine engine, final AtomicInteger maxThreads) {
|
||||
if (maxThreadCount < 1) {
|
||||
|
@ -1470,7 +1509,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
* @param group the ProcessGroup that is to become the new Root Group
|
||||
*
|
||||
* @throws IllegalArgumentException if the ProcessGroup has a parent
|
||||
* @throws IllegalStateException if the FlowController does not know about the given process group
|
||||
* @throws IllegalStateException if the FlowController does not know about
|
||||
* the given process group
|
||||
*/
|
||||
void setRootGroup(final ProcessGroup group) {
|
||||
if (requireNonNull(group).getParent() != null) {
|
||||
|
@ -1501,13 +1541,17 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
// ProcessGroup access
|
||||
//
|
||||
/**
|
||||
* Updates the process group corresponding to the specified DTO. Any field in DTO that is <code>null</code> (with the exception of the required ID) will be ignored.
|
||||
* Updates the process group corresponding to the specified DTO. Any field
|
||||
* in DTO that is <code>null</code> (with the exception of the required ID)
|
||||
* will be ignored.
|
||||
*
|
||||
* @param dto group
|
||||
* @throws ProcessorInstantiationException
|
||||
*
|
||||
* @throws IllegalStateException if no process group can be found with the ID of DTO or with the ID of the DTO's parentGroupId, if the template ID specified is invalid, or if the DTO's Parent
|
||||
* Group ID changes but the parent group has incoming or outgoing connections
|
||||
* @throws IllegalStateException if no process group can be found with the
|
||||
* ID of DTO or with the ID of the DTO's parentGroupId, if the template ID
|
||||
* specified is invalid, or if the DTO's Parent Group ID changes but the
|
||||
* parent group has incoming or outgoing connections
|
||||
*
|
||||
* @throws NullPointerException if the DTO or its ID is null
|
||||
*/
|
||||
|
@ -1529,7 +1573,6 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
private Position toPosition(final PositionDTO dto) {
|
||||
return new Position(dto.getX(), dto.getY());
|
||||
}
|
||||
|
@ -1538,15 +1581,21 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
// Snippet
|
||||
//
|
||||
/**
|
||||
* Creates an instance of the given snippet and adds the components to the given group
|
||||
* Creates an instance of the given snippet and adds the components to the
|
||||
* given group
|
||||
*
|
||||
* @param group group
|
||||
* @param dto dto
|
||||
*
|
||||
* @throws NullPointerException if either argument is null
|
||||
* @throws IllegalStateException if the snippet is not valid because a component in the snippet has an ID that is not unique to this flow, or because it shares an Input Port or Output Port at the
|
||||
* root level whose name already exists in the given ProcessGroup, or because the Template contains a Processor or a Prioritizer whose class is not valid within this instance of NiFi.
|
||||
* @throws ProcessorInstantiationException if unable to instantiate a processor
|
||||
* @throws IllegalStateException if the snippet is not valid because a
|
||||
* component in the snippet has an ID that is not unique to this flow, or
|
||||
* because it shares an Input Port or Output Port at the root level whose
|
||||
* name already exists in the given ProcessGroup, or because the Template
|
||||
* contains a Processor or a Prioritizer whose class is not valid within
|
||||
* this instance of NiFi.
|
||||
* @throws ProcessorInstantiationException if unable to instantiate a
|
||||
* processor
|
||||
*/
|
||||
public void instantiateSnippet(final ProcessGroup group, final FlowSnippetDTO dto) throws ProcessorInstantiationException {
|
||||
writeLock.lock();
|
||||
|
@ -1866,7 +1915,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the parent of the specified Connectable. This only considers this group and any direct child sub groups.
|
||||
* Returns the parent of the specified Connectable. This only considers this
|
||||
* group and any direct child sub groups.
|
||||
*
|
||||
* @param parentGroupId group id
|
||||
* @return parent group
|
||||
|
@ -1884,16 +1934,20 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
* Verifies that the given DTO is valid, according to the following:
|
||||
*
|
||||
* <ul>
|
||||
* <li>None of the ID's in any component of the DTO can be used in this flow.</li>
|
||||
* <li>The ProcessGroup to which the template's contents will be added must not contain any InputPort or OutputPort with the same name as one of the corresponding components in the root level of
|
||||
* the template.</li>
|
||||
* <li>None of the ID's in any component of the DTO can be used in this
|
||||
* flow.</li>
|
||||
* <li>The ProcessGroup to which the template's contents will be added must
|
||||
* not contain any InputPort or OutputPort with the same name as one of the
|
||||
* corresponding components in the root level of the template.</li>
|
||||
* <li>All Processors' classes must exist in this instance.</li>
|
||||
* <li>All Flow File Prioritizers' classes must exist in this instance.</li>
|
||||
* </ul>
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* If any of the above statements does not hold true, an {@link IllegalStateException} or a {@link ProcessorInstantiationException} will be thrown.
|
||||
* If any of the above statements does not hold true, an
|
||||
* {@link IllegalStateException} or a
|
||||
* {@link ProcessorInstantiationException} will be thrown.
|
||||
* </p>
|
||||
*
|
||||
* @param group group
|
||||
|
@ -2004,7 +2058,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
// Processor access
|
||||
//
|
||||
/**
|
||||
* Indicates whether or not the two ID's point to the same ProcessGroup. If either id is null, will return <code>false</code>.
|
||||
* Indicates whether or not the two ID's point to the same ProcessGroup. If
|
||||
* either id is null, will return <code>false</code>.
|
||||
*
|
||||
* @param id1 group id
|
||||
* @param id2 other group id
|
||||
|
@ -2156,7 +2211,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the status of all components in the controller. This request is not in the context of a user so the results will be unfiltered.
|
||||
* Returns the status of all components in the controller. This request is
|
||||
* not in the context of a user so the results will be unfiltered.
|
||||
*
|
||||
* @return the component status
|
||||
*/
|
||||
|
@ -2166,7 +2222,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the status of all components in the specified group. This request is not in the context of a user so the results will be unfiltered.
|
||||
* Returns the status of all components in the specified group. This request
|
||||
* is not in the context of a user so the results will be unfiltered.
|
||||
*
|
||||
* @param groupId group id
|
||||
* @return the component status
|
||||
|
@ -2176,7 +2233,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the status for components in the specified group. This request is made by the specified user so the results will be filtered accordingly.
|
||||
* Returns the status for components in the specified group. This request is
|
||||
* made by the specified user so the results will be filtered accordingly.
|
||||
*
|
||||
* @param groupId group id
|
||||
* @param user user making request
|
||||
|
@ -2187,8 +2245,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the status for the components in the specified group with the specified report. This request is not in the context of a user so the results
|
||||
* will be unfiltered.
|
||||
* Returns the status for the components in the specified group with the
|
||||
* specified report. This request is not in the context of a user so the
|
||||
* results will be unfiltered.
|
||||
*
|
||||
* @param groupId group id
|
||||
* @param statusReport report
|
||||
|
@ -2202,8 +2261,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the status for the components in the specified group with the specified report. This request is made by the specified user
|
||||
* so the results will be filtered accordingly.
|
||||
* Returns the status for the components in the specified group with the
|
||||
* specified report. This request is made by the specified user so the
|
||||
* results will be filtered accordingly.
|
||||
*
|
||||
* @param groupId group id
|
||||
* @param statusReport report
|
||||
|
@ -2218,8 +2278,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the status for the components in the specified group with the specified report. The results will be filtered by executing
|
||||
* the specified predicate.
|
||||
* Returns the status for the components in the specified group with the
|
||||
* specified report. The results will be filtered by executing the specified
|
||||
* predicate.
|
||||
*
|
||||
* @param group group id
|
||||
* @param statusReport report
|
||||
|
@ -2795,7 +2856,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
}
|
||||
|
||||
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider,variableRegistry);
|
||||
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, variableRegistry);
|
||||
final ReportingTaskNode taskNode;
|
||||
if (creationSuccessful) {
|
||||
taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, variableRegistry);
|
||||
|
@ -2803,7 +2864,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
final String simpleClassName = type.contains(".") ? StringUtils.substringAfterLast(type, ".") : type;
|
||||
final String componentType = "(Missing) " + simpleClassName;
|
||||
|
||||
taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, componentType, type,variableRegistry);
|
||||
taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, componentType, type, variableRegistry);
|
||||
}
|
||||
|
||||
taskNode.setName(task.getClass().getSimpleName());
|
||||
|
@ -2811,7 +2872,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
if (firstTimeAdded) {
|
||||
final ComponentLog componentLog = new SimpleProcessLogger(id, taskNode.getReportingTask());
|
||||
final ReportingInitializationContext config = new StandardReportingInitializationContext(id, taskNode.getName(),
|
||||
SchedulingStrategy.TIMER_DRIVEN, "1 min", componentLog, this);
|
||||
SchedulingStrategy.TIMER_DRIVEN, "1 min", componentLog, this);
|
||||
|
||||
try {
|
||||
task.initialize(config);
|
||||
|
@ -2832,7 +2893,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
// Register log observer to provide bulletins when reporting task logs anything at WARN level or above
|
||||
final LogRepository logRepository = LogRepositoryFactory.getRepository(id);
|
||||
logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, LogLevel.WARN,
|
||||
new ReportingTaskLogObserver(getBulletinRepository(), taskNode));
|
||||
new ReportingTaskLogObserver(getBulletinRepository(), taskNode));
|
||||
|
||||
return taskNode;
|
||||
}
|
||||
|
@ -2903,7 +2964,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
// Register log observer to provide bulletins when reporting task logs anything at WARN level or above
|
||||
final LogRepository logRepository = LogRepositoryFactory.getRepository(id);
|
||||
logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, LogLevel.WARN,
|
||||
new ControllerServiceLogObserver(getBulletinRepository(), serviceNode));
|
||||
new ControllerServiceLogObserver(getBulletinRepository(), serviceNode));
|
||||
|
||||
if (firstTimeAdded) {
|
||||
final ControllerService service = serviceNode.getControllerServiceImplementation();
|
||||
|
@ -3028,7 +3089,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
service.verifyCanDelete();
|
||||
|
||||
try (final NarCloseable x = NarCloseable.withNarLoader()) {
|
||||
final ConfigurationContext configurationContext = new StandardConfigurationContext(service, controllerServiceProvider, null,variableRegistry);
|
||||
final ConfigurationContext configurationContext = new StandardConfigurationContext(service, controllerServiceProvider, null, variableRegistry);
|
||||
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, service.getControllerServiceImplementation(), configurationContext);
|
||||
}
|
||||
|
||||
|
@ -3139,7 +3200,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
// Clustering methods
|
||||
//
|
||||
/**
|
||||
* Starts heartbeating to the cluster. May only be called if the instance was constructed for a clustered environment.
|
||||
* Starts heartbeating to the cluster. May only be called if the instance
|
||||
* was constructed for a clustered environment.
|
||||
*
|
||||
* @throws IllegalStateException if not configured for clustering
|
||||
*/
|
||||
|
@ -3161,23 +3223,31 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* Notifies controller that the sending of heartbeats should be temporarily suspended. This method does not cancel any background tasks as does {@link #stopHeartbeating()} and does not require any
|
||||
* lock on the FlowController. Background tasks will still generate heartbeat messages and any background task currently in the process of sending a Heartbeat to the cluster will continue.
|
||||
* Notifies controller that the sending of heartbeats should be temporarily
|
||||
* suspended. This method does not cancel any background tasks as does
|
||||
* {@link #stopHeartbeating()} and does not require any lock on the
|
||||
* FlowController. Background tasks will still generate heartbeat messages
|
||||
* and any background task currently in the process of sending a Heartbeat
|
||||
* to the cluster will continue.
|
||||
*/
|
||||
public void suspendHeartbeats() {
|
||||
heartbeatsSuspended.set(true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Notifies controller that the sending of heartbeats should be re-enabled. This method does not submit any background tasks to take affect as does {@link #startHeartbeating()} and does not
|
||||
* require any lock on the FlowController.
|
||||
* Notifies controller that the sending of heartbeats should be re-enabled.
|
||||
* This method does not submit any background tasks to take affect as does
|
||||
* {@link #startHeartbeating()} and does not require any lock on the
|
||||
* FlowController.
|
||||
*/
|
||||
public void resumeHeartbeats() {
|
||||
heartbeatsSuspended.set(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Stops heartbeating to the cluster. May only be called if the instance was constructed for a clustered environment. If the controller was not heartbeating, then this method has no effect.
|
||||
* Stops heartbeating to the cluster. May only be called if the instance was
|
||||
* constructed for a clustered environment. If the controller was not
|
||||
* heartbeating, then this method has no effect.
|
||||
*
|
||||
* @throws IllegalStateException if not clustered
|
||||
*/
|
||||
|
@ -3244,7 +3314,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* @return true if this instance is clustered; false otherwise. Clustered means that a node is either connected or trying to connect to the cluster.
|
||||
* @return true if this instance is clustered; false otherwise. Clustered
|
||||
* means that a node is either connected or trying to connect to the
|
||||
* cluster.
|
||||
*/
|
||||
@Override
|
||||
public boolean isClustered() {
|
||||
|
@ -3261,8 +3333,10 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* @return the DN of the Cluster Manager that we are currently connected to, if available. This will return null if the instance is not clustered or if the instance is clustered but the NCM's DN
|
||||
* is not available - for instance, if cluster communications are not secure
|
||||
* @return the DN of the Cluster Manager that we are currently connected to,
|
||||
* if available. This will return null if the instance is not clustered or
|
||||
* if the instance is clustered but the NCM's DN is not available - for
|
||||
* instance, if cluster communications are not secure
|
||||
*/
|
||||
public String getClusterManagerDN() {
|
||||
readLock.lock();
|
||||
|
@ -3274,10 +3348,12 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* Sets whether this instance is clustered. Clustered means that a node is either connected or trying to connect to the cluster.
|
||||
* Sets whether this instance is clustered. Clustered means that a node is
|
||||
* either connected or trying to connect to the cluster.
|
||||
*
|
||||
* @param clustered true if clustered
|
||||
* @param clusterInstanceId if clustered is true, indicates the InstanceID of the Cluster Manager
|
||||
* @param clusterInstanceId if clustered is true, indicates the InstanceID
|
||||
* of the Cluster Manager
|
||||
*/
|
||||
public void setClustered(final boolean clustered, final String clusterInstanceId) {
|
||||
setClustered(clustered, clusterInstanceId, null);
|
||||
|
@ -3320,10 +3396,12 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* Sets whether this instance is clustered. Clustered means that a node is either connected or trying to connect to the cluster.
|
||||
* Sets whether this instance is clustered. Clustered means that a node is
|
||||
* either connected or trying to connect to the cluster.
|
||||
*
|
||||
* @param clustered true if clustered
|
||||
* @param clusterInstanceId if clustered is true, indicates the InstanceID of the Cluster Manager
|
||||
* @param clusterInstanceId if clustered is true, indicates the InstanceID
|
||||
* of the Cluster Manager
|
||||
* @param clusterManagerDn the DN of the NCM
|
||||
*/
|
||||
public void setClustered(final boolean clustered, final String clusterInstanceId, final String clusterManagerDn) {
|
||||
|
@ -3387,7 +3465,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
/**
|
||||
* @return true if this instance is the primary node in the cluster; false otherwise
|
||||
* @return true if this instance is the primary node in the cluster; false
|
||||
* otherwise
|
||||
*/
|
||||
@Override
|
||||
public boolean isPrimary() {
|
||||
|
@ -3469,17 +3548,17 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
@Override
|
||||
public boolean isContentSame() {
|
||||
return areEqual(event.getPreviousContentClaimContainer(), event.getContentClaimContainer())
|
||||
&& areEqual(event.getPreviousContentClaimSection(), event.getContentClaimSection())
|
||||
&& areEqual(event.getPreviousContentClaimIdentifier(), event.getContentClaimIdentifier())
|
||||
&& areEqual(event.getPreviousContentClaimOffset(), event.getContentClaimOffset())
|
||||
&& areEqual(event.getPreviousFileSize(), event.getFileSize());
|
||||
&& areEqual(event.getPreviousContentClaimSection(), event.getContentClaimSection())
|
||||
&& areEqual(event.getPreviousContentClaimIdentifier(), event.getContentClaimIdentifier())
|
||||
&& areEqual(event.getPreviousContentClaimOffset(), event.getContentClaimOffset())
|
||||
&& areEqual(event.getPreviousFileSize(), event.getFileSize());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isInputAvailable() {
|
||||
try {
|
||||
return contentRepository.isAccessible(createClaim(event.getPreviousContentClaimContainer(), event.getPreviousContentClaimSection(),
|
||||
event.getPreviousContentClaimIdentifier(), event.getPreviousContentClaimOffset()));
|
||||
event.getPreviousContentClaimIdentifier(), event.getPreviousContentClaimOffset()));
|
||||
} catch (final IOException e) {
|
||||
return false;
|
||||
}
|
||||
|
@ -3489,7 +3568,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
public boolean isOutputAvailable() {
|
||||
try {
|
||||
return contentRepository.isAccessible(createClaim(event.getContentClaimContainer(), event.getContentClaimSection(),
|
||||
event.getContentClaimIdentifier(), event.getContentClaimOffset()));
|
||||
event.getContentClaimIdentifier(), event.getContentClaimOffset()));
|
||||
} catch (final IOException e) {
|
||||
return false;
|
||||
}
|
||||
|
@ -3526,7 +3605,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
final ResourceClaim resourceClaim = resourceClaimManager.newResourceClaim(provEvent.getPreviousContentClaimContainer(), provEvent.getPreviousContentClaimSection(),
|
||||
provEvent.getPreviousContentClaimIdentifier(), false);
|
||||
provEvent.getPreviousContentClaimIdentifier(), false);
|
||||
claim = new StandardContentClaim(resourceClaim, provEvent.getPreviousContentClaimOffset());
|
||||
offset = provEvent.getPreviousContentClaimOffset() == null ? 0L : provEvent.getPreviousContentClaimOffset();
|
||||
size = provEvent.getPreviousFileSize();
|
||||
|
@ -3536,7 +3615,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
final ResourceClaim resourceClaim = resourceClaimManager.newResourceClaim(provEvent.getContentClaimContainer(), provEvent.getContentClaimSection(),
|
||||
provEvent.getContentClaimIdentifier(), false);
|
||||
provEvent.getContentClaimIdentifier(), false);
|
||||
|
||||
claim = new StandardContentClaim(resourceClaim, provEvent.getContentClaimOffset());
|
||||
offset = provEvent.getContentClaimOffset() == null ? 0L : provEvent.getContentClaimOffset();
|
||||
|
@ -3548,18 +3627,18 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
|
||||
// Register a Provenance Event to indicate that we replayed the data.
|
||||
final ProvenanceEventRecord sendEvent = new StandardProvenanceEventRecord.Builder()
|
||||
.setEventType(ProvenanceEventType.DOWNLOAD)
|
||||
.setFlowFileUUID(provEvent.getFlowFileUuid())
|
||||
.setAttributes(provEvent.getAttributes(), Collections.<String, String> emptyMap())
|
||||
.setCurrentContentClaim(resourceClaim.getContainer(), resourceClaim.getSection(), resourceClaim.getId(), offset, size)
|
||||
.setTransitUri(requestUri)
|
||||
.setEventTime(System.currentTimeMillis())
|
||||
.setFlowFileEntryDate(provEvent.getFlowFileEntryDate())
|
||||
.setLineageStartDate(provEvent.getLineageStartDate())
|
||||
.setComponentType(getName())
|
||||
.setComponentId(getRootGroupId())
|
||||
.setDetails("Download of " + (direction == ContentDirection.INPUT ? "Input" : "Output") + " Content requested by " + requestor + " for Provenance Event " + provEvent.getEventId())
|
||||
.build();
|
||||
.setEventType(ProvenanceEventType.DOWNLOAD)
|
||||
.setFlowFileUUID(provEvent.getFlowFileUuid())
|
||||
.setAttributes(provEvent.getAttributes(), Collections.<String, String>emptyMap())
|
||||
.setCurrentContentClaim(resourceClaim.getContainer(), resourceClaim.getSection(), resourceClaim.getId(), offset, size)
|
||||
.setTransitUri(requestUri)
|
||||
.setEventTime(System.currentTimeMillis())
|
||||
.setFlowFileEntryDate(provEvent.getFlowFileEntryDate())
|
||||
.setLineageStartDate(provEvent.getLineageStartDate())
|
||||
.setComponentType(getName())
|
||||
.setComponentId(getRootGroupId())
|
||||
.setDetails("Download of " + (direction == ContentDirection.INPUT ? "Input" : "Output") + " Content requested by " + requestor + " for Provenance Event " + provEvent.getEventId())
|
||||
.build();
|
||||
|
||||
provenanceRepository.registerEvent(sendEvent);
|
||||
|
||||
|
@ -3590,20 +3669,20 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
|
||||
// Register a Provenance Event to indicate that we replayed the data.
|
||||
final StandardProvenanceEventRecord.Builder sendEventBuilder = new StandardProvenanceEventRecord.Builder()
|
||||
.setEventType(ProvenanceEventType.DOWNLOAD)
|
||||
.setFlowFileUUID(flowFile.getAttribute(CoreAttributes.UUID.key()))
|
||||
.setAttributes(flowFile.getAttributes(), Collections.<String, String> emptyMap())
|
||||
.setTransitUri(requestUri)
|
||||
.setEventTime(System.currentTimeMillis())
|
||||
.setFlowFileEntryDate(flowFile.getEntryDate())
|
||||
.setLineageStartDate(flowFile.getLineageStartDate())
|
||||
.setComponentType(getName())
|
||||
.setComponentId(getRootGroupId())
|
||||
.setDetails("Download of Content requested by " + requestor + " for " + flowFile);
|
||||
.setEventType(ProvenanceEventType.DOWNLOAD)
|
||||
.setFlowFileUUID(flowFile.getAttribute(CoreAttributes.UUID.key()))
|
||||
.setAttributes(flowFile.getAttributes(), Collections.<String, String>emptyMap())
|
||||
.setTransitUri(requestUri)
|
||||
.setEventTime(System.currentTimeMillis())
|
||||
.setFlowFileEntryDate(flowFile.getEntryDate())
|
||||
.setLineageStartDate(flowFile.getLineageStartDate())
|
||||
.setComponentType(getName())
|
||||
.setComponentId(getRootGroupId())
|
||||
.setDetails("Download of Content requested by " + requestor + " for " + flowFile);
|
||||
|
||||
if (contentClaim != null) {
|
||||
sendEventBuilder.setCurrentContentClaim(resourceClaim.getContainer(), resourceClaim.getSection(), resourceClaim.getId(),
|
||||
contentClaim.getOffset() + flowFile.getContentClaimOffset(), flowFile.getSize());
|
||||
contentClaim.getOffset() + flowFile.getContentClaimOffset(), flowFile.getSize());
|
||||
}
|
||||
|
||||
final ProvenanceEventRecord sendEvent = sendEventBuilder.build();
|
||||
|
@ -3710,7 +3789,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
|
||||
// Create the ContentClaim
|
||||
final ResourceClaim resourceClaim = resourceClaimManager.newResourceClaim(event.getPreviousContentClaimContainer(),
|
||||
event.getPreviousContentClaimSection(), event.getPreviousContentClaimIdentifier(), false);
|
||||
event.getPreviousContentClaimSection(), event.getPreviousContentClaimIdentifier(), false);
|
||||
|
||||
// Increment Claimant Count, since we will now be referencing the Content Claim
|
||||
resourceClaimManager.incrementClaimantCount(resourceClaim);
|
||||
|
@ -3737,22 +3816,22 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
// To avoid this, we just always set the offset in the Content Claim itself and set the
|
||||
// FlowFileRecord's contentClaimOffset to 0.
|
||||
final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
|
||||
// Copy relevant info from source FlowFile
|
||||
.addAttributes(event.getPreviousAttributes())
|
||||
.contentClaim(contentClaim)
|
||||
.contentClaimOffset(0L) // use 0 because we used the content claim offset in the Content Claim itself
|
||||
.entryDate(System.currentTimeMillis())
|
||||
.id(flowFileRepository.getNextFlowFileSequence())
|
||||
.lineageStart(event.getLineageStartDate(), 0L)
|
||||
.size(contentSize.longValue())
|
||||
// Create a new UUID and add attributes indicating that this is a replay
|
||||
.addAttribute("flowfile.replay", "true")
|
||||
.addAttribute("flowfile.replay.timestamp", String.valueOf(new Date()))
|
||||
.addAttribute(CoreAttributes.UUID.key(), newFlowFileUUID)
|
||||
// remove attributes that may have existed on the source FlowFile that we don't want to exist on the new FlowFile
|
||||
.removeAttributes(CoreAttributes.DISCARD_REASON.key(), CoreAttributes.ALTERNATE_IDENTIFIER.key())
|
||||
// build the record
|
||||
.build();
|
||||
// Copy relevant info from source FlowFile
|
||||
.addAttributes(event.getPreviousAttributes())
|
||||
.contentClaim(contentClaim)
|
||||
.contentClaimOffset(0L) // use 0 because we used the content claim offset in the Content Claim itself
|
||||
.entryDate(System.currentTimeMillis())
|
||||
.id(flowFileRepository.getNextFlowFileSequence())
|
||||
.lineageStart(event.getLineageStartDate(), 0L)
|
||||
.size(contentSize.longValue())
|
||||
// Create a new UUID and add attributes indicating that this is a replay
|
||||
.addAttribute("flowfile.replay", "true")
|
||||
.addAttribute("flowfile.replay.timestamp", String.valueOf(new Date()))
|
||||
.addAttribute(CoreAttributes.UUID.key(), newFlowFileUUID)
|
||||
// remove attributes that may have existed on the source FlowFile that we don't want to exist on the new FlowFile
|
||||
.removeAttributes(CoreAttributes.DISCARD_REASON.key(), CoreAttributes.ALTERNATE_IDENTIFIER.key())
|
||||
// build the record
|
||||
.build();
|
||||
|
||||
// Register a Provenance Event to indicate that we replayed the data.
|
||||
final ProvenanceEventRecord replayEvent = new StandardProvenanceEventRecord.Builder()
|
||||
|
@ -3774,7 +3853,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
// Update the FlowFile Repository to indicate that we have added the FlowFile to the flow
|
||||
final StandardRepositoryRecord record = new StandardRepositoryRecord(queue, flowFileRecord);
|
||||
record.setDestination(queue);
|
||||
flowFileRepository.updateRepository(Collections.<RepositoryRecord> singleton(record));
|
||||
flowFileRepository.updateRepository(Collections.<RepositoryRecord>singleton(record));
|
||||
|
||||
// Enqueue the data
|
||||
queue.put(flowFileRecord);
|
||||
|
@ -3817,8 +3896,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
private class HeartbeatSendTask implements Runnable {
|
||||
|
||||
private final DateFormat dateFormatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS", Locale.US);
|
||||
|
||||
@Override
|
||||
|
@ -3848,10 +3927,10 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
}
|
||||
|
||||
heartbeatLogger.info("Heartbeat created at {} and sent to {} at {}; send took {} millis",
|
||||
dateFormatter.format(new Date(message.getHeartbeat().getCreatedTimestamp())),
|
||||
heartbeatAddress,
|
||||
dateFormatter.format(new Date()),
|
||||
sendMillis);
|
||||
dateFormatter.format(new Date(message.getHeartbeat().getCreatedTimestamp())),
|
||||
heartbeatAddress,
|
||||
dateFormatter.format(new Date()),
|
||||
sendMillis);
|
||||
} catch (final UnknownServiceAddressException usae) {
|
||||
if (heartbeatLogger.isDebugEnabled()) {
|
||||
heartbeatLogger.debug(usae.getMessage());
|
||||
|
@ -4028,8 +4107,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
return queues;
|
||||
}
|
||||
|
||||
|
||||
private static class HeartbeatBean {
|
||||
|
||||
private final ProcessGroup rootGroup;
|
||||
private final boolean primary;
|
||||
|
||||
|
|
|
@ -0,0 +1,88 @@
|
|||
/*
|
||||
* 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.util;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import org.apache.nifi.registry.VariableDescriptor;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* A file based variable registry that loads all properties from files specified
|
||||
* during construction and is backed by system properties and environment
|
||||
* variables accessible to the JVM.
|
||||
*/
|
||||
public class FileBasedVariableRegistry implements VariableRegistry {
|
||||
|
||||
private final static Logger LOG = LoggerFactory.getLogger(FileBasedVariableRegistry.class);
|
||||
final Map<VariableDescriptor, String> map;
|
||||
|
||||
public FileBasedVariableRegistry(final Path[] propertiesPaths) {
|
||||
final Map<VariableDescriptor, String> newMap = new HashMap<>(VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY.getVariableMap());
|
||||
final int systemEnvPropCount = newMap.size();
|
||||
int totalPropertiesLoaded = systemEnvPropCount;
|
||||
LOG.info("Loaded {} properties from system properties and environment variables",systemEnvPropCount);
|
||||
try {
|
||||
for (final Path path : propertiesPaths) {
|
||||
if (Files.exists(path)) {
|
||||
final AtomicInteger propsLoaded = new AtomicInteger(0);
|
||||
try (final InputStream inStream = new BufferedInputStream(new FileInputStream(path.toFile()))) {
|
||||
Properties properties = new Properties();
|
||||
properties.load(inStream);
|
||||
properties.entrySet().stream().forEach((entry) -> {
|
||||
final VariableDescriptor desc = new VariableDescriptor.Builder(entry.getKey().toString())
|
||||
.description(path.toString())
|
||||
.sensitive(false)
|
||||
.build();
|
||||
newMap.put(desc, entry.getValue().toString());
|
||||
propsLoaded.incrementAndGet();
|
||||
});
|
||||
}
|
||||
totalPropertiesLoaded += propsLoaded.get();
|
||||
if(propsLoaded.get() > 0){
|
||||
LOG.info("Loaded {} properties from '{}'", propsLoaded.get(), path);
|
||||
}else{
|
||||
LOG.warn("No properties loaded from '{}'", path);
|
||||
}
|
||||
} else {
|
||||
LOG.warn("Skipping property file {} as it does not appear to exist", path);
|
||||
}
|
||||
}
|
||||
} catch (final IOException ioe) {
|
||||
LOG.error("Unable to complete variable registry loading from files due to ", ioe);
|
||||
}
|
||||
LOG.info("Loaded a total of {} properties. Including precedence overrides effective accessible registry key size is {}", totalPropertiesLoaded, newMap.size());
|
||||
map = newMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<VariableDescriptor, String> getVariableMap() {
|
||||
return Collections.unmodifiableMap(map);
|
||||
}
|
||||
|
||||
}
|
|
@ -28,7 +28,7 @@
|
|||
<bean id="nifiProperties" class="org.apache.nifi.util.NiFiProperties" factory-method="getInstance"/>
|
||||
|
||||
<!-- variable registry -->
|
||||
<bean id="variableRegistry" class="org.apache.nifi.registry.VariableRegistryUtils" factory-method="createCustomVariableRegistry">
|
||||
<bean id="variableRegistry" class="org.apache.nifi.util.FileBasedVariableRegistry">
|
||||
<constructor-arg type="java.nio.file.Path[]" value="#{nifiProperties.getVariableRegistryPropertiesPaths()}" />
|
||||
</bean>
|
||||
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.apache.nifi.controller.serialization.StandardFlowSerializer;
|
|||
import org.apache.nifi.encrypt.StringEncryptor;
|
||||
import org.apache.nifi.events.VolatileBulletinRepository;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.nifi.web.api.dto.ConnectableDTO;
|
||||
import org.apache.nifi.web.api.dto.ConnectionDTO;
|
||||
|
@ -48,6 +47,7 @@ import java.io.ByteArrayOutputStream;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import org.apache.nifi.util.FileBasedVariableRegistry;
|
||||
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
@ -75,7 +75,7 @@ public class StandardFlowServiceTest {
|
|||
@Before
|
||||
public void setup() throws Exception {
|
||||
properties = NiFiProperties.getInstance();
|
||||
variableRegistry = VariableRegistryUtils.createCustomVariableRegistry(properties.getVariableRegistryPropertiesPaths());
|
||||
variableRegistry = new FileBasedVariableRegistry(properties.getVariableRegistryPropertiesPaths());
|
||||
mockFlowFileEventRepository = mock(FlowFileEventRepository.class);
|
||||
authorizer = mock(Authorizer.class);
|
||||
mockAuditService = mock(AuditService.class);
|
||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.nifi.encrypt.StringEncryptor;
|
|||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.provenance.MockProvenanceRepository;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.apache.nifi.reporting.BulletinRepository;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.junit.After;
|
||||
|
@ -45,6 +44,7 @@ import org.mockito.Mockito;
|
|||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.Set;
|
||||
import org.apache.nifi.util.FileBasedVariableRegistry;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
|
@ -114,7 +114,7 @@ public class TestFlowController {
|
|||
policies1.add(policy2);
|
||||
|
||||
authorizer = new MockPolicyBasedAuthorizer(groups1, users1, policies1);
|
||||
variableRegistry = VariableRegistryUtils.createCustomVariableRegistry(properties.getVariableRegistryPropertiesPaths());
|
||||
variableRegistry = new FileBasedVariableRegistry(properties.getVariableRegistryPropertiesPaths());
|
||||
|
||||
bulletinRepo = Mockito.mock(BulletinRepository.class);
|
||||
controller = FlowController.createStandaloneInstance(flowFileEventRepo, properties, authorizer, auditService, encryptor, bulletinRepo,variableRegistry);
|
||||
|
|
|
@ -43,7 +43,6 @@ import org.apache.nifi.processor.ProcessSession;
|
|||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.provenance.MockProvenanceRepository;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
@ -66,6 +65,7 @@ import java.util.concurrent.ExecutorService;
|
|||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.locks.LockSupport;
|
||||
import org.apache.nifi.util.FileBasedVariableRegistry;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
@ -663,7 +663,7 @@ public class TestProcessorLifecycle {
|
|||
|
||||
return FlowController.createStandaloneInstance(mock(FlowFileEventRepository.class), properties,
|
||||
mock(Authorizer.class), mock(AuditService.class), null, new VolatileBulletinRepository(),
|
||||
VariableRegistryUtils.createCustomVariableRegistry(properties.getVariableRegistryPropertiesPaths()));
|
||||
new FileBasedVariableRegistry(properties.getVariableRegistryPropertiesPaths()));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -62,7 +62,6 @@ import org.apache.nifi.processor.Processor;
|
|||
import org.apache.nifi.processor.StandardValidationContextFactory;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.apache.nifi.reporting.AbstractReportingTask;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.apache.nifi.reporting.ReportingContext;
|
||||
|
@ -78,7 +77,7 @@ public class TestStandardProcessScheduler {
|
|||
private ReportingTaskNode taskNode = null;
|
||||
private TestReportingTask reportingTask = null;
|
||||
private final StateManagerProvider stateMgrProvider = Mockito.mock(StateManagerProvider.class);
|
||||
private VariableRegistry variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
private VariableRegistry variableRegistry = VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY;
|
||||
private FlowController controller;
|
||||
private ProcessGroup rootGroup;
|
||||
|
||||
|
|
|
@ -23,8 +23,8 @@ import org.apache.nifi.controller.StandardFlowServiceTest;
|
|||
import org.apache.nifi.nar.ExtensionManager;
|
||||
import org.apache.nifi.nar.NarClassLoaders;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.apache.nifi.util.FileBasedVariableRegistry;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
|
@ -43,7 +43,7 @@ public class StandardControllerServiceProviderTest {
|
|||
NiFiProperties properties = NiFiProperties.getInstance();
|
||||
NarClassLoaders.getInstance().init(properties.getFrameworkWorkingDirectory(), properties.getExtensionsWorkingDirectory());
|
||||
ExtensionManager.discoverExtensions(NarClassLoaders.getInstance().getExtensionClassLoaders());
|
||||
variableRegistry = VariableRegistryUtils.createCustomVariableRegistry(properties.getVariableRegistryPropertiesPaths());
|
||||
variableRegistry = new FileBasedVariableRegistry(properties.getVariableRegistryPropertiesPaths());
|
||||
}
|
||||
|
||||
@Before
|
||||
|
|
|
@ -47,7 +47,6 @@ import org.apache.nifi.groups.ProcessGroup;
|
|||
import org.apache.nifi.groups.StandardProcessGroup;
|
||||
import org.apache.nifi.processor.StandardValidationContextFactory;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
@ -77,7 +76,7 @@ public class TestStandardControllerServiceProvider {
|
|||
}
|
||||
};
|
||||
|
||||
private static VariableRegistry variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
private static VariableRegistry variableRegistry = VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY;
|
||||
|
||||
@BeforeClass
|
||||
public static void setNiFiProps() {
|
||||
|
|
|
@ -32,8 +32,6 @@ import org.apache.nifi.components.state.StateProvider;
|
|||
import org.apache.nifi.components.state.StateProviderInitializationContext;
|
||||
import org.apache.nifi.controller.state.StateMapUpdate;
|
||||
import org.apache.nifi.controller.state.providers.AbstractTestStateProvider;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.wali.WriteAheadRepository;
|
||||
|
@ -45,9 +43,8 @@ public class TestWriteAheadLocalStateProvider extends AbstractTestStateProvider
|
|||
@Before
|
||||
public void setup() throws IOException {
|
||||
provider = new WriteAheadLocalStateProvider();
|
||||
final VariableRegistry variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
final Map<PropertyDescriptor, PropertyValue> properties = new HashMap<>();
|
||||
properties.put(WriteAheadLocalStateProvider.PATH, new StandardPropertyValue("target/local-state-provider/" + UUID.randomUUID().toString(), null, variableRegistry));
|
||||
properties.put(WriteAheadLocalStateProvider.PATH, new StandardPropertyValue("target/local-state-provider/" + UUID.randomUUID().toString(), null));
|
||||
|
||||
provider.initialize(new StateProviderInitializationContext() {
|
||||
@Override
|
||||
|
@ -64,7 +61,7 @@ public class TestWriteAheadLocalStateProvider extends AbstractTestStateProvider
|
|||
public PropertyValue getProperty(final PropertyDescriptor property) {
|
||||
final PropertyValue prop = properties.get(property);
|
||||
if (prop == null) {
|
||||
return new StandardPropertyValue(null, null, variableRegistry);
|
||||
return new StandardPropertyValue(null, null);
|
||||
}
|
||||
return prop;
|
||||
}
|
||||
|
|
|
@ -31,8 +31,6 @@ import org.apache.nifi.components.state.StateProvider;
|
|||
import org.apache.nifi.components.state.StateProviderInitializationContext;
|
||||
import org.apache.nifi.components.state.exception.StateTooLargeException;
|
||||
import org.apache.nifi.controller.state.providers.AbstractTestStateProvider;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
@ -42,7 +40,6 @@ public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
|
|||
|
||||
private StateProvider provider;
|
||||
private TestingServer zkServer;
|
||||
private VariableRegistry variableRegistry;
|
||||
|
||||
private static final Map<PropertyDescriptor, String> defaultProperties = new HashMap<>();
|
||||
|
||||
|
@ -61,7 +58,6 @@ public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
|
|||
final Map<PropertyDescriptor, String> properties = new HashMap<>(defaultProperties);
|
||||
properties.put(ZooKeeperStateProvider.CONNECTION_STRING, zkServer.getConnectString());
|
||||
this.provider = createProvider(properties);
|
||||
variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
}
|
||||
|
||||
private void initializeProvider(final ZooKeeperStateProvider provider, final Map<PropertyDescriptor, String> properties) throws IOException {
|
||||
|
@ -75,7 +71,7 @@ public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
|
|||
public Map<PropertyDescriptor, PropertyValue> getProperties() {
|
||||
final Map<PropertyDescriptor, PropertyValue> propValueMap = new HashMap<>();
|
||||
for (final Map.Entry<PropertyDescriptor, String> entry : properties.entrySet()) {
|
||||
propValueMap.put(entry.getKey(), new StandardPropertyValue(entry.getValue(), null, variableRegistry));
|
||||
propValueMap.put(entry.getKey(), new StandardPropertyValue(entry.getValue(), null));
|
||||
}
|
||||
return propValueMap;
|
||||
}
|
||||
|
@ -83,7 +79,7 @@ public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
|
|||
@Override
|
||||
public PropertyValue getProperty(final PropertyDescriptor property) {
|
||||
final String prop = properties.get(property);
|
||||
return new StandardPropertyValue(prop, null, variableRegistry);
|
||||
return new StandardPropertyValue(prop, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -29,19 +29,16 @@ import org.apache.nifi.controller.ControllerService;
|
|||
import org.apache.nifi.controller.ControllerServiceLookup;
|
||||
import org.apache.nifi.controller.repository.StandardFlowFileRecord;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.junit.Test;
|
||||
|
||||
|
||||
public class TestStandardPropertyValue {
|
||||
|
||||
private final ControllerServiceLookup lookup = new TestControllerServiceLookup();
|
||||
private final VariableRegistry variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
|
||||
@Test
|
||||
public void testSubstituteAttributesWithOneMatchingArg() {
|
||||
final PropertyValue value = new StandardPropertyValue("Hello, ${audience}!", lookup, variableRegistry);
|
||||
final PropertyValue value = new StandardPropertyValue("Hello, ${audience}!", lookup);
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("audience", "World");
|
||||
assertEquals("Hello, World!", value.evaluateAttributeExpressions(createFlowFile(attributes)).getValue());
|
||||
|
@ -49,7 +46,7 @@ public class TestStandardPropertyValue {
|
|||
|
||||
@Test
|
||||
public void testMissingEndBraceEvaluatesToStringLiteral() {
|
||||
final PropertyValue value = new StandardPropertyValue("Hello, ${audience!", lookup, variableRegistry);
|
||||
final PropertyValue value = new StandardPropertyValue("Hello, ${audience!", lookup);
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("audience", "World");
|
||||
assertEquals("Hello, ${audience!", value.evaluateAttributeExpressions(createFlowFile(attributes)).getValue());
|
||||
|
@ -57,7 +54,7 @@ public class TestStandardPropertyValue {
|
|||
|
||||
@Test
|
||||
public void testEscaped() {
|
||||
final PropertyValue value = new StandardPropertyValue("Hello, $${audience}!", lookup, variableRegistry);
|
||||
final PropertyValue value = new StandardPropertyValue("Hello, $${audience}!", lookup);
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("audience", "World");
|
||||
assertEquals("Hello, ${audience}!", value.evaluateAttributeExpressions(createFlowFile(attributes)).getValue());
|
||||
|
@ -65,7 +62,7 @@ public class TestStandardPropertyValue {
|
|||
|
||||
@Test
|
||||
public void testSubstituteAttributesWithMultipleMatchingArgs() {
|
||||
final PropertyValue value = new StandardPropertyValue("Hello, ${audience}${comma}${question}!", lookup, variableRegistry);
|
||||
final PropertyValue value = new StandardPropertyValue("Hello, ${audience}${comma}${question}!", lookup);
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("audience", "World");
|
||||
attributes.put("comma", ",");
|
||||
|
@ -75,14 +72,14 @@ public class TestStandardPropertyValue {
|
|||
|
||||
@Test
|
||||
public void testSubstituteAttributesWithNoMatch() {
|
||||
final PropertyValue value = new StandardPropertyValue("Hello, ${audience}${comma}${question:replaceNull('')}!", lookup, variableRegistry);
|
||||
final PropertyValue value = new StandardPropertyValue("Hello, ${audience}${comma}${question:replaceNull('')}!", lookup);
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
assertEquals("Hello, !", value.evaluateAttributeExpressions(createFlowFile(attributes)).getValue());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSubstituteAttributesRecursively() {
|
||||
final PropertyValue value = new StandardPropertyValue("Hello, ${'${a}${b}'}!", lookup, variableRegistry);
|
||||
final PropertyValue value = new StandardPropertyValue("Hello, ${'${a}${b}'}!", lookup);
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("a", "b");
|
||||
attributes.put("b", "World");
|
||||
|
@ -92,7 +89,7 @@ public class TestStandardPropertyValue {
|
|||
|
||||
@Test
|
||||
public void testGetValueAsIntegerAfterSubstitute() {
|
||||
final PropertyValue value = new StandardPropertyValue("1${value}", lookup, variableRegistry);
|
||||
final PropertyValue value = new StandardPropertyValue("1${value}", lookup);
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("value", "39");
|
||||
assertEquals(139, value.evaluateAttributeExpressions(createFlowFile(attributes)).asInteger().intValue());
|
||||
|
@ -100,7 +97,7 @@ public class TestStandardPropertyValue {
|
|||
|
||||
@Test(expected = NumberFormatException.class)
|
||||
public void testGetValueAsIntegerAfterSubstitutingWithNonInteger() {
|
||||
final PropertyValue value = new StandardPropertyValue("1${value}", lookup, variableRegistry);
|
||||
final PropertyValue value = new StandardPropertyValue("1${value}", lookup);
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("value", "Yes");
|
||||
final PropertyValue substituted = value.evaluateAttributeExpressions(createFlowFile(attributes));
|
||||
|
@ -109,7 +106,7 @@ public class TestStandardPropertyValue {
|
|||
|
||||
@Test
|
||||
public void testFileSize() {
|
||||
final PropertyValue value = new StandardPropertyValue("${fileSize}", lookup, variableRegistry);
|
||||
final PropertyValue value = new StandardPropertyValue("${fileSize}", lookup);
|
||||
final FlowFile flowFile = new StandardFlowFileRecord.Builder().size(1024 * 1024L).build();
|
||||
final long val = value.evaluateAttributeExpressions(flowFile).asLong().longValue();
|
||||
assertEquals(1024 * 1024L, val);
|
||||
|
@ -119,25 +116,12 @@ public class TestStandardPropertyValue {
|
|||
public void testFlowFileEntryYear() {
|
||||
final Calendar now = Calendar.getInstance();
|
||||
final int year = now.get(Calendar.YEAR);
|
||||
final PropertyValue value = new StandardPropertyValue("${entryDate:toNumber():toDate():format('yyyy')}", lookup, variableRegistry);
|
||||
final PropertyValue value = new StandardPropertyValue("${entryDate:toNumber():toDate():format('yyyy')}", lookup);
|
||||
final FlowFile flowFile = new StandardFlowFileRecord.Builder().entryDate(now.getTimeInMillis()).build();
|
||||
final int val = value.evaluateAttributeExpressions(flowFile).asInteger().intValue();
|
||||
assertEquals(year, val);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSystemProperty() {
|
||||
System.setProperty("Prop1", "Foo");
|
||||
System.setProperty("Prop2", "Bar");
|
||||
final PropertyValue value = new StandardPropertyValue("${Prop1}${Prop2}${abc}", lookup, VariableRegistryUtils.createSystemVariableRegistry());
|
||||
final Map<String, String> attributes = new HashMap<>();
|
||||
attributes.put("abc", "Baz");
|
||||
final FlowFile flowFile = createFlowFile(attributes);
|
||||
final String val = value.evaluateAttributeExpressions(flowFile).getValue();
|
||||
assertEquals("FooBarBaz", val);
|
||||
|
||||
}
|
||||
|
||||
private FlowFile createFlowFile(final Map<String, String> attributes) {
|
||||
return new StandardFlowFileRecord.Builder().addAttributes(attributes).build();
|
||||
}
|
||||
|
|
|
@ -0,0 +1,45 @@
|
|||
/*
|
||||
* 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.util;
|
||||
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Map;
|
||||
import org.apache.nifi.registry.VariableDescriptor;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.junit.Test;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class TestFileBasedVariableRegistry {
|
||||
|
||||
@Test
|
||||
public void testCreateCustomVariableRegistry() {
|
||||
final Path fooPath = Paths.get("src/test/resources/TestVariableRegistry/foobar.properties");
|
||||
final Path testPath = Paths.get("src/test/resources/TestVariableRegistry/test.properties");
|
||||
Path[] paths = {fooPath, testPath};
|
||||
final String vendorUrl = System.getProperty("java.vendor.url");
|
||||
VariableRegistry variableRegistry = new FileBasedVariableRegistry(paths);
|
||||
final Map<VariableDescriptor, String> variables = variableRegistry.getVariableMap();
|
||||
assertTrue(variables.containsKey(new VariableDescriptor("fake.property.3")));
|
||||
assertEquals(vendorUrl, variableRegistry.getVariableValue("java.vendor.url"));
|
||||
assertEquals("test me out 3, test me out 4", variableRegistry.getVariableValue("fake.property.3"));
|
||||
}
|
||||
}
|
|
@ -288,12 +288,6 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
|
|||
return componentFacade.updateComponent(requestContext, annotationData, properties);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public VariableRegistry getVariableRegistry() {
|
||||
return this.variableRegistry;
|
||||
}
|
||||
|
||||
private NodeResponse replicate(final String method, final URI uri, final Object entity, final Map<String, String> headers) throws InterruptedException {
|
||||
final NodeIdentifier coordinatorNode = clusterCoordinator.getElectedActiveCoordinatorNode();
|
||||
if (coordinatorNode == null) {
|
||||
|
|
|
@ -29,8 +29,6 @@ import org.apache.nifi.authorization.RequestAction;
|
|||
import org.apache.nifi.authorization.Resource;
|
||||
import org.apache.nifi.authorization.UserContextKeys;
|
||||
import org.apache.nifi.authorization.exception.AuthorizerCreationException;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.apache.nifi.util.MockPropertyValue;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.ranger.plugin.policyengine.RangerAccessRequest;
|
||||
|
@ -66,10 +64,9 @@ public class TestRangerNiFiAuthorizer {
|
|||
private RangerBasePluginWithPolicies rangerBasePlugin;
|
||||
private AuthorizerConfigurationContext configurationContext;
|
||||
private NiFiProperties nifiProperties;
|
||||
private VariableRegistry variableRegistry;
|
||||
|
||||
private String serviceType = "nifiService";
|
||||
private String appId = "nifiAppId";
|
||||
private final String serviceType = "nifiService";
|
||||
private final String appId = "nifiAppId";
|
||||
|
||||
private RangerAccessResult allowedResult;
|
||||
private RangerAccessResult notAllowedResult;
|
||||
|
@ -98,23 +95,22 @@ public class TestRangerNiFiAuthorizer {
|
|||
|
||||
notAllowedResult = Mockito.mock(RangerAccessResult.class);
|
||||
when(notAllowedResult.getIsAllowed()).thenReturn(false);
|
||||
variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
}
|
||||
|
||||
private AuthorizerConfigurationContext createMockConfigContext() {
|
||||
AuthorizerConfigurationContext configurationContext = Mockito.mock(AuthorizerConfigurationContext.class);
|
||||
|
||||
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_SECURITY_PATH_PROP)))
|
||||
.thenReturn(new MockPropertyValue("src/test/resources/ranger/ranger-nifi-security.xml", null,variableRegistry));
|
||||
.thenReturn(new MockPropertyValue("src/test/resources/ranger/ranger-nifi-security.xml"));
|
||||
|
||||
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_AUDIT_PATH_PROP)))
|
||||
.thenReturn(new MockPropertyValue("src/test/resources/ranger/ranger-nifi-audit.xml", null,variableRegistry));
|
||||
.thenReturn(new MockPropertyValue("src/test/resources/ranger/ranger-nifi-audit.xml"));
|
||||
|
||||
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_APP_ID_PROP)))
|
||||
.thenReturn(new MockPropertyValue(appId, null,variableRegistry));
|
||||
.thenReturn(new MockPropertyValue(appId));
|
||||
|
||||
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_SERVICE_TYPE_PROP)))
|
||||
.thenReturn(new MockPropertyValue(serviceType, null,variableRegistry));
|
||||
.thenReturn(new MockPropertyValue(serviceType));
|
||||
|
||||
return configurationContext;
|
||||
}
|
||||
|
@ -130,7 +126,7 @@ public class TestRangerNiFiAuthorizer {
|
|||
@Test
|
||||
public void testKerberosEnabledWithoutKeytab() {
|
||||
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_KERBEROS_ENABLED_PROP)))
|
||||
.thenReturn(new MockPropertyValue("true", null,variableRegistry));
|
||||
.thenReturn(new MockPropertyValue("true"));
|
||||
|
||||
nifiProperties = Mockito.mock(NiFiProperties.class);
|
||||
when(nifiProperties.getKerberosServicePrincipal()).thenReturn("");
|
||||
|
@ -150,7 +146,7 @@ public class TestRangerNiFiAuthorizer {
|
|||
@Test
|
||||
public void testKerberosEnabledWithoutPrincipal() {
|
||||
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_KERBEROS_ENABLED_PROP)))
|
||||
.thenReturn(new MockPropertyValue("true", null,variableRegistry));
|
||||
.thenReturn(new MockPropertyValue("true"));
|
||||
|
||||
nifiProperties = Mockito.mock(NiFiProperties.class);
|
||||
when(nifiProperties.getKerberosKeytabLocation()).thenReturn("");
|
||||
|
@ -170,7 +166,7 @@ public class TestRangerNiFiAuthorizer {
|
|||
@Test
|
||||
public void testKerberosEnabledWithoutKeytabOrPrincipal() {
|
||||
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_KERBEROS_ENABLED_PROP)))
|
||||
.thenReturn(new MockPropertyValue("true", null,variableRegistry));
|
||||
.thenReturn(new MockPropertyValue("true"));
|
||||
|
||||
nifiProperties = Mockito.mock(NiFiProperties.class);
|
||||
when(nifiProperties.getKerberosKeytabLocation()).thenReturn("");
|
||||
|
@ -204,7 +200,7 @@ public class TestRangerNiFiAuthorizer {
|
|||
@Test
|
||||
public void testKerberosEnabled() {
|
||||
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_KERBEROS_ENABLED_PROP)))
|
||||
.thenReturn(new MockPropertyValue("true", null,variableRegistry));
|
||||
.thenReturn(new MockPropertyValue("true"));
|
||||
|
||||
nifiProperties = Mockito.mock(NiFiProperties.class);
|
||||
when(nifiProperties.getKerberosKeytabLocation()).thenReturn("test");
|
||||
|
@ -402,7 +398,7 @@ public class TestRangerNiFiAuthorizer {
|
|||
|
||||
final String rangerAdminIdentity = "ranger-admin";
|
||||
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_ADMIN_IDENTITY_PROP)))
|
||||
.thenReturn(new MockPropertyValue(rangerAdminIdentity, null,variableRegistry));
|
||||
.thenReturn(new MockPropertyValue(rangerAdminIdentity));
|
||||
|
||||
rangerBasePlugin = Mockito.mock(RangerBasePluginWithPolicies.class);
|
||||
authorizer = new MockRangerNiFiAuthorizer(rangerBasePlugin);
|
||||
|
@ -450,10 +446,10 @@ public class TestRangerNiFiAuthorizer {
|
|||
final AuthorizerConfigurationContext configurationContext = Mockito.mock(AuthorizerConfigurationContext.class);
|
||||
|
||||
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_SECURITY_PATH_PROP)))
|
||||
.thenReturn(new MockPropertyValue("src/test/resources/ranger/ranger-nifi-security.xml", null,variableRegistry));
|
||||
.thenReturn(new MockPropertyValue("src/test/resources/ranger/ranger-nifi-security.xml"));
|
||||
|
||||
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_AUDIT_PATH_PROP)))
|
||||
.thenReturn(new MockPropertyValue("src/test/resources/ranger/ranger-nifi-audit.xml", null,variableRegistry));
|
||||
.thenReturn(new MockPropertyValue("src/test/resources/ranger/ranger-nifi-audit.xml"));
|
||||
|
||||
Authorizer authorizer = new RangerNiFiAuthorizer();
|
||||
try {
|
||||
|
@ -513,8 +509,8 @@ public class TestRangerNiFiAuthorizer {
|
|||
*/
|
||||
private static class MockResource implements Resource {
|
||||
|
||||
private String identifier;
|
||||
private String name;
|
||||
private final String identifier;
|
||||
private final String name;
|
||||
|
||||
public MockResource(String identifier, String name) {
|
||||
this.identifier = identifier;
|
||||
|
|
|
@ -18,8 +18,6 @@ package org.apache.nifi.processors.script;
|
|||
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.MockProcessContext;
|
||||
import org.apache.nifi.util.MockProcessorInitializationContext;
|
||||
|
@ -39,12 +37,9 @@ import static org.junit.Assert.assertTrue;
|
|||
|
||||
public class TestInvokeGroovy extends BaseScriptTest {
|
||||
|
||||
private VariableRegistry variableRegistry;
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
super.setupInvokeScriptProcessor();
|
||||
variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -77,7 +72,7 @@ public class TestInvokeGroovy extends BaseScriptTest {
|
|||
@Test
|
||||
public void testScriptDefinedAttribute() throws Exception {
|
||||
InvokeScriptedProcessor processor = new InvokeScriptedProcessor();
|
||||
MockProcessContext context = new MockProcessContext(processor, variableRegistry);
|
||||
MockProcessContext context = new MockProcessContext(processor);
|
||||
MockProcessorInitializationContext initContext = new MockProcessorInitializationContext(processor, context);
|
||||
|
||||
processor.initialize(initContext);
|
||||
|
@ -86,7 +81,7 @@ public class TestInvokeGroovy extends BaseScriptTest {
|
|||
context.setProperty(InvokeScriptedProcessor.SCRIPT_FILE, "target/test/resources/groovy/test_reader.groovy");
|
||||
context.setProperty(InvokeScriptedProcessor.MODULES, "target/test/resources/groovy");
|
||||
// State Manger is unused, and a null reference is specified
|
||||
processor.customValidate(new MockValidationContext(context, null, variableRegistry));
|
||||
processor.customValidate(new MockValidationContext(context));
|
||||
processor.setup(context);
|
||||
|
||||
List<PropertyDescriptor> descriptors = processor.getSupportedPropertyDescriptors();
|
||||
|
@ -111,7 +106,7 @@ public class TestInvokeGroovy extends BaseScriptTest {
|
|||
@Test
|
||||
public void testScriptDefinedRelationship() throws Exception {
|
||||
InvokeScriptedProcessor processor = new InvokeScriptedProcessor();
|
||||
MockProcessContext context = new MockProcessContext(processor, variableRegistry);
|
||||
MockProcessContext context = new MockProcessContext(processor);
|
||||
MockProcessorInitializationContext initContext = new MockProcessorInitializationContext(processor, context);
|
||||
|
||||
processor.initialize(initContext);
|
||||
|
@ -119,7 +114,7 @@ public class TestInvokeGroovy extends BaseScriptTest {
|
|||
context.setProperty(InvokeScriptedProcessor.SCRIPT_ENGINE, "Groovy");
|
||||
context.setProperty(InvokeScriptedProcessor.SCRIPT_FILE, "target/test/resources/groovy/test_reader.groovy");
|
||||
// State Manger is unused, and a null reference is specified
|
||||
processor.customValidate(new MockValidationContext(context, null, variableRegistry));
|
||||
processor.customValidate(new MockValidationContext(context));
|
||||
processor.setup(context);
|
||||
|
||||
Set<Relationship> relationships = processor.getRelationships();
|
||||
|
|
|
@ -28,7 +28,6 @@ import org.apache.nifi.provenance.ProvenanceEventRecord;
|
|||
import org.apache.nifi.provenance.ProvenanceEventRepository;
|
||||
import org.apache.nifi.provenance.ProvenanceEventType;
|
||||
import org.apache.nifi.provenance.StandardProvenanceEventRecord;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.apache.nifi.remote.Transaction;
|
||||
import org.apache.nifi.remote.TransferDirection;
|
||||
import org.apache.nifi.remote.client.SiteToSiteClient;
|
||||
|
@ -95,7 +94,7 @@ public class TestSiteToSiteProvenanceReportingTask {
|
|||
@Override
|
||||
public PropertyValue answer(final InvocationOnMock invocation) throws Throwable {
|
||||
final PropertyDescriptor descriptor = invocation.getArgumentAt(0, PropertyDescriptor.class);
|
||||
return new MockPropertyValue(properties.get(descriptor), null, VariableRegistryUtils.createSystemVariableRegistry());
|
||||
return new MockPropertyValue(properties.get(descriptor));
|
||||
}
|
||||
}).when(context).getProperty(Mockito.any(PropertyDescriptor.class));
|
||||
|
||||
|
|
|
@ -26,9 +26,6 @@ import java.util.Map;
|
|||
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.apache.nifi.state.MockStateManager;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.MockProcessContext;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
|
@ -38,12 +35,10 @@ import org.junit.Test;
|
|||
|
||||
public class TestRouteOnAttribute {
|
||||
|
||||
private VariableRegistry variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
|
||||
@Test
|
||||
public void testInvalidOnMisconfiguredProperty() {
|
||||
final RouteOnAttribute proc = new RouteOnAttribute();
|
||||
final MockProcessContext ctx = new MockProcessContext(proc, new MockStateManager(proc), variableRegistry);
|
||||
final MockProcessContext ctx = new MockProcessContext(proc);
|
||||
final ValidationResult validationResult = ctx.setProperty("RouteA", "${a:equals('b')"); // Missing closing brace
|
||||
assertFalse(validationResult.isValid());
|
||||
}
|
||||
|
@ -51,7 +46,7 @@ public class TestRouteOnAttribute {
|
|||
@Test
|
||||
public void testInvalidOnNonBooleanProperty() {
|
||||
final RouteOnAttribute proc = new RouteOnAttribute();
|
||||
final MockProcessContext ctx = new MockProcessContext(proc, new MockStateManager(proc), variableRegistry);
|
||||
final MockProcessContext ctx = new MockProcessContext(proc);
|
||||
final ValidationResult validationResult = ctx.setProperty("RouteA", "${a:length()"); // Should be boolean
|
||||
assertFalse(validationResult.isValid());
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ import org.apache.nifi.admin.service.AuditService;
|
|||
import org.apache.nifi.authorization.Authorizer;
|
||||
import org.apache.nifi.controller.repository.FlowFileEventRepository;
|
||||
import org.apache.nifi.provenance.MockProvenanceRepository;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.apache.nifi.util.CapturingLogger;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.junit.After;
|
||||
|
@ -67,8 +66,8 @@ public class MonitorMemoryTest {
|
|||
|
||||
@Test
|
||||
@Ignore // temporarily ignoring it since it fails intermittently due to
|
||||
// unpredictability during full build
|
||||
// still keeping it for local testing
|
||||
// unpredictability during full build
|
||||
// still keeping it for local testing
|
||||
public void validateWarnWhenPercentThresholdReached() throws Exception {
|
||||
this.doValidate("10%");
|
||||
}
|
||||
|
@ -136,7 +135,13 @@ public class MonitorMemoryTest {
|
|||
properties.setProperty("nifi.remote.input.socket.port", "");
|
||||
properties.setProperty("nifi.remote.input.secure", "");
|
||||
|
||||
return FlowController.createStandaloneInstance(mock(FlowFileEventRepository.class), properties,
|
||||
mock(Authorizer.class), mock(AuditService.class), null, null, VariableRegistryUtils.createCustomVariableRegistry(properties.getVariableRegistryPropertiesPaths()));
|
||||
return FlowController.createStandaloneInstance(
|
||||
mock(FlowFileEventRepository.class),
|
||||
properties,
|
||||
mock(Authorizer.class),
|
||||
mock(AuditService.class),
|
||||
null,
|
||||
null,
|
||||
null);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -41,8 +41,6 @@ import org.apache.nifi.distributed.cache.client.exception.DeserializationExcepti
|
|||
import org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer;
|
||||
import org.apache.nifi.processor.Processor;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.registry.VariableRegistryUtils;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.apache.nifi.util.MockConfigurationContext;
|
||||
import org.apache.nifi.util.MockControllerServiceInitializationContext;
|
||||
|
@ -57,7 +55,6 @@ import org.slf4j.LoggerFactory;
|
|||
public class TestServerAndClient {
|
||||
|
||||
private static Logger LOGGER;
|
||||
private static VariableRegistry variableRegistry;
|
||||
|
||||
static {
|
||||
System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
|
||||
|
@ -67,7 +64,6 @@ public class TestServerAndClient {
|
|||
System.setProperty("org.slf4j.simpleLogger.log.nifi.distributed.cache.server.TestServerAndClient", "debug");
|
||||
System.setProperty("org.slf4j.simpleLogger.log.nifi.remote.io.socket.ssl.SSLSocketChannel", "trace");
|
||||
LOGGER = LoggerFactory.getLogger(TestServerAndClient.class);
|
||||
variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -333,7 +329,7 @@ public class TestServerAndClient {
|
|||
clientProperties.put(DistributedMapCacheClientService.HOSTNAME, "localhost");
|
||||
clientProperties.put(DistributedMapCacheClientService.PORT, String.valueOf(server.getPort()));
|
||||
clientProperties.put(DistributedMapCacheClientService.COMMUNICATIONS_TIMEOUT, "360 secs");
|
||||
MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup(), variableRegistry);
|
||||
MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup());
|
||||
client.cacheConfig(clientContext);
|
||||
final Serializer<String> valueSerializer = new StringSerializer();
|
||||
final Serializer<String> keySerializer = new StringSerializer();
|
||||
|
@ -379,7 +375,7 @@ public class TestServerAndClient {
|
|||
client2.initialize(clientInitContext2);
|
||||
|
||||
MockConfigurationContext clientContext2 = new MockConfigurationContext(clientProperties,
|
||||
clientInitContext2.getControllerServiceLookup(), variableRegistry);
|
||||
clientInitContext2.getControllerServiceLookup());
|
||||
client2.cacheConfig(clientContext2);
|
||||
assertFalse(client2.putIfAbsent("testKey", "test", keySerializer, valueSerializer));
|
||||
assertTrue(client2.containsKey("testKey", keySerializer));
|
||||
|
@ -412,7 +408,7 @@ public class TestServerAndClient {
|
|||
server.initialize(serverInitContext);
|
||||
|
||||
final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
|
||||
final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup(), variableRegistry);
|
||||
final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
|
||||
server.startServer(serverContext);
|
||||
|
||||
DistributedMapCacheClientService client = new DistributedMapCacheClientService();
|
||||
|
@ -422,7 +418,7 @@ public class TestServerAndClient {
|
|||
final Map<PropertyDescriptor, String> clientProperties = new HashMap<>();
|
||||
clientProperties.put(DistributedMapCacheClientService.HOSTNAME, "localhost");
|
||||
clientProperties.put(DistributedMapCacheClientService.COMMUNICATIONS_TIMEOUT, "360 secs");
|
||||
MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup(), variableRegistry);
|
||||
MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup());
|
||||
client.cacheConfig(clientContext);
|
||||
final Serializer<String> valueSerializer = new StringSerializer();
|
||||
final Serializer<String> keySerializer = new StringSerializer();
|
||||
|
@ -469,7 +465,7 @@ public class TestServerAndClient {
|
|||
final Map<PropertyDescriptor, String> clientProperties = new HashMap<>();
|
||||
clientProperties.put(DistributedSetCacheClientService.HOSTNAME, "localhost");
|
||||
clientProperties.put(DistributedSetCacheClientService.PORT, String.valueOf(port));
|
||||
final MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup(), variableRegistry);
|
||||
final MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup());
|
||||
client.onConfigured(clientContext);
|
||||
|
||||
return client;
|
||||
|
|
|
@ -23,7 +23,6 @@ import org.apache.nifi.attribute.expression.language.Query;
|
|||
import org.apache.nifi.attribute.expression.language.StandardExpressionLanguageCompiler;
|
||||
import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
|
||||
import org.apache.nifi.expression.AttributeExpression.ResultType;
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.update.attributes.dto.ActionDTO;
|
||||
import org.apache.nifi.update.attributes.dto.ConditionDTO;
|
||||
import org.apache.nifi.update.attributes.dto.RuleDTO;
|
||||
|
@ -33,10 +32,7 @@ import org.apache.nifi.update.attributes.dto.RuleDTO;
|
|||
*/
|
||||
public class UpdateAttributeModelFactory {
|
||||
|
||||
private final VariableRegistry variableRegistry;
|
||||
|
||||
public UpdateAttributeModelFactory(VariableRegistry variableRegistry) {
|
||||
this.variableRegistry = variableRegistry;
|
||||
public UpdateAttributeModelFactory() {
|
||||
}
|
||||
|
||||
public Rule createRule(final RuleDTO dto) {
|
||||
|
@ -79,7 +75,7 @@ public class UpdateAttributeModelFactory {
|
|||
}
|
||||
|
||||
// validate the condition's expression
|
||||
final StandardExpressionLanguageCompiler elCompiler = new StandardExpressionLanguageCompiler(variableRegistry);
|
||||
final StandardExpressionLanguageCompiler elCompiler = new StandardExpressionLanguageCompiler();
|
||||
final String syntaxError = elCompiler.validateExpression(dto.getExpression(), false);
|
||||
if (syntaxError != null) {
|
||||
throw new IllegalArgumentException(syntaxError);
|
||||
|
|
|
@ -46,7 +46,6 @@ import javax.ws.rs.core.Response.ResponseBuilder;
|
|||
import javax.ws.rs.core.UriBuilder;
|
||||
import javax.ws.rs.core.UriInfo;
|
||||
|
||||
import org.apache.nifi.registry.VariableRegistry;
|
||||
import org.apache.nifi.update.attributes.Action;
|
||||
import org.apache.nifi.update.attributes.Condition;
|
||||
import org.apache.nifi.update.attributes.Criteria;
|
||||
|
@ -188,7 +187,6 @@ public class RuleResource {
|
|||
|
||||
// get the web context
|
||||
final NiFiWebConfigurationContext configurationContext = (NiFiWebConfigurationContext) servletContext.getAttribute("nifi-web-configuration-context");
|
||||
final VariableRegistry variableRegistry = configurationContext.getVariableRegistry();
|
||||
|
||||
// ensure the rule has been specified
|
||||
if (requestEntity == null || requestEntity.getRule() == null) {
|
||||
|
@ -221,7 +219,7 @@ public class RuleResource {
|
|||
|
||||
// load the criteria
|
||||
final Criteria criteria = getCriteria(configurationContext, requestContext);
|
||||
final UpdateAttributeModelFactory factory = new UpdateAttributeModelFactory(variableRegistry);
|
||||
final UpdateAttributeModelFactory factory = new UpdateAttributeModelFactory();
|
||||
|
||||
// create the new rule
|
||||
final Rule rule;
|
||||
|
@ -263,14 +261,10 @@ public class RuleResource {
|
|||
// generate a new id
|
||||
final String uuid = UUID.randomUUID().toString();
|
||||
|
||||
// get the variable registry
|
||||
final NiFiWebConfigurationContext configurationContext = (NiFiWebConfigurationContext) servletContext.getAttribute("nifi-web-configuration-context");
|
||||
final VariableRegistry variableRegistry = configurationContext.getVariableRegistry();
|
||||
|
||||
final Condition condition;
|
||||
try {
|
||||
// create the condition object
|
||||
final UpdateAttributeModelFactory factory = new UpdateAttributeModelFactory(variableRegistry);
|
||||
final UpdateAttributeModelFactory factory = new UpdateAttributeModelFactory();
|
||||
condition = factory.createCondition(requestEntity.getCondition());
|
||||
condition.setId(uuid);
|
||||
} catch (final IllegalArgumentException iae) {
|
||||
|
@ -301,14 +295,10 @@ public class RuleResource {
|
|||
// generate a new id
|
||||
final String uuid = UUID.randomUUID().toString();
|
||||
|
||||
// get the variable registry
|
||||
final NiFiWebConfigurationContext configurationContext = (NiFiWebConfigurationContext) servletContext.getAttribute("nifi-web-configuration-context");
|
||||
final VariableRegistry variableRegistry = configurationContext.getVariableRegistry();
|
||||
|
||||
final Action action;
|
||||
try {
|
||||
// create the condition object
|
||||
final UpdateAttributeModelFactory factory = new UpdateAttributeModelFactory(variableRegistry);
|
||||
final UpdateAttributeModelFactory factory = new UpdateAttributeModelFactory();
|
||||
action = factory.createAction(requestEntity.getAction());
|
||||
action.setId(uuid);
|
||||
} catch (final IllegalArgumentException iae) {
|
||||
|
@ -471,8 +461,6 @@ public class RuleResource {
|
|||
|
||||
// get the web context
|
||||
final NiFiWebConfigurationContext nifiWebContext = (NiFiWebConfigurationContext) servletContext.getAttribute("nifi-web-configuration-context");
|
||||
// get the variable registry
|
||||
final VariableRegistry variableRegistry = nifiWebContext.getVariableRegistry();
|
||||
|
||||
// ensure the rule has been specified
|
||||
if (requestEntity == null || requestEntity.getRule() == null) {
|
||||
|
@ -509,7 +497,7 @@ public class RuleResource {
|
|||
requestEntity.getProcessorId(), requestEntity.getRevision(), requestEntity.getClientId());
|
||||
|
||||
// load the criteria
|
||||
final UpdateAttributeModelFactory factory = new UpdateAttributeModelFactory(variableRegistry);
|
||||
final UpdateAttributeModelFactory factory = new UpdateAttributeModelFactory();
|
||||
final Criteria criteria = getCriteria(nifiWebContext, requestContext);
|
||||
|
||||
// attempt to locate the rule
|
||||
|
|
Loading…
Reference in New Issue