NIFI-2208 - initial commit Custom Property Expression Language support with Variable Registry, includes bug fix for NIFI-2057

This closes #529

Signed-off-by: jpercivall <joepercivall@yahoo.com>
This commit is contained in:
Yolanda M. Davis 2016-06-23 13:25:17 -04:00 committed by jpercivall
parent b213ed95e0
commit 8412d2662a
87 changed files with 1769 additions and 608 deletions

View File

@ -21,5 +21,5 @@
<version>1.0.0-SNAPSHOT</version>
</parent>
<artifactId>nifi-api</artifactId>
<packaging>jar</packaging>
</project>
<packaging>jar</packaging>
</project>

View File

@ -0,0 +1,67 @@
/*
* 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;
}

View File

@ -0,0 +1,145 @@
/*
* 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);
}
}

View File

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

View File

@ -0,0 +1,82 @@
/*
* 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;
}
}

View File

@ -0,0 +1,60 @@
/*
* 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;
/**
* Access key/value pairs throughout the application.
*/
public interface VariableRegistry {
/**
* Returns a map of key/value pairs stored in the registry
* @return variables
**/
Map<String, String> getVariables();
/**
* Return a value for a given variable
* @param variable variable
* @return value
**/
String getVariableValue(String variable);
/**
* Concatenate a variable registry
* @param variableRegistry variableRegistry
* */
void addRegistry(VariableRegistry variableRegistry);
/**
* Returns a set variable names in the registry
* @return variableNames
**/
Set<String> getVariableNames();
/**
* Concatenate variable key value pair to registry
* @param variables variable Map
* */
void addVariables(Map<String,String> variables);
}

View File

@ -0,0 +1,48 @@
/*
* 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();
}
}

View File

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

View File

@ -0,0 +1,87 @@
/*
* 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;
}
}

View File

@ -0,0 +1,170 @@
/*
* 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.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;
public class TestVariableRegistry {
@Test
public void testReadMap(){
Map<String,String> variables1 = new HashMap<>();
variables1.put("fake.property.1","fake test value");
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
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"));
}
}

View File

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

View File

@ -0,0 +1,16 @@
# 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.
fake.property.3=test me out 3, test me out 4

View File

@ -0,0 +1,17 @@
# 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.
fake.property.1=test me out 1
fake.property.2=test me out 2

View File

@ -41,5 +41,10 @@
<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>
</project>

View File

@ -46,6 +46,8 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.registry.VariableRegistryUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.w3c.dom.Document;
@ -62,8 +64,15 @@ public class NotificationServiceManager {
private final ScheduledExecutorService notificationExecutor;
private int maxAttempts = 5;
private final VariableRegistry variableRegistry;
public NotificationServiceManager() {
this(VariableRegistryUtils.createSystemVariableRegistry());
}
NotificationServiceManager(VariableRegistry variableRegistry){
this.variableRegistry = variableRegistry;
notificationExecutor = Executors.newScheduledThreadPool(1, new ThreadFactory() {
@Override
public Thread newThread(final Runnable r) {
@ -141,7 +150,7 @@ public class NotificationServiceManager {
}
// Check if the service is valid; if not, warn now so that users know this before they fail to receive notifications
final ValidationContext validationContext = new NotificationValidationContext(buildNotificationContext(config));
final ValidationContext validationContext = new NotificationValidationContext(buildNotificationContext(config), variableRegistry);
final Collection<ValidationResult> validationResults = service.validate(validationContext);
final List<String> invalidReasons = new ArrayList<>();
@ -179,7 +188,7 @@ public class NotificationServiceManager {
@Override
public void run() {
// Check if the service is valid; if not, warn now so that users know this before they fail to receive notifications
final ValidationContext validationContext = new NotificationValidationContext(buildNotificationContext(config));
final ValidationContext validationContext = new NotificationValidationContext(buildNotificationContext(config), variableRegistry);
final Collection<ValidationResult> validationResults = service.validate(validationContext);
final List<String> invalidReasons = new ArrayList<>();
@ -247,7 +256,7 @@ public class NotificationServiceManager {
configuredValue = fullPropDescriptor.getDefaultValue();
}
return new StandardPropertyValue(configuredValue, null);
return new StandardPropertyValue(configuredValue, null, variableRegistry);
}
@Override
@ -364,7 +373,7 @@ public class NotificationServiceManager {
value = descriptor.getDefaultValue();
}
return new StandardPropertyValue(value, null);
return new StandardPropertyValue(value, null, variableRegistry);
}
@Override

View File

@ -30,12 +30,14 @@ import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.expression.ExpressionLanguageCompiler;
import org.apache.nifi.registry.VariableRegistry;
public class NotificationValidationContext implements ValidationContext {
private final NotificationContext context;
private final Map<String, Boolean> expressionLanguageSupported;
private final VariableRegistry variableRegistry;
public NotificationValidationContext(final NotificationContext processContext) {
public NotificationValidationContext(final NotificationContext processContext, VariableRegistry variableRegistry) {
this.context = processContext;
final Map<PropertyDescriptor, String> properties = processContext.getProperties();
@ -43,17 +45,19 @@ public class NotificationValidationContext implements ValidationContext {
for (final PropertyDescriptor descriptor : properties.keySet()) {
expressionLanguageSupported.put(descriptor.getName(), descriptor.isExpressionLanguageSupported());
}
this.variableRegistry = variableRegistry;
}
@Override
public PropertyValue newPropertyValue(final String rawValue) {
return new StandardPropertyValue(rawValue, null);
return new StandardPropertyValue(rawValue, null, variableRegistry);
}
@Override
public ExpressionLanguageCompiler newExpressionLanguageCompiler() {
return new StandardExpressionLanguageCompiler();
return new StandardExpressionLanguageCompiler(null);
}
@Override

View File

@ -0,0 +1,50 @@
/*
* 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.bootstrap
import org.apache.nifi.bootstrap.notification.NotificationType
import org.apache.nifi.registry.VariableRegistry
import spock.lang.Specification
import java.nio.file.Paths
class NotificationServiceManagerSpec extends Specification{
def setupSpec(){
}
def "should acess variable registry to replace EL values"(){
given:
def mockRegistry = Mock(VariableRegistry.class)
def notificationServiceManager = new NotificationServiceManager(mockRegistry);
def file = Paths.get("src/test/resources/notification-services.xml").toFile()
notificationServiceManager.loadNotificationServices(file)
//testing with stopped becasue it will block until method is completed
notificationServiceManager.registerNotificationService(NotificationType.NIFI_STOPPED,"custom-notification")
when:
notificationServiceManager.notify(NotificationType.NIFI_STOPPED,"NiFi Stopped","NiFi Stopped")
then:
6 * mockRegistry.getVariables() >> ["test.server":"smtp://fakeserver.com","test.username":"user","test.password":"pass"]
}
}

View File

@ -0,0 +1,85 @@
package org.apache.nifi.bootstrap.notification;
/*
* 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.
*/
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.processor.util.StandardValidators;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class TestCustomNotificationService extends AbstractNotificationService {
private static Logger logger = LoggerFactory.getLogger(TestCustomNotificationService.class);
public static final PropertyDescriptor CUSTOM_HOSTNAME = new PropertyDescriptor.Builder()
.name("Custom Hostname")
.description("The hostname of the Custom Server that is used to send notifications")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.required(true)
.build();
public static final PropertyDescriptor CUSTOM_USERNAME = new PropertyDescriptor.Builder()
.name("Custom Username")
.description("Username for the account")
.expressionLanguageSupported(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.required(false)
.build();
public static final PropertyDescriptor CUSTOM_PASSWORD = new PropertyDescriptor.Builder()
.name("Custom Password")
.description("Password for the account")
.expressionLanguageSupported(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.required(false)
.sensitive(true)
.build();
/**
* Mapping of the mail properties to the NiFi PropertyDescriptors that will be evaluated at runtime
*/
private static final Map<String, PropertyDescriptor> propertyToContext = new HashMap<>();
static {
propertyToContext.put("custom.host", CUSTOM_HOSTNAME);
propertyToContext.put("custom.user", CUSTOM_USERNAME);
propertyToContext.put("custom.password", CUSTOM_PASSWORD);
}
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.add(CUSTOM_HOSTNAME);
properties.add(CUSTOM_USERNAME);
properties.add(CUSTOM_PASSWORD);
return properties;
}
@Override
public void notify(NotificationContext context, String subject, String message) throws NotificationFailedException {
logger.info(context.getProperty(CUSTOM_HOSTNAME).evaluateAttributeExpressions().getValue());
logger.info(context.getProperty(CUSTOM_USERNAME).evaluateAttributeExpressions().getValue());
logger.info(context.getProperty(CUSTOM_PASSWORD).evaluateAttributeExpressions().getValue());
}
}

View File

@ -0,0 +1,24 @@
<?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.
-->
<services>
<service>
<id>custom-notification</id>
<class>org.apache.nifi.bootstrap.notification.TestCustomNotificationService</class>
<property name="Custom Hostname">${test.server}</property>
<property name="Custom Username">${test.username}</property>
<property name="Custom Password">${test.password}</property>
</service>
</services>

View File

@ -16,11 +16,10 @@
*/
package org.apache.nifi.attribute.expression.language;
import java.util.Map;
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;
public class EmptyPreparedQuery implements PreparedQuery {
@ -31,37 +30,7 @@ public class EmptyPreparedQuery implements PreparedQuery {
}
@Override
public String evaluateExpressions(final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException {
return value;
}
@Override
public String evaluateExpressions() throws ProcessException {
return value;
}
@Override
public String evaluateExpressions(final AttributeValueDecorator decorator) throws ProcessException {
return value;
}
@Override
public String evaluateExpressions(final FlowFile flowFile) throws ProcessException {
return value;
}
@Override
public String evaluateExpressions(Map<String, String> attributes) throws ProcessException {
return value;
}
@Override
public String evaluateExpressions(Map<String, String> attributes, AttributeValueDecorator decorator) throws ProcessException {
return value;
}
@Override
public String evaluateExpressions(FlowFile flowFile, Map<String, String> additionalAttributes, AttributeValueDecorator decorator) throws ProcessException {
public String evaluateExpressions(VariableRegistry variableRegistry, AttributeValueDecorator decorator) throws ProcessException {
return value;
}
}

View File

@ -16,12 +16,11 @@
*/
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.flowfile.FlowFile;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.registry.VariableRegistry;
/**
* An implementation of PreparedQuery that throws an
@ -40,37 +39,9 @@ public class InvalidPreparedQuery implements PreparedQuery {
}
@Override
public String evaluateExpressions(final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException {
public String evaluateExpressions(final VariableRegistry variableRegistry, final AttributeValueDecorator decorator) throws ProcessException {
throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation);
}
@Override
public String evaluateExpressions() throws ProcessException {
throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation);
}
@Override
public String evaluateExpressions(final AttributeValueDecorator decorator) throws ProcessException {
throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation);
}
@Override
public String evaluateExpressions(final FlowFile flowFile) throws ProcessException {
throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation);
}
@Override
public String evaluateExpressions(final Map<String, String> attributes) throws ProcessException {
throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation);
}
@Override
public String evaluateExpressions(final Map<String, String> attributes, final AttributeValueDecorator decorator) throws ProcessException {
throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation);
}
@Override
public String evaluateExpressions(FlowFile flowFile, Map<String, String> additionalAttributes, AttributeValueDecorator decorator) throws ProcessException {
throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation);
}
}

View File

@ -16,25 +16,13 @@
*/
package org.apache.nifi.attribute.expression.language;
import java.util.Map;
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;
public interface PreparedQuery {
String evaluateExpressions(FlowFile flowFile, AttributeValueDecorator decorator) throws ProcessException;
String evaluateExpressions(VariableRegistry registry, AttributeValueDecorator decorator) throws ProcessException;
String evaluateExpressions() throws ProcessException;
String evaluateExpressions(AttributeValueDecorator decorator) throws ProcessException;
String evaluateExpressions(FlowFile flowFile) throws ProcessException;
String evaluateExpressions(Map<String, String> attributes) throws ProcessException;
String evaluateExpressions(Map<String, String> attributes, AttributeValueDecorator decorator) throws ProcessException;
String evaluateExpressions(FlowFile flowFile, Map<String, String> additionalAttributes, AttributeValueDecorator decorator) throws ProcessException;
}

View File

@ -18,13 +18,9 @@ package org.apache.nifi.attribute.expression.language;
import java.net.UnknownHostException;
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 java.util.concurrent.atomic.AtomicBoolean;
import org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionLexer;
@ -198,6 +194,7 @@ 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
@ -367,8 +364,8 @@ public class Query {
return -1;
}
static String evaluateExpression(final Tree tree, final String queryText, final Map<String, String> expressionMap, final AttributeValueDecorator decorator) throws ProcessException {
final Object evaluated = Query.fromTree(tree, queryText).evaluate(expressionMap).getValue();
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();
if (evaluated == null) {
return null;
}
@ -378,29 +375,12 @@ public class Query {
return decorator == null ? escaped : decorator.decorate(escaped);
}
static String evaluateExpressions(final String rawValue, Map<String, String> expressionMap) throws ProcessException {
return evaluateExpressions(rawValue, expressionMap, null);
static String evaluateExpressions(final String rawValue, VariableRegistry registry) throws ProcessException {
return evaluateExpressions(rawValue, registry, null);
}
static String evaluateExpressions(final String rawValue) throws ProcessException {
return evaluateExpressions(rawValue, createExpressionMap(null), null);
}
static String evaluateExpressions(final String rawValue, final FlowFile flowFile) throws ProcessException {
return evaluateExpressions(rawValue, createExpressionMap(flowFile), null);
}
static String evaluateExpressions(final String rawValue, Map<String, String> expressionMap, final AttributeValueDecorator decorator) throws ProcessException {
return Query.prepare(rawValue).evaluateExpressions(expressionMap, decorator);
}
public static String evaluateExpressions(final String rawValue, final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException {
if (rawValue == null) {
return null;
}
final Map<String, String> expressionMap = createExpressionMap(flowFile);
return evaluateExpressions(rawValue, expressionMap, decorator);
static String evaluateExpressions(final String rawValue, VariableRegistry registry, final AttributeValueDecorator decorator) throws ProcessException {
return Query.prepare(rawValue).evaluateExpressions(registry, decorator);
}
private static Evaluator<?> getRootSubjectEvaluator(final Evaluator<?> evaluator) {
@ -426,150 +406,6 @@ public class Query {
return value.replaceAll("\\$\\$(?=\\$*\\{.*?\\})", "\\$");
}
static Map<String, String> createExpressionMap(final FlowFile flowFile) {
return createExpressionMap(flowFile, null);
}
static Map<String, String> createExpressionMap(final FlowFile flowFile, final Map<String, String> additionalAttributes) {
final Map<String, String> attributeMap = flowFile == null ? Collections.emptyMap() : flowFile.getAttributes();
final Map<String, String> additionalOrEmpty = additionalAttributes == null ? Collections.emptyMap() : additionalAttributes;
final Map<String, String> envMap = System.getenv();
final Map<?, ?> sysProps = System.getProperties();
final Map<String, String> flowFileProps = new HashMap<>();
if (flowFile != null) {
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()));
}
return wrap(additionalOrEmpty, attributeMap, flowFileProps, envMap, sysProps);
}
private static Map<String, String> wrap(final Map<String, String> additional, final Map<String, String> attributes, final Map<String, String> flowFileProps,
final Map<String, String> env, final Map<?, ?> sysProps) {
@SuppressWarnings("rawtypes")
final Map[] maps = new Map[] {additional, attributes, flowFileProps, env, sysProps};
return new Map<String, String>() {
@Override
public int size() {
int size = 0;
for (final Map<?, ?> map : maps) {
size += map.size();
}
return size;
}
@Override
public boolean isEmpty() {
for (final Map<?, ?> map : maps) {
if (!map.isEmpty()) {
return false;
}
}
return true;
}
@Override
public boolean containsKey(final Object key) {
if (key == null) {
return false;
}
if (!(key instanceof String)) {
return false;
}
for (final Map<?, ?> map : maps) {
if (map.containsKey(key)) {
return true;
}
}
return false;
}
@Override
public boolean containsValue(final Object value) {
for (final Map<?, ?> map : maps) {
if (map.containsValue(value)) {
return true;
}
}
return false;
}
@Override
@SuppressWarnings("rawtypes")
public String get(final Object key) {
if (key == null) {
throw new IllegalArgumentException("Null Keys are not allowed");
}
if (!(key instanceof String)) {
return null;
}
for (final Map map : maps) {
final Object val = map.get(key);
if (val != null) {
return String.valueOf(val);
}
}
return null;
}
@Override
public String put(String key, String value) {
throw new UnsupportedOperationException();
}
@Override
public String remove(final Object key) {
throw new UnsupportedOperationException();
}
@Override
public void putAll(final Map<? extends String, ? extends String> 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<String> values() {
final Set<String> 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, String>> entrySet() {
final Set<java.util.Map.Entry<String, String>> entrySet = new HashSet<>();
for (final Map map : maps) {
entrySet.addAll(map.entrySet());
}
return entrySet;
}
};
}
public static Query fromTree(final Tree tree, final String text) {
return new Query(text, tree, buildEvaluator(tree));
@ -706,20 +542,12 @@ public class Query {
return evaluator.getResultType();
}
QueryResult<?> evaluate() {
return evaluate(createExpressionMap(null));
}
QueryResult<?> evaluate(final FlowFile flowFile) {
return evaluate(createExpressionMap(flowFile));
}
QueryResult<?> evaluate(final Map<String, String> attributes) {
QueryResult<?> evaluate(final VariableRegistry registry) {
if (evaluated.getAndSet(true)) {
throw new IllegalStateException("A Query cannot be evaluated more than once");
}
return evaluator.evaluate(attributes);
return evaluator.evaluate(registry.getVariables());
}
Tree getTree() {

View File

@ -20,13 +20,17 @@ import org.apache.nifi.expression.AttributeExpression;
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) {
public StandardAttributeExpression(final Query query, final VariableRegistry variableRegistry) {
this.query = query;
this.variableRegistry = variableRegistry;
}
@Override
@ -51,7 +55,8 @@ public class StandardAttributeExpression implements AttributeExpression {
@Override
public String evaluate(final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException {
final Object evaluationResult = query.evaluate(flowFile).getValue();
VariableRegistry flowFileRegistry = VariableRegistryUtils.createFlowVariableRegistry(variableRegistry,flowFile,null);
final Object evaluationResult = query.evaluate(flowFileRegistry).getValue();
if (evaluationResult == null) {
return "";
}

View File

@ -20,13 +20,20 @@ import org.apache.nifi.attribute.expression.language.exception.AttributeExpressi
import org.apache.nifi.expression.AttributeExpression;
import org.apache.nifi.expression.ExpressionLanguageCompiler;
import org.apache.nifi.expression.AttributeExpression.ResultType;
import org.apache.nifi.registry.VariableRegistry;
public class StandardExpressionLanguageCompiler implements ExpressionLanguageCompiler {
private final VariableRegistry variableRegistry;
public StandardExpressionLanguageCompiler(final VariableRegistry variableRegistry) {
this.variableRegistry = variableRegistry;
}
@Override
public AttributeExpression compile(final String expression) throws IllegalArgumentException {
try {
return new StandardAttributeExpression(Query.compile(expression));
return new StandardAttributeExpression(Query.compile(expression),variableRegistry);
} catch (final AttributeExpressionLanguageParsingException e) {
throw new IllegalArgumentException(e.getMessage());
}

View File

@ -22,10 +22,10 @@ import java.util.List;
import java.util.Map;
import org.apache.nifi.expression.AttributeValueDecorator;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.exception.ProcessException;
import org.antlr.runtime.tree.Tree;
import org.apache.nifi.registry.VariableRegistry;
public class StandardPreparedQuery implements PreparedQuery {
@ -37,20 +37,16 @@ public class StandardPreparedQuery implements PreparedQuery {
this.trees = new HashMap<>(trees);
}
@Override
public String evaluateExpressions(Map<String, String> attributes) throws ProcessException {
return evaluateExpressions(attributes, null);
}
@Override
public String evaluateExpressions(final Map<String, String> attributes, final AttributeValueDecorator decorator) throws ProcessException {
public String evaluateExpressions(final VariableRegistry registry, 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, attributes, decorator);
final String evaluated = Query.evaluateExpression(tree, val, registry, decorator);
if (evaluated != null) {
sb.append(evaluated);
}
@ -59,31 +55,4 @@ public class StandardPreparedQuery implements PreparedQuery {
return sb.toString();
}
@Override
public String evaluateExpressions(final FlowFile flowFile, final Map<String, String> additionalAttributes, final AttributeValueDecorator decorator) throws ProcessException {
final Map<String, String> expressionMap = Query.createExpressionMap(flowFile, additionalAttributes);
return evaluateExpressions(expressionMap, decorator);
}
@Override
public String evaluateExpressions(final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException {
final Map<String, String> expressionMap = Query.createExpressionMap(flowFile);
return evaluateExpressions(expressionMap, decorator);
}
@Override
public String evaluateExpressions() throws ProcessException {
return evaluateExpressions((FlowFile) null, null);
}
@Override
public String evaluateExpressions(final AttributeValueDecorator decorator) throws ProcessException {
return evaluateExpressions((FlowFile) null, decorator);
}
@Override
public String evaluateExpressions(final FlowFile flowFile) throws ProcessException {
return evaluateExpressions(flowFile, null);
}
}

View File

@ -26,6 +26,8 @@ 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;
import org.apache.nifi.registry.VariableRegistryUtils;
import org.apache.nifi.util.FormatUtils;
public class StandardPropertyValue implements PropertyValue {
@ -33,24 +35,27 @@ public class StandardPropertyValue implements PropertyValue {
private final String rawValue;
private final ControllerServiceLookup serviceLookup;
private final PreparedQuery preparedQuery;
private final VariableRegistry variableRegistry;
public StandardPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup) {
this(rawValue, serviceLookup, Query.prepare(rawValue));
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)} 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
*
* 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) {
public StandardPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup, final PreparedQuery preparedQuery,
final VariableRegistry variableRegistry) {
this.rawValue = rawValue;
this.serviceLookup = serviceLookup;
this.preparedQuery = preparedQuery;
this.variableRegistry = variableRegistry;
}
@Override
@ -133,10 +138,12 @@ public class StandardPropertyValue implements PropertyValue {
if (rawValue == null || preparedQuery == null) {
return this;
}
return new StandardPropertyValue(preparedQuery.evaluateExpressions(flowFile, additionalAttributes, decorator), serviceLookup, null);
VariableRegistry registry = VariableRegistryUtils.createFlowVariableRegistry(variableRegistry,flowFile,additionalAttributes);
return new StandardPropertyValue(preparedQuery.evaluateExpressions(registry,decorator), serviceLookup,null,registry);
}
@Override
public String toString() {
return rawValue;

View File

@ -18,6 +18,8 @@ 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
@ -57,6 +59,7 @@ 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"
@ -75,10 +78,10 @@ public class QueryGroovyTest extends GroovyTestCase {
Query replaceRepeatingQuery = Query.compile(REPLACE_REPEATING_EXPRESSION)
// Act
QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(attributes)
QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(variableRegistry)
logger.info("Replace single result: ${replaceSingleResult.value}")
QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(attributes)
QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(variableRegistry)
logger.info("Replace repeating result: ${replaceRepeatingResult.value}")
// Assert
@ -97,6 +100,7 @@ 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"
@ -115,10 +119,10 @@ public class QueryGroovyTest extends GroovyTestCase {
Query replaceRepeatingQuery = Query.compile(REPLACE_REPEATING_EXPRESSION)
// Act
QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(attributes)
QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(variableRegistry)
logger.info("Replace single result: ${replaceSingleResult.value}")
QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(attributes)
QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(variableRegistry)
logger.info("Replace repeating result: ${replaceRepeatingResult.value}")
// Assert
@ -137,6 +141,7 @@ 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"
@ -155,10 +160,10 @@ public class QueryGroovyTest extends GroovyTestCase {
Query replaceRepeatingQuery = Query.compile(REPLACE_REPEATING_EXPRESSION)
// Act
QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(attributes)
QueryResult<?> replaceSingleResult = replaceSingleQuery.evaluate(variableRegistry)
logger.info("Replace single result: ${replaceSingleResult.value}")
QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(attributes)
QueryResult<?> replaceRepeatingResult = replaceRepeatingQuery.evaluate(variableRegistry)
logger.info("Replace repeating result: ${replaceRepeatingResult.value}")
// Assert

View File

@ -35,6 +35,7 @@ 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;
@ -43,7 +44,12 @@ import org.apache.nifi.attribute.expression.language.exception.AttributeExpressi
import org.apache.nifi.expression.AttributeExpression.ResultType;
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;
import org.junit.Test;
@ -51,6 +57,7 @@ import org.mockito.Mockito;
public class TestQuery {
@Test
public void testCompilation() {
assertInvalid("${attr:uuid()}");
@ -114,7 +121,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, attributes, null);
final String result = Query.evaluateExpressions(expression,VariableRegistryFactory.getInstance(attributes), null);
assertEquals("true", result);
Query.validateExpression(expression, false);
@ -174,14 +181,14 @@ 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}'", attributes, null));
assertEquals("'My Value", Query.evaluateExpressions("'${attr}", attributes, null));
assertEquals("'My Value'", Query.evaluateExpressions("'${attr}'", registry, null));
assertEquals("'My Value", Query.evaluateExpressions("'${attr}", registry, null));
}
@Test
@ -191,7 +198,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(attributes);
final QueryResult<?> result = query.evaluate(VariableRegistryFactory.getInstance(attributes));
assertEquals(ResultType.NUMBER, result.getResultType());
assertEquals(1384788147678L, result.getValue());
}
@ -220,7 +227,7 @@ public class TestQuery {
final Date roundedToNearestSecond = new Date(date.getTime() - millis);
final String formatted = sdf.format(roundedToNearestSecond);
final QueryResult<?> result = query.evaluate(attributes);
final QueryResult<?> result = query.evaluate(VariableRegistryFactory.getInstance(attributes));
assertEquals(ResultType.STRING, result.getResultType());
assertEquals(formatted, result.getValue());
}
@ -230,14 +237,15 @@ 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}')}", attributes, null));
assertEquals("true", Query.evaluateExpressions("${x:equals('${a}')}", registry, null));
Query.validateExpression("${x:equals(\"${a}\")}", false);
assertEquals("true", Query.evaluateExpressions("${x:equals(\"${a}\")}", attributes, null));
assertEquals("true", Query.evaluateExpressions("${x:equals(\"${a}\")}", registry, null));
}
@Test
@ -268,6 +276,22 @@ public class TestQuery {
}
}
@Test
public void testEmbeddedExpressionsAndQuotesWithProperties() {
final Properties attributes = new Properties();
attributes.put("x", "abc");
attributes.put("a", "abc");
VariableRegistry registry = VariableRegistryFactory.getPropertiesInstance(attributes);
verifyEquals("${x:equals(${a})}",registry,true);
Query.validateExpression("${x:equals('${a}')}", false);
assertEquals("true", Query.evaluateExpressions("${x:equals('${a}')}", registry, null));
Query.validateExpression("${x:equals(\"${a}\")}", false);
assertEquals("true", Query.evaluateExpressions("${x:equals(\"${a}\")}", registry, null));
}
@Test
public void testJoin() {
final Map<String, String> attributes = new HashMap<>();
@ -342,7 +366,9 @@ public class TestQuery {
Mockito.when(mockFlowFile.getEntryDate()).thenReturn(System.currentTimeMillis());
Mockito.when(mockFlowFile.getSize()).thenReturn(1L);
Mockito.when(mockFlowFile.getLineageStartDate()).thenReturn(System.currentTimeMillis());
return Query.evaluateExpressions(queryString, mockFlowFile);
final VariableRegistry variableRegistry = VariableRegistryUtils.createFlowVariableRegistry(VariableRegistryUtils.createSystemVariableRegistry(),mockFlowFile,null);
return Query.evaluateExpressions(queryString,variableRegistry);
}
@Test
@ -498,7 +524,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}' )}", attributes, null));
assertEquals("true", Query.evaluateExpressions("${x:equals( '${y}' )}", VariableRegistryFactory.getInstance(attributes), null));
}
@Test
@ -546,7 +572,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, attributes, null);
final String result = Query.evaluateExpressions(query, VariableRegistryFactory.getInstance(attributes), null);
final String expectedTime = new SimpleDateFormat(format, Locale.US).format(timestamp);
assertEquals("startDateTime=\"" + expectedTime + "\"", result);
@ -615,7 +641,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));
assertEquals("true", Query.evaluateExpressions(query,VariableRegistryUtils.createSystemVariableRegistry()));
}
@Test
@ -631,7 +657,7 @@ 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"
@ -642,12 +668,12 @@ public class TestQuery {
+ "}";
Query query = Query.compile(expression);
QueryResult<?> result = query.evaluate(attributes);
QueryResult<?> result = query.evaluate(registry);
assertEquals(ResultType.STRING, result.getResultType());
assertEquals("xyz", result.getValue());
query = Query.compile("${abc:append('# hello') #good-bye \n}");
result = query.evaluate(attributes);
result = query.evaluate(registry);
assertEquals(ResultType.STRING, result.getResultType());
assertEquals("xyz# hello", result.getValue());
}
@ -776,14 +802,15 @@ 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')}", attributes, 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')}", registry, null));
assertEquals("63", Query.evaluateExpressions("${year:append('/'):append('${month}'):append('/'):append('${day}'):toDate('yyyy/MM/dd'):format('D')}", registry, null));
verifyEquals("${year:append('/'):append(${month}):append('/'):append(${day}):toDate('yyyy/MM/dd'):format('D')}", attributes, "63");
}
@ -791,8 +818,9 @@ public class TestQuery {
@Test
public void testSystemProperty() {
System.setProperty("hello", "good-bye");
assertEquals("good-bye", Query.evaluateExpressions("${hello}"));
assertEquals("good-bye", Query.compile("${hello}").evaluate().getValue());
VariableRegistry variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
assertEquals("good-bye", Query.evaluateExpressions("${hello}",variableRegistry));
assertEquals("good-bye", Query.compile("${hello}").evaluate(variableRegistry).getValue());
}
@Test
@ -832,14 +860,15 @@ 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, 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));
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));
verifyEquals("${anyDelineatedValue(${abc}, ','):equals('b')}", attributes, true);
verifyEquals("${anyDelineatedValue(${abc}, ','):equals('a')}", attributes, true);
@ -853,13 +882,15 @@ 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, 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));
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));
verifyEquals("${allDelineatedValues(${abc}, ','):matches('[abc]')}", attributes, true);
verifyEquals("${allDelineatedValues(${abc}, ','):matches('[abd]')}", attributes, false);
@ -925,12 +956,13 @@ 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')}", attributes, null);
final String evaluated = Query.evaluateExpressions("${abc:matches('1234${end}4321')}", registry, null);
assertEquals("true", evaluated);
attributes.put("end", "888");
final String secondEvaluation = Query.evaluateExpressions("${abc:matches('1234${end}4321')}", attributes, null);
final String secondEvaluation = Query.evaluateExpressions("${abc:matches('1234${end}4321')}", registry, null);
assertEquals("false", secondEvaluation);
verifyEquals("${dotted:matches('abc\\.xyz')}", attributes, true);
@ -945,11 +977,13 @@ public class TestQuery {
attributes.put("hello", "world!");
attributes.put("dotted", "abc.xyz");
final String evaluated = Query.evaluateExpressions("${abc:find('1234${end}4321')}", attributes, null);
final String evaluated = Query.evaluateExpressions("${abc:find('1234${end}4321')}", VariableRegistryFactory.getInstance(attributes), null);
assertEquals("true", evaluated);
attributes.put("end", "888");
final String secondEvaluation = Query.evaluateExpressions("${abc:find('${end}4321')}", attributes, null);
final String secondEvaluation = Query.evaluateExpressions("${abc:find('${end}4321')}",VariableRegistryFactory.getInstance(attributes), null);
assertEquals("false", secondEvaluation);
verifyEquals("${dotted:find('\\.')}", attributes, true);
@ -1130,7 +1164,7 @@ public class TestQuery {
attributes.put("b", "x");
attributes.put("abcxcba", "hello");
final String evaluated = Query.evaluateExpressions("${ 'abc${b}cba':substring(0, 1) }", attributes, null);
final String evaluated = Query.evaluateExpressions("${ 'abc${b}cba':substring(0, 1) }", VariableRegistryFactory.getInstance(attributes), null);
assertEquals("h", evaluated);
}
@ -1164,7 +1198,7 @@ public class TestQuery {
final List<String> expressions = Query.extractExpressions(query);
assertEquals(1, expressions.size());
assertEquals("${abc}", expressions.get(0));
assertEquals("{ xyz }", Query.evaluateExpressions(query, attributes));
assertEquals("{ xyz }", Query.evaluateExpressions(query, VariableRegistryFactory.getInstance(attributes)));
}
@Test
@ -1188,7 +1222,7 @@ public class TestQuery {
QueryResult<?> getResult(String expr, Map<String, String> attrs) {
final Query query = Query.compile(expr);
final QueryResult<?> result = query.evaluate(attrs);
final QueryResult<?> result = query.evaluate(VariableRegistryFactory.getInstance(attrs));
return result;
}
@ -1297,11 +1331,17 @@ 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, attributes, null));
assertEquals(String.valueOf(expectedResult), Query.evaluateExpressions(expression, registry, null));
final Query query = Query.compile(expression);
final QueryResult<?> result = query.evaluate(attributes);
final QueryResult<?> result = query.evaluate(registry);
if (expectedResult instanceof Number) {
assertEquals(ResultType.NUMBER, result.getResultType());

View File

@ -22,6 +22,7 @@ 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;
@ -53,7 +54,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(attrs, null));
assertEquals("world", prepared.evaluateExpressions(VariableRegistryFactory.getInstance(attrs), null));
}
final long nanos = System.nanoTime() - start;
System.out.println(TimeUnit.NANOSECONDS.toMillis(nanos));
@ -67,7 +68,7 @@ public class TestStandardPreparedQuery {
final long start = System.nanoTime();
for (int i = 0; i < 10000000; i++) {
assertEquals("world", Query.evaluateExpressions("${xx}", attrs));
assertEquals("world", Query.evaluateExpressions("${xx}", VariableRegistryFactory.getInstance(attrs)));
}
final long nanos = System.nanoTime() - start;
System.out.println(TimeUnit.NANOSECONDS.toMillis(nanos));
@ -85,7 +86,7 @@ public class TestStandardPreparedQuery {
}
private String evaluate(final String query, final Map<String, String> attrs) {
final String evaluated = ((StandardPreparedQuery) Query.prepare(query)).evaluateExpressions(attrs, null);
final String evaluated = ((StandardPreparedQuery) Query.prepare(query)).evaluateExpressions(VariableRegistryFactory.getInstance(attrs), null);
return evaluated;
}

View File

@ -28,6 +28,7 @@ import java.nio.file.InvalidPathException;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -188,6 +189,9 @@ public class NiFiProperties extends Properties {
public static final String STATE_MANAGEMENT_START_EMBEDDED_ZOOKEEPER = "nifi.state.management.embedded.zookeeper.start";
public static final String STATE_MANAGEMENT_ZOOKEEPER_PROPERTIES = "nifi.state.management.embedded.zookeeper.properties";
// expression language properties
public static final String VARIABLE_REGISTRY_PROPERTIES = "nifi.variable.registry.properties";
// defaults
public static final String DEFAULT_TITLE = "NiFi";
public static final Boolean DEFAULT_AUTO_RESUME_STATE = true;
@ -963,4 +967,27 @@ public class NiFiProperties extends Properties {
public String getFlowConfigurationArchiveMaxStorage() {
return getProperty(FLOW_CONFIGURATION_ARCHIVE_MAX_STORAGE, DEFAULT_FLOW_CONFIGURATION_ARCHIVE_MAX_STORAGE);
}
public String getVariableRegistryProperties(){
return getProperty(VARIABLE_REGISTRY_PROPERTIES);
}
public Path[] getVariableRegistryPropertiesPaths() {
final List<Path> vrPropertiesPaths = new ArrayList<>();
final String vrPropertiesFiles = getVariableRegistryProperties();
if(!StringUtils.isEmpty(vrPropertiesFiles)) {
final List<String> vrPropertiesFileList = Arrays.asList(vrPropertiesFiles.split(","));
for(String propertiesFile : vrPropertiesFileList){
vrPropertiesPaths.add(Paths.get(propertiesFile));
}
return vrPropertiesPaths.toArray( new Path[vrPropertiesPaths.size()]);
} else {
return null;
}
}
}

View File

@ -16,16 +16,17 @@
*/
package org.apache.nifi.web;
import org.apache.nifi.controller.ControllerService;
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 {
public interface NiFiWebConfigurationContext extends VariableRegistryProvider {
/**
* @param serviceIdentifier of the controller service

View File

@ -25,21 +25,26 @@ import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.registry.VariableRegistry;
public class MockConfigurationContext implements ConfigurationContext {
private final Map<PropertyDescriptor, String> properties;
private final ControllerServiceLookup serviceLookup;
private final ControllerService service;
private final VariableRegistry variableRegistry;
public MockConfigurationContext(final Map<PropertyDescriptor, String> properties, final ControllerServiceLookup serviceLookup) {
this(null, properties, serviceLookup);
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) {
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;
this.variableRegistry = variableRegistry;
}
@Override
@ -48,7 +53,7 @@ public class MockConfigurationContext implements ConfigurationContext {
if (value == null) {
value = getActualDescriptor(property).getDefaultValue();
}
return new MockPropertyValue(value, serviceLookup);
return new MockPropertyValue(value, serviceLookup, variableRegistry);
}
@Override

View File

@ -40,6 +40,7 @@ import org.apache.nifi.controller.NodeTypeProvider;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.SchedulingContext;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.state.MockStateManager;
import org.junit.Assert;
@ -48,6 +49,7 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
private final ConfigurableComponent component;
private final Map<PropertyDescriptor, String> properties = new HashMap<>();
private final StateManager stateManager;
private final VariableRegistry variableRegistry;
private String annotationData = null;
private boolean yieldCalled = false;
@ -63,22 +65,24 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
private volatile boolean isClustered;
private volatile boolean isPrimaryNode;
public MockProcessContext(final ConfigurableComponent component) {
this(component, new MockStateManager(component));
public MockProcessContext(final ConfigurableComponent component, final VariableRegistry variableRegistry) {
this(component, new MockStateManager(component),variableRegistry);
}
/**
* Creates a new MockProcessContext for the given Processor
*
* @param component being mocked
* @param variableRegistry variableRegistry
*/
public MockProcessContext(final ConfigurableComponent component, final StateManager stateManager) {
public MockProcessContext(final ConfigurableComponent component, final StateManager stateManager, final VariableRegistry variableRegistry) {
this.component = Objects.requireNonNull(component);
this.stateManager = stateManager;
this.variableRegistry = variableRegistry;
}
public MockProcessContext(final ControllerService component, final MockProcessContext context, final StateManager stateManager) {
this(component, stateManager);
public MockProcessContext(final ControllerService component, final MockProcessContext context, final StateManager stateManager, final VariableRegistry variableRegistry) {
this(component, stateManager, variableRegistry);
try {
annotationData = context.getControllerServiceAnnotationData(component);
@ -105,12 +109,13 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
final String setPropertyValue = properties.get(descriptor);
final String propValue = (setPropertyValue == null) ? descriptor.getDefaultValue() : setPropertyValue;
return new MockPropertyValue(propValue, this, (enableExpressionValidation && allowExpressionValidation) ? descriptor : null);
return new MockPropertyValue(propValue, this, variableRegistry, (enableExpressionValidation && allowExpressionValidation) ? descriptor : null);
}
@Override
public PropertyValue newPropertyValue(final String rawValue) {
return new MockPropertyValue(rawValue, this);
return new MockPropertyValue(rawValue, this, variableRegistry);
}
public ValidationResult setProperty(final String propertyName, final String propertyValue) {
@ -133,7 +138,7 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
requireNonNull(value, "Cannot set property to null value; if the intent is to remove the property, call removeProperty instead");
final PropertyDescriptor fullyPopulatedDescriptor = component.getPropertyDescriptor(descriptor.getName());
final ValidationResult result = fullyPopulatedDescriptor.validate(value, new MockValidationContext(this, stateManager));
final ValidationResult result = fullyPopulatedDescriptor.validate(value, new MockValidationContext(this, stateManager, variableRegistry));
String oldValue = properties.put(fullyPopulatedDescriptor, value);
if (oldValue == null) {
oldValue = fullyPopulatedDescriptor.getDefaultValue();
@ -216,7 +221,7 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
* non-null
*/
public Collection<ValidationResult> validate() {
return component.validate(new MockValidationContext(this, stateManager));
return component.validate(new MockValidationContext(this, stateManager, variableRegistry));
}
public boolean isValid() {

View File

@ -28,6 +28,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;
public class MockPropertyValue implements PropertyValue {
private final String rawValue;
@ -35,24 +36,27 @@ public class MockPropertyValue implements PropertyValue {
private final ControllerServiceLookup serviceLookup;
private final PropertyDescriptor propertyDescriptor;
private final PropertyValue stdPropValue;
private final VariableRegistry variableRegistry;
private boolean expressionsEvaluated = false;
public MockPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup) {
this(rawValue, serviceLookup, null);
public MockPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup, final VariableRegistry variableRegistry) {
this(rawValue, serviceLookup, variableRegistry, null);
}
public MockPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup, final PropertyDescriptor propertyDescriptor) {
this(rawValue, serviceLookup, propertyDescriptor, false);
public MockPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup, VariableRegistry variableRegistry, final PropertyDescriptor propertyDescriptor) {
this(rawValue, serviceLookup, propertyDescriptor, false, variableRegistry);
}
private MockPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup, final PropertyDescriptor propertyDescriptor, final boolean alreadyEvaluated) {
this.stdPropValue = new StandardPropertyValue(rawValue, serviceLookup);
private MockPropertyValue(final String rawValue, final ControllerServiceLookup serviceLookup, final PropertyDescriptor propertyDescriptor, final boolean alreadyEvaluated,
final VariableRegistry variableRegistry) {
this.stdPropValue = new StandardPropertyValue(rawValue, serviceLookup, variableRegistry);
this.rawValue = rawValue;
this.serviceLookup = serviceLookup;
this.expectExpressions = propertyDescriptor == null ? null : propertyDescriptor.isExpressionLanguageSupported();
this.propertyDescriptor = propertyDescriptor;
this.expressionsEvaluated = alreadyEvaluated;
this.variableRegistry = variableRegistry;
}
@ -165,7 +169,7 @@ public class MockPropertyValue implements PropertyValue {
}
final PropertyValue newValue = stdPropValue.evaluateAttributeExpressions(flowFile, additionalAttributes, decorator);
return new MockPropertyValue(newValue.getValue(), serviceLookup, propertyDescriptor, true);
return new MockPropertyValue(newValue.getValue(), serviceLookup, propertyDescriptor, true, variableRegistry);
}
@Override

View File

@ -27,6 +27,7 @@ import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.reporting.Bulletin;
import org.apache.nifi.reporting.BulletinFactory;
import org.apache.nifi.reporting.BulletinRepository;
@ -39,12 +40,14 @@ public class MockReportingContext extends MockControllerServiceLookup implements
private final MockEventAccess eventAccess = new MockEventAccess();
private final Map<PropertyDescriptor, String> properties = new HashMap<>();
private final StateManager stateManager;
private final VariableRegistry variableRegistry;
private final Map<String, List<Bulletin>> componentBulletinsCreated = new HashMap<>();
public MockReportingContext(final Map<String, ControllerService> controllerServices, final StateManager stateManager) {
public MockReportingContext(final Map<String, ControllerService> controllerServices, final StateManager stateManager, final VariableRegistry variableRegistry) {
this.controllerServices = new HashMap<>();
this.stateManager = stateManager;
this.variableRegistry = variableRegistry;
for (final Map.Entry<String, ControllerService> entry : controllerServices.entrySet()) {
this.controllerServices.put(entry.getKey(), new ControllerServiceConfiguration(entry.getValue()));
}
@ -58,7 +61,7 @@ public class MockReportingContext extends MockControllerServiceLookup implements
@Override
public PropertyValue getProperty(final PropertyDescriptor property) {
final String configuredValue = properties.get(property);
return new MockPropertyValue(configuredValue == null ? property.getDefaultValue() : configuredValue, this);
return new MockPropertyValue(configuredValue == null ? property.getDefaultValue() : configuredValue, this, variableRegistry);
}
public void setProperty(final String propertyName, final String value) {

View File

@ -31,16 +31,20 @@ import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.expression.ExpressionLanguageCompiler;
import org.apache.nifi.registry.VariableRegistry;
public class MockValidationContext implements ValidationContext, ControllerServiceLookup {
private final MockProcessContext context;
private final Map<String, Boolean> expressionLanguageSupported;
private final StateManager stateManager;
private final VariableRegistry variableRegistry;
public MockValidationContext(final MockProcessContext processContext, final StateManager stateManager) {
public MockValidationContext(final MockProcessContext processContext, final StateManager stateManager, final VariableRegistry variableRegistry) {
this.context = processContext;
this.stateManager = stateManager;
this.variableRegistry = variableRegistry;
final Map<PropertyDescriptor, String> properties = processContext.getProperties();
expressionLanguageSupported = new HashMap<>(properties.size());
@ -56,18 +60,18 @@ public class MockValidationContext implements ValidationContext, ControllerServi
@Override
public PropertyValue newPropertyValue(final String rawValue) {
return new MockPropertyValue(rawValue, this);
return new MockPropertyValue(rawValue, this, variableRegistry);
}
@Override
public ExpressionLanguageCompiler newExpressionLanguageCompiler() {
return new StandardExpressionLanguageCompiler();
return new StandardExpressionLanguageCompiler(variableRegistry);
}
@Override
public ValidationContext getControllerServiceValidationContext(final ControllerService controllerService) {
final MockProcessContext serviceProcessContext = new MockProcessContext(controllerService, context, stateManager);
return new MockValidationContext(serviceProcessContext, stateManager);
final MockProcessContext serviceProcessContext = new MockProcessContext(controllerService, context, stateManager, variableRegistry);
return new MockValidationContext(serviceProcessContext, stateManager, variableRegistry);
}
@Override
@ -141,4 +145,5 @@ public class MockValidationContext implements ValidationContext, ControllerServi
public String getProcessGroupIdentifier() {
return "unit test";
}
}

View File

@ -65,6 +65,7 @@ import org.apache.nifi.processor.ProcessSessionFactory;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.state.MockStateManager;
import org.junit.Assert;
@ -80,6 +81,7 @@ public class StandardProcessorTestRunner implements TestRunner {
private final boolean triggerSerially;
private final MockStateManager processorStateManager;
private final Map<String, MockStateManager> controllerServiceStateManagers = new HashMap<>();
private final VariableRegistry variableRegistry;
private int numThreads = 1;
private final AtomicInteger invocations = new AtomicInteger(0);
@ -87,14 +89,15 @@ public class StandardProcessorTestRunner implements TestRunner {
private final Map<String, MockComponentLog> controllerServiceLoggers = new HashMap<>();
private final MockComponentLog logger;
StandardProcessorTestRunner(final Processor processor) {
StandardProcessorTestRunner(final Processor processor,final VariableRegistry variableRegistry) {
this.processor = processor;
this.idGenerator = new AtomicLong(0L);
this.sharedState = new SharedSessionState(processor, idGenerator);
this.flowFileQueue = sharedState.getFlowFileQueue();
this.sessionFactory = new MockSessionFactory(sharedState, processor);
this.processorStateManager = new MockStateManager(processor);
this.context = new MockProcessContext(processor, processorStateManager);
this.variableRegistry = variableRegistry;
this.context = new MockProcessContext(processor, processorStateManager, variableRegistry);
final MockProcessorInitializationContext mockInitContext = new MockProcessorInitializationContext(processor, context);
processor.initialize(mockInitContext);
@ -603,7 +606,7 @@ public class StandardProcessorTestRunner implements TestRunner {
throw new IllegalStateException("Controller Service has not been added to this TestRunner via the #addControllerService method");
}
final ValidationContext validationContext = new MockValidationContext(context, serviceStateManager).getControllerServiceValidationContext(service);
final ValidationContext validationContext = new MockValidationContext(context, serviceStateManager, variableRegistry).getControllerServiceValidationContext(service);
final Collection<ValidationResult> results = context.getControllerService(service.getIdentifier()).validate(validationContext);
for (final ValidationResult result : results) {
@ -622,7 +625,7 @@ public class StandardProcessorTestRunner implements TestRunner {
throw new IllegalStateException("Controller Service has not been added to this TestRunner via the #addControllerService method");
}
final ValidationContext validationContext = new MockValidationContext(context, serviceStateManager).getControllerServiceValidationContext(service);
final ValidationContext validationContext = new MockValidationContext(context, serviceStateManager, variableRegistry).getControllerServiceValidationContext(service);
final Collection<ValidationResult> results = context.getControllerService(service.getIdentifier()).validate(validationContext);
for (final ValidationResult result : results) {
@ -665,7 +668,7 @@ public class StandardProcessorTestRunner implements TestRunner {
}
try {
final ConfigurationContext configContext = new MockConfigurationContext(service, configuration.getProperties(), context);
final ConfigurationContext configContext = new MockConfigurationContext(service, configuration.getProperties(), context,variableRegistry);
ReflectionUtils.invokeMethodsWithAnnotation(OnEnabled.class, service, configContext);
} catch (final InvocationTargetException ite) {
ite.getCause().printStackTrace();
@ -737,7 +740,7 @@ public class StandardProcessorTestRunner implements TestRunner {
final Map<PropertyDescriptor, String> curProps = configuration.getProperties();
final Map<PropertyDescriptor, String> updatedProps = new HashMap<>(curProps);
final ValidationContext validationContext = new MockValidationContext(context, serviceStateManager).getControllerServiceValidationContext(service);
final ValidationContext validationContext = new MockValidationContext(context, serviceStateManager, variableRegistry).getControllerServiceValidationContext(service);
final ValidationResult validationResult = property.validate(value, validationContext);
updatedProps.put(property, value);

View File

@ -17,11 +17,17 @@
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 new StandardProcessorTestRunner(processor);
return newTestRunner(processor,VariableRegistryUtils.createSystemVariableRegistry());
}
public static TestRunner newTestRunner(final Processor processor, VariableRegistry variableRegistry){
return new StandardProcessorTestRunner(processor, variableRegistry);
}
public static TestRunner newTestRunner(final Class<? extends Processor> processorClass) {

View File

@ -20,6 +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.junit.Assert;
import org.junit.Test;
@ -31,7 +32,7 @@ public class CurrentTestStandardProcessorTestRunner {
@Test
public void testOnScheduledCalledAfterRunFinished() {
SlowRunProcessor processor = new SlowRunProcessor();
StandardProcessorTestRunner runner = new StandardProcessorTestRunner(processor);
StandardProcessorTestRunner runner = new StandardProcessorTestRunner(processor, VariableRegistryUtils.createSystemVariableRegistry());
final int iterations = 5;
runner.run(iterations);
// if the counter is not equal to iterations, the the processor must have been unscheduled

View File

@ -32,6 +32,7 @@ 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 {
@ -39,7 +40,7 @@ public class TestMockProcessContext {
@Test
public void testRemoveProperty() {
final DummyProcessor proc = new DummyProcessor();
final MockProcessContext context = new MockProcessContext(proc);
final MockProcessContext context = new MockProcessContext(proc, VariableRegistryUtils.createSystemVariableRegistry());
context.setProperty(DummyProcessor.REQUIRED_PROP, "req-value");
context.setProperty(DummyProcessor.OPTIONAL_PROP, "opt-value");
context.setProperty(DummyProcessor.DEFAULTED_PROP, "custom-value");

View File

@ -20,6 +20,8 @@ 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;
@ -43,6 +45,7 @@ import java.util.UUID;
public class TestAmbariReportingTask {
private ProcessGroupStatus status;
private VariableRegistry variableRegistry;
@Before
public void setup() {
@ -73,6 +76,7 @@ public class TestAmbariReportingTask {
Collection<ProcessGroupStatus> groupStatuses = new ArrayList<>();
groupStatuses.add(groupStatus);
status.setProcessGroupStatus(groupStatuses);
variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
}
@Test
@ -105,11 +109,12 @@ 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));
.thenReturn(new MockPropertyValue(metricsUrl, null, variableRegistry));
Mockito.when(context.getProperty(AmbariReportingTask.APPLICATION_ID))
.thenReturn(new MockPropertyValue(applicationId, null));
.thenReturn(new MockPropertyValue(applicationId, null, variableRegistry));
Mockito.when(context.getProperty(AmbariReportingTask.HOSTNAME))
.thenReturn(new MockPropertyValue(hostName, null));
.thenReturn(new MockPropertyValue(hostName, null, variableRegistry));
final EventAccess eventAccess = Mockito.mock(EventAccess.class);
Mockito.when(context.getEventAccess()).thenReturn(eventAccess);
@ -121,7 +126,6 @@ public class TestAmbariReportingTask {
task.setup(configurationContext);
task.onTrigger(context);
}
// override the creation of the client to provide a mock
private class TestableAmbariReportingTask extends AmbariReportingTask {

View File

@ -18,6 +18,8 @@ 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;
@ -62,11 +64,13 @@ 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();
}
@ -216,7 +220,7 @@ public class TestFetchElasticsearch {
}
};
MockProcessContext context = new MockProcessContext(processor);
MockProcessContext context = new MockProcessContext(processor, variableRegistry);
processor.initialize(new MockProcessorInitializationContext(processor, context));
processor.callCreateElasticsearchClient(context);
}

View File

@ -25,6 +25,7 @@ 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;
@ -72,8 +73,10 @@ public class AuthorizerFactoryBean implements FactoryBean, DisposableBean, Autho
private Authorizer authorizer;
private NiFiProperties properties;
private VariableRegistry variableRegistry;
private final Map<String, Authorizer> authorizers = new HashMap<>();
@Override
public Authorizer getAuthorizer(String identifier) {
return authorizers.get(identifier);
@ -189,8 +192,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);
return new StandardAuthorizerConfigurationContext(authorizer.getIdentifier(), authorizerProperties, variableRegistry);
}
private void performMethodInjection(final Authorizer instance, final Class authorizerClass) throws IllegalAccessException, IllegalArgumentException, InvocationTargetException {
@ -484,4 +486,8 @@ public class AuthorizerFactoryBean implements FactoryBean, DisposableBean, Autho
public void setProperties(NiFiProperties properties) {
this.properties = properties;
}
public void setVariableRegistry(VariableRegistry variableRegistry) {
this.variableRegistry = variableRegistry;
}
}

View File

@ -21,6 +21,7 @@
<!-- user/entity authorizer -->
<bean id="authorizer" class="org.apache.nifi.authorization.AuthorizerFactoryBean">
<property name="properties" ref="nifiProperties"/>
<property name="variableRegistry" ref="variableRegistry"/>
</bean>
</beans>

View File

@ -21,6 +21,8 @@ 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;
@ -143,6 +145,7 @@ public class FileAuthorizerTest {
private File flow;
private File flowNoPorts;
private File flowWithDns;
private VariableRegistry variableRegistry;
private AuthorizerConfigurationContext configurationContext;
@ -178,8 +181,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));
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_TENANTS_FILE))).thenReturn(new StandardPropertyValue(primaryTenants.getPath(), null));
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));
authorizer = new FileAuthorizer();
authorizer.setNiFiProperties(properties);
@ -197,7 +200,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));
.thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-multirole.xml", null,VariableRegistryUtils.createSystemVariableRegistry()));
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@ -222,7 +225,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));
.thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null,variableRegistry));
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@ -242,7 +245,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));
.thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null,VariableRegistryUtils.createSystemVariableRegistry()));
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@ -392,7 +395,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));
.thenReturn(new StandardPropertyValue("src/test/resources/authorized-users-with-dns.xml", null,variableRegistry));
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@ -440,7 +443,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));
.thenReturn(new StandardPropertyValue("src/test/resources/does-not-exist.xml", null,VariableRegistryUtils.createSystemVariableRegistry()));
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@ -450,12 +453,12 @@ 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));
.thenReturn(new StandardPropertyValue(adminIdentity, null,variableRegistry));
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)))
.thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null));
.thenReturn(new StandardPropertyValue("src/test/resources/authorized-users.xml", null,variableRegistry));
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@ -480,7 +483,7 @@ public class FileAuthorizerTest {
final String adminIdentity = "admin-user";
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_INITIAL_ADMIN_IDENTITY)))
.thenReturn(new StandardPropertyValue(adminIdentity, null));
.thenReturn(new StandardPropertyValue(adminIdentity, null,VariableRegistryUtils.createSystemVariableRegistry()));
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@ -510,6 +513,7 @@ 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());
@ -518,7 +522,7 @@ public class FileAuthorizerTest {
final String adminIdentity = "admin-user";
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_INITIAL_ADMIN_IDENTITY)))
.thenReturn(new StandardPropertyValue(adminIdentity, null));
.thenReturn(new StandardPropertyValue(adminIdentity, null, variableRegistry));
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@ -549,6 +553,7 @@ 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);
@ -556,7 +561,7 @@ public class FileAuthorizerTest {
final String adminIdentity = "admin-user";
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_INITIAL_ADMIN_IDENTITY)))
.thenReturn(new StandardPropertyValue(adminIdentity, null));
.thenReturn(new StandardPropertyValue(adminIdentity, null, variableRegistry));
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@ -597,7 +602,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));
.thenReturn(new StandardPropertyValue(adminIdentity, null,variableRegistry));
writeFile(primaryAuthorizations, EMPTY_AUTHORIZATIONS_CONCISE);
writeFile(primaryTenants, EMPTY_TENANTS_CONCISE);
@ -615,7 +620,7 @@ public class FileAuthorizerTest {
final String adminIdentity = "admin-user";
when(configurationContext.getProperty(Mockito.eq(FileAuthorizer.PROP_INITIAL_ADMIN_IDENTITY)))
.thenReturn(new StandardPropertyValue(adminIdentity, null));
.thenReturn(new StandardPropertyValue(adminIdentity, null,VariableRegistryUtils.createSystemVariableRegistry()));
final String nodeIdentity1 = "node1";
final String nodeIdentity2 = "node2";
@ -664,7 +669,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));
.thenReturn(new StandardPropertyValue(adminIdentity, null,variableRegistry));
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";

View File

@ -18,6 +18,7 @@ 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;
@ -30,10 +31,12 @@ 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) {
public StandardAuthorizerConfigurationContext(String identifier, Map<String, String> properties, VariableRegistry variableRegistry) {
this.identifier = identifier;
this.properties = Collections.unmodifiableMap(new HashMap<String, String>(properties));
this.variableRegistry = variableRegistry;
}
@Override
@ -48,7 +51,7 @@ public class StandardAuthorizerConfigurationContext implements AuthorizerConfigu
@Override
public PropertyValue getProperty(String property) {
return new StandardPropertyValue(properties.get(property), null);
return new StandardPropertyValue(properties.get(property), null,variableRegistry);
}
}

View File

@ -158,6 +158,7 @@ import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceEventType;
import org.apache.nifi.provenance.ProvenanceRepository;
import org.apache.nifi.provenance.StandardProvenanceEventRecord;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.remote.HttpRemoteSiteListener;
import org.apache.nifi.remote.RemoteGroupPort;
import org.apache.nifi.remote.RemoteResourceManager;
@ -286,6 +287,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
private final StateManagerProvider stateManagerProvider;
private final long systemStartTime = System.currentTimeMillis(); // time at which the node was started
private final ConcurrentMap<String, ReportingTaskNode> reportingTasks = new ConcurrentHashMap<>();
private final VariableRegistry variableRegistry;
private final ConcurrentMap<String, ControllerServiceNode> rootControllerServices = new ConcurrentHashMap<>();
private volatile ZooKeeperStateServer zooKeeperStateServer;
@ -368,12 +370,13 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
private static final Logger heartbeatLogger = LoggerFactory.getLogger("org.apache.nifi.cluster.heartbeat");
public static FlowController createStandaloneInstance(
final FlowFileEventRepository flowFileEventRepo,
final NiFiProperties properties,
final Authorizer authorizer,
final AuditService auditService,
final StringEncryptor encryptor,
final BulletinRepository bulletinRepo) {
final FlowFileEventRepository flowFileEventRepo,
final NiFiProperties properties,
final Authorizer authorizer,
final AuditService auditService,
final StringEncryptor encryptor,
final BulletinRepository bulletinRepo, VariableRegistry variableRegistry) {
return new FlowController(
flowFileEventRepo,
properties,
@ -384,19 +387,21 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
/* NodeProtocolSender */ null,
bulletinRepo,
/* cluster coordinator */ null,
/* heartbeat monitor */ null);
/* heartbeat monitor */ null, variableRegistry);
}
public static FlowController createClusteredInstance(
final FlowFileEventRepository flowFileEventRepo,
final NiFiProperties properties,
final Authorizer authorizer,
final AuditService auditService,
final StringEncryptor encryptor,
final NodeProtocolSender protocolSender,
final BulletinRepository bulletinRepo,
final ClusterCoordinator clusterCoordinator,
final HeartbeatMonitor heartbeatMonitor) {
final FlowFileEventRepository flowFileEventRepo,
final NiFiProperties properties,
final Authorizer authorizer,
final AuditService auditService,
final StringEncryptor encryptor,
final NodeProtocolSender protocolSender,
final BulletinRepository bulletinRepo,
final ClusterCoordinator clusterCoordinator,
final HeartbeatMonitor heartbeatMonitor,
VariableRegistry variableRegistry) {
final FlowController flowController = new FlowController(
flowFileEventRepo,
properties,
@ -407,22 +412,23 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
protocolSender,
bulletinRepo,
clusterCoordinator,
heartbeatMonitor);
heartbeatMonitor, variableRegistry);
return flowController;
}
private FlowController(
final FlowFileEventRepository flowFileEventRepo,
final NiFiProperties properties,
final Authorizer authorizer,
final AuditService auditService,
final StringEncryptor encryptor,
final boolean configuredForClustering,
final NodeProtocolSender protocolSender,
final BulletinRepository bulletinRepo,
final ClusterCoordinator clusterCoordinator,
final HeartbeatMonitor heartbeatMonitor) {
final FlowFileEventRepository flowFileEventRepo,
final NiFiProperties properties,
final Authorizer authorizer,
final AuditService auditService,
final StringEncryptor encryptor,
final boolean configuredForClustering,
final NodeProtocolSender protocolSender,
final BulletinRepository bulletinRepo,
final ClusterCoordinator clusterCoordinator,
final HeartbeatMonitor heartbeatMonitor,
VariableRegistry variableRegistry) {
maxTimerDrivenThreads = new AtomicInteger(10);
maxEventDrivenThreads = new AtomicInteger(5);
@ -443,6 +449,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
counterRepositoryRef = new AtomicReference<CounterRepository>(new StandardCounterRepository());
bulletinRepository = bulletinRepo;
this.variableRegistry = variableRegistry;
try {
this.provenanceRepository = createProvenanceRepository(properties);
@ -458,20 +466,20 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
}
try {
this.stateManagerProvider = StandardStateManagerProvider.create(properties);
this.stateManagerProvider = StandardStateManagerProvider.create(properties, this.variableRegistry);
} catch (final IOException e) {
throw new RuntimeException(e);
}
processScheduler = new StandardProcessScheduler(this, encryptor, stateManagerProvider);
processScheduler = new StandardProcessScheduler(this, encryptor, stateManagerProvider, this.variableRegistry);
eventDrivenWorkerQueue = new EventDrivenWorkerQueue(false, false, processScheduler);
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));
eventDrivenEngineRef.get(), this, stateManagerProvider, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor, this.variableRegistry));
final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor);
final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor);
final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor, this.variableRegistry);
final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor, this.variableRegistry);
processScheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, timerDrivenAgent);
processScheduler.setSchedulingAgent(SchedulingStrategy.PRIMARY_NODE_ONLY, timerDrivenAgent);
processScheduler.setSchedulingAgent(SchedulingStrategy.CRON_DRIVEN, quartzSchedulingAgent);
@ -507,11 +515,11 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
this.snippetManager = new SnippetManager();
rootGroup = new StandardProcessGroup(UUID.randomUUID().toString(), this, processScheduler, properties, encryptor, this);
rootGroup = new StandardProcessGroup(UUID.randomUUID().toString(), this, processScheduler, properties, encryptor, this, this.variableRegistry);
rootGroup.setName(DEFAULT_ROOT_GROUP_NAME);
instanceId = UUID.randomUUID().toString();
controllerServiceProvider = new StandardControllerServiceProvider(this, processScheduler, bulletinRepository, stateManagerProvider);
controllerServiceProvider = new StandardControllerServiceProvider(this, processScheduler, bulletinRepository, stateManagerProvider, this.variableRegistry);
if (remoteInputSocketPort == null) {
LOG.info("Not enabling RAW Socket Site-to-Site functionality because nifi.remote.input.socket.port is not set");
@ -971,7 +979,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
* @throws NullPointerException if the argument is null
*/
public ProcessGroup createProcessGroup(final String id) {
return new StandardProcessGroup(requireNonNull(id).intern(), this, processScheduler, properties, encryptor, this);
return new StandardProcessGroup(requireNonNull(id).intern(), this, processScheduler, properties, encryptor, this, variableRegistry);
}
/**
@ -1018,7 +1026,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
creationSuccessful = false;
}
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider);
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, variableRegistry);
final ProcessorNode procNode;
if (creationSuccessful) {
procNode = new StandardProcessorNode(processor, id, validationContextFactory, processScheduler, controllerServiceProvider);
@ -1295,7 +1303,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
// invoke any methods annotated with @OnShutdown on Controller Services
for (final ControllerServiceNode serviceNode : getAllControllerServices()) {
try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
final ConfigurationContext configContext = new StandardConfigurationContext(serviceNode, controllerServiceProvider, null);
final ConfigurationContext configContext = new StandardConfigurationContext(serviceNode, controllerServiceProvider, null, variableRegistry);
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, serviceNode.getControllerServiceImplementation(), configContext);
}
}
@ -2785,15 +2793,15 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
}
}
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider);
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider,variableRegistry);
final ReportingTaskNode taskNode;
if (creationSuccessful) {
taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory);
taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, variableRegistry);
} else {
final String simpleClassName = type.contains(".") ? StringUtils.substringAfterLast(type, ".") : type;
final String componentType = "(Missing) " + simpleClassName;
taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, componentType, type);
taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, componentType, type,variableRegistry);
}
taskNode.setName(task.getClass().getSimpleName());
@ -3018,7 +3026,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
service.verifyCanDelete();
try (final NarCloseable x = NarCloseable.withNarLoader()) {
final ConfigurationContext configurationContext = new StandardConfigurationContext(service, controllerServiceProvider, null);
final ConfigurationContext configurationContext = new StandardConfigurationContext(service, controllerServiceProvider, null,variableRegistry);
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, service.getControllerServiceImplementation(), configurationContext);
}

View File

@ -33,6 +33,7 @@ import org.apache.nifi.controller.ValidationContextFactory;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.controller.service.StandardConfigurationContext;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.reporting.ReportingTask;
import org.apache.nifi.scheduling.SchedulingStrategy;
import org.apache.nifi.util.FormatUtils;
@ -49,24 +50,27 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon
private volatile String comment;
private volatile ScheduledState scheduledState = ScheduledState.STOPPED;
protected final VariableRegistry variableRegistry;
public AbstractReportingTaskNode(final ReportingTask reportingTask, final String id,
final ControllerServiceProvider controllerServiceProvider, final ProcessScheduler processScheduler,
final ValidationContextFactory validationContextFactory) {
final ValidationContextFactory validationContextFactory, final VariableRegistry variableRegistry) {
this(reportingTask, id, controllerServiceProvider, processScheduler, validationContextFactory,
reportingTask.getClass().getSimpleName(), reportingTask.getClass().getCanonicalName());
reportingTask.getClass().getSimpleName(), reportingTask.getClass().getCanonicalName(),variableRegistry);
}
public AbstractReportingTaskNode(final ReportingTask reportingTask, final String id,
final ControllerServiceProvider controllerServiceProvider, final ProcessScheduler processScheduler,
final ValidationContextFactory validationContextFactory,
final String componentType, final String componentCanonicalClass) {
final String componentType, final String componentCanonicalClass, VariableRegistry variableRegistry) {
super(reportingTask, id, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass);
this.reportingTask = reportingTask;
this.processScheduler = processScheduler;
this.serviceLookup = controllerServiceProvider;
this.variableRegistry = variableRegistry;
}
@Override
@ -111,7 +115,7 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon
@Override
public ConfigurationContext getConfigurationContext() {
return new StandardConfigurationContext(this, serviceLookup, getSchedulingPeriod());
return new StandardConfigurationContext(this, serviceLookup, getSchedulingPeriod(), variableRegistry);
}
@Override
@ -141,6 +145,7 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon
return super.removeProperty(name);
}
public boolean isDisabled() {
return scheduledState == ScheduledState.DISABLED;
}

View File

@ -34,6 +34,7 @@ import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.events.BulletinFactory;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.reporting.Bulletin;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.reporting.EventAccess;
@ -50,16 +51,18 @@ public class StandardReportingContext implements ReportingContext, ControllerSer
private final ControllerServiceProvider serviceProvider;
private final Map<PropertyDescriptor, String> properties;
private final Map<PropertyDescriptor, PreparedQuery> preparedQueries;
private final VariableRegistry variableRegistry;
public StandardReportingContext(final FlowController flowController, final BulletinRepository bulletinRepository,
final Map<PropertyDescriptor, String> properties, final ControllerServiceProvider serviceProvider, final ReportingTask reportingTask) {
final Map<PropertyDescriptor, String> properties, final ControllerServiceProvider serviceProvider, final ReportingTask reportingTask,
final VariableRegistry variableRegistry) {
this.flowController = flowController;
this.eventAccess = flowController;
this.bulletinRepository = bulletinRepository;
this.properties = Collections.unmodifiableMap(properties);
this.serviceProvider = serviceProvider;
this.reportingTask = reportingTask;
this.variableRegistry = variableRegistry;
preparedQueries = new HashMap<>();
for (final Map.Entry<PropertyDescriptor, String> entry : properties.entrySet()) {
final PropertyDescriptor desc = entry.getKey();
@ -106,7 +109,7 @@ public class StandardReportingContext implements ReportingContext, ControllerSer
@Override
public PropertyValue getProperty(final PropertyDescriptor property) {
final String configuredValue = properties.get(property);
return new StandardPropertyValue(configuredValue == null ? property.getDefaultValue() : configuredValue, this, preparedQueries.get(property));
return new StandardPropertyValue(configuredValue == null ? property.getDefaultValue() : configuredValue, this, preparedQueries.get(property), variableRegistry);
}
@Override
@ -148,4 +151,5 @@ public class StandardReportingContext implements ReportingContext, ControllerSer
public StateManager getStateManager() {
return flowController.getStateManagerProvider().getStateManager(reportingTask.getIdentifier());
}
}

View File

@ -24,6 +24,7 @@ import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.ProcessScheduler;
import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.ValidationContextFactory;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.reporting.ReportingContext;
import org.apache.nifi.reporting.ReportingTask;
@ -32,15 +33,16 @@ public class StandardReportingTaskNode extends AbstractReportingTaskNode impleme
private final FlowController flowController;
public StandardReportingTaskNode(final ReportingTask reportingTask, final String id, final FlowController controller,
final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory) {
super(reportingTask, id, controller, processScheduler, validationContextFactory);
final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory,
final VariableRegistry variableRegistry) {
super(reportingTask, id, controller, processScheduler, validationContextFactory, variableRegistry);
this.flowController = controller;
}
public StandardReportingTaskNode(final ReportingTask reportingTask, final String id, final FlowController controller,
final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory,
final String componentType, final String canonicalClassName) {
super(reportingTask, id, controller, processScheduler, validationContextFactory, componentType, canonicalClassName);
final String componentType, final String canonicalClassName, VariableRegistry variableRegistry) {
super(reportingTask, id, controller, processScheduler, validationContextFactory, componentType, canonicalClassName,variableRegistry);
this.flowController = controller;
}
@ -56,6 +58,6 @@ public class StandardReportingTaskNode extends AbstractReportingTaskNode impleme
@Override
public ReportingContext getReportingContext() {
return new StandardReportingContext(flowController, flowController.getBulletinRepository(), getProperties(), flowController, getReportingTask());
return new StandardReportingContext(flowController, flowController.getBulletinRepository(), getProperties(), flowController, getReportingTask(), variableRegistry);
}
}

View File

@ -45,6 +45,7 @@ import org.apache.nifi.processor.ProcessSessionFactory;
import org.apache.nifi.processor.SimpleProcessLogger;
import org.apache.nifi.processor.StandardProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.util.Connectables;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.ReflectionUtils;
@ -61,6 +62,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
private final ProcessContextFactory contextFactory;
private final AtomicInteger maxThreadCount;
private final StringEncryptor encryptor;
private final VariableRegistry variableRegistry;
private volatile String adminYieldDuration = "1 sec";
@ -68,7 +70,8 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
private final ConcurrentMap<Connectable, ScheduleState> scheduleStates = new ConcurrentHashMap<>();
public EventDrivenSchedulingAgent(final FlowEngine flowEngine, final ControllerServiceProvider serviceProvider, final StateManagerProvider stateManagerProvider,
final EventDrivenWorkerQueue workerQueue, final ProcessContextFactory contextFactory, final int maxThreadCount, final StringEncryptor encryptor) {
final EventDrivenWorkerQueue workerQueue, final ProcessContextFactory contextFactory, final int maxThreadCount, final StringEncryptor encryptor,
final VariableRegistry variableRegistry) {
super(flowEngine);
this.serviceProvider = serviceProvider;
this.stateManagerProvider = stateManagerProvider;
@ -76,6 +79,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
this.contextFactory = contextFactory;
this.maxThreadCount = new AtomicInteger(maxThreadCount);
this.encryptor = encryptor;
this.variableRegistry = variableRegistry;
for (int i = 0; i < maxThreadCount; i++) {
final Runnable eventDrivenTask = new EventDrivenTask(workerQueue);
@ -185,7 +189,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
if (connectable instanceof ProcessorNode) {
final ProcessorNode procNode = (ProcessorNode) connectable;
final StandardProcessContext standardProcessContext = new StandardProcessContext(procNode, serviceProvider,
encryptor, getStateManager(connectable.getIdentifier()));
encryptor, getStateManager(connectable.getIdentifier()), variableRegistry);
final long runNanos = procNode.getRunDuration(TimeUnit.NANOSECONDS);
final ProcessSessionFactory sessionFactory;

View File

@ -38,6 +38,7 @@ import org.apache.nifi.encrypt.StringEncryptor;
import org.apache.nifi.engine.FlowEngine;
import org.apache.nifi.processor.StandardProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.util.FormatUtils;
import org.quartz.CronExpression;
import org.slf4j.Logger;
@ -50,15 +51,18 @@ public class QuartzSchedulingAgent extends AbstractSchedulingAgent {
private final FlowController flowController;
private final ProcessContextFactory contextFactory;
private final StringEncryptor encryptor;
private final VariableRegistry variableRegistry;
private volatile String adminYieldDuration = "1 sec";
private final Map<Object, List<AtomicBoolean>> canceledTriggers = new HashMap<>();
public QuartzSchedulingAgent(final FlowController flowController, final FlowEngine flowEngine, final ProcessContextFactory contextFactory, final StringEncryptor enryptor) {
public QuartzSchedulingAgent(final FlowController flowController, final FlowEngine flowEngine, final ProcessContextFactory contextFactory, final StringEncryptor enryptor,
final VariableRegistry variableRegistry) {
super(flowEngine);
this.flowController = flowController;
this.contextFactory = contextFactory;
this.encryptor = enryptor;
this.variableRegistry = variableRegistry;
}
private StateManager getStateManager(final String componentId) {
@ -141,7 +145,7 @@ public class QuartzSchedulingAgent extends AbstractSchedulingAgent {
if (connectable.getConnectableType() == ConnectableType.PROCESSOR) {
final ProcessorNode procNode = (ProcessorNode) connectable;
final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor, getStateManager(connectable.getIdentifier()));
final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor, getStateManager(connectable.getIdentifier()), variableRegistry);
ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController, contextFactory, scheduleState, standardProcContext);
continuallyRunTask = runnableTask;
} else {

View File

@ -52,6 +52,7 @@ import org.apache.nifi.nar.NarCloseable;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.processor.SimpleProcessLogger;
import org.apache.nifi.processor.StandardProcessContext;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.reporting.ReportingTask;
import org.apache.nifi.scheduling.SchedulingStrategy;
import org.apache.nifi.util.FormatUtils;
@ -81,12 +82,14 @@ public final class StandardProcessScheduler implements ProcessScheduler {
private final ScheduledExecutorService componentMonitoringThreadPool = new FlowEngine(8, "StandardProcessScheduler", true);
private final StringEncryptor encryptor;
private final VariableRegistry variableRegistry;
public StandardProcessScheduler(final ControllerServiceProvider controllerServiceProvider, final StringEncryptor encryptor,
final StateManagerProvider stateManagerProvider) {
final StateManagerProvider stateManagerProvider, final VariableRegistry variableRegistry) {
this.controllerServiceProvider = controllerServiceProvider;
this.encryptor = encryptor;
this.stateManagerProvider = stateManagerProvider;
this.variableRegistry = variableRegistry;
administrativeYieldDuration = NiFiProperties.getInstance().getAdministrativeYieldDuration();
administrativeYieldMillis = FormatUtils.getTimeDuration(administrativeYieldDuration, TimeUnit.MILLISECONDS);
@ -290,7 +293,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
@Override
public synchronized void startProcessor(final ProcessorNode procNode) {
StandardProcessContext processContext = new StandardProcessContext(procNode, this.controllerServiceProvider,
this.encryptor, getStateManager(procNode.getIdentifier()));
this.encryptor, getStateManager(procNode.getIdentifier()), variableRegistry);
final ScheduleState scheduleState = getScheduleState(requireNonNull(procNode));
SchedulingAgentCallback callback = new SchedulingAgentCallback() {
@ -324,7 +327,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
@Override
public synchronized void stopProcessor(final ProcessorNode procNode) {
StandardProcessContext processContext = new StandardProcessContext(procNode, this.controllerServiceProvider,
this.encryptor, getStateManager(procNode.getIdentifier()));
this.encryptor, getStateManager(procNode.getIdentifier()), variableRegistry);
final ScheduleState state = getScheduleState(procNode);
procNode.stop(this.componentLifeCycleThreadPool, processContext, new Callable<Boolean>() {

View File

@ -37,6 +37,7 @@ import org.apache.nifi.engine.FlowEngine;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.StandardProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.slf4j.Logger;
@ -50,14 +51,17 @@ public class TimerDrivenSchedulingAgent extends AbstractSchedulingAgent {
private final FlowController flowController;
private final ProcessContextFactory contextFactory;
private final StringEncryptor encryptor;
private final VariableRegistry variableRegistry;
private volatile String adminYieldDuration = "1 sec";
public TimerDrivenSchedulingAgent(final FlowController flowController, final FlowEngine flowEngine, final ProcessContextFactory contextFactory, final StringEncryptor encryptor) {
public TimerDrivenSchedulingAgent(final FlowController flowController, final FlowEngine flowEngine, final ProcessContextFactory contextFactory, final StringEncryptor encryptor,
final VariableRegistry variableRegistry) {
super(flowEngine);
this.flowController = flowController;
this.contextFactory = contextFactory;
this.encryptor = encryptor;
this.variableRegistry = variableRegistry;
final String boredYieldDuration = NiFiProperties.getInstance().getBoredYieldDuration();
try {
@ -100,7 +104,7 @@ public class TimerDrivenSchedulingAgent extends AbstractSchedulingAgent {
// Determine the task to run and create it.
if (connectable.getConnectableType() == ConnectableType.PROCESSOR) {
final ProcessorNode procNode = (ProcessorNode) connectable;
final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor, getStateManager(connectable.getIdentifier()));
final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor, getStateManager(connectable.getIdentifier()), variableRegistry);
final ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController,
contextFactory, scheduleState, standardProcContext);

View File

@ -28,6 +28,7 @@ import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.controller.ConfiguredComponent;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.util.FormatUtils;
public class StandardConfigurationContext implements ConfigurationContext {
@ -35,13 +36,17 @@ public class StandardConfigurationContext implements ConfigurationContext {
private final ConfiguredComponent component;
private final ControllerServiceLookup serviceLookup;
private final Map<PropertyDescriptor, PreparedQuery> preparedQueries;
private final VariableRegistry variableRegistry;
private final String schedulingPeriod;
private final Long schedulingNanos;
public StandardConfigurationContext(final ConfiguredComponent component, final ControllerServiceLookup serviceLookup, final String schedulingPeriod) {
public StandardConfigurationContext(final ConfiguredComponent component, final ControllerServiceLookup serviceLookup, final String schedulingPeriod,
final VariableRegistry variableRegistry) {
this.component = component;
this.serviceLookup = serviceLookup;
this.schedulingPeriod = schedulingPeriod;
this.variableRegistry = variableRegistry;
if (schedulingPeriod == null) {
schedulingNanos = null;
} else {
@ -68,7 +73,7 @@ public class StandardConfigurationContext implements ConfigurationContext {
@Override
public PropertyValue getProperty(final PropertyDescriptor property) {
final String configuredValue = component.getProperty(property);
return new StandardPropertyValue(configuredValue == null ? property.getDefaultValue() : configuredValue, serviceLookup, preparedQueries.get(property));
return new StandardPropertyValue(configuredValue == null ? property.getDefaultValue() : configuredValue, serviceLookup, preparedQueries.get(property), variableRegistry);
}
@Override

View File

@ -48,6 +48,7 @@ import org.apache.nifi.controller.ValidationContextFactory;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.SimpleProcessLogger;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.util.ReflectionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -59,7 +60,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
private final ControllerService proxedControllerService;
private final ControllerService implementation;
private final ControllerServiceProvider serviceProvider;
private final VariableRegistry variableRegistry;
private final AtomicReference<ControllerServiceState> stateRef = new AtomicReference<>(ControllerServiceState.DISABLED);
private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
@ -73,21 +74,24 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
private final AtomicBoolean active;
public StandardControllerServiceNode(final ControllerService proxiedControllerService, final ControllerService implementation, final String id,
final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider) {
final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider,
final VariableRegistry variableRegistry) {
this(proxiedControllerService, implementation, id, validationContextFactory, serviceProvider,
implementation.getClass().getSimpleName(), implementation.getClass().getCanonicalName());
implementation.getClass().getSimpleName(), implementation.getClass().getCanonicalName(), variableRegistry);
}
public StandardControllerServiceNode(final ControllerService proxiedControllerService, final ControllerService implementation, final String id,
final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider,
final String componentType, final String componentCanonicalClass) {
final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider,
final String componentType, final String componentCanonicalClass, VariableRegistry variableRegistry) {
super(implementation, id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass);
this.proxedControllerService = proxiedControllerService;
this.implementation = implementation;
this.serviceProvider = serviceProvider;
this.active = new AtomicBoolean();
this.variableRegistry = variableRegistry;
}
@Override
@ -335,7 +339,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
public void enable(final ScheduledExecutorService scheduler, final long administrativeYieldMillis) {
if (this.stateRef.compareAndSet(ControllerServiceState.DISABLED, ControllerServiceState.ENABLING)) {
this.active.set(true);
final ConfigurationContext configContext = new StandardConfigurationContext(this, this.serviceProvider, null);
final ConfigurationContext configContext = new StandardConfigurationContext(this, this.serviceProvider, null, variableRegistry);
scheduler.execute(new Runnable() {
@Override
public void run() {
@ -398,7 +402,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
}
if (this.stateRef.compareAndSet(ControllerServiceState.ENABLED, ControllerServiceState.DISABLING)) {
final ConfigurationContext configContext = new StandardConfigurationContext(this, this.serviceProvider, null);
final ConfigurationContext configContext = new StandardConfigurationContext(this, this.serviceProvider, null, variableRegistry);
scheduler.execute(new Runnable() {
@Override
public void run() {

View File

@ -55,6 +55,8 @@ import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.NarCloseable;
import org.apache.nifi.processor.SimpleProcessLogger;
import org.apache.nifi.processor.StandardValidationContextFactory;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.reporting.Severity;
import org.apache.nifi.util.ReflectionUtils;
@ -69,6 +71,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
private static final Set<Method> validDisabledMethods;
private final BulletinRepository bulletinRepo;
private final StateManagerProvider stateManagerProvider;
private final VariableRegistry variableRegistry;
private final FlowController flowController;
static {
@ -84,12 +87,13 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
}
public StandardControllerServiceProvider(final FlowController flowController, final ProcessScheduler scheduler, final BulletinRepository bulletinRepo,
final StateManagerProvider stateManagerProvider) {
final StateManagerProvider stateManagerProvider,final VariableRegistry variableRegistry) {
this.flowController = flowController;
this.processScheduler = scheduler;
this.bulletinRepo = bulletinRepo;
this.stateManagerProvider = stateManagerProvider;
this.variableRegistry = variableRegistry;
}
@ -187,9 +191,9 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
final ComponentLog serviceLogger = new SimpleProcessLogger(id, originalService);
originalService.initialize(new StandardControllerServiceInitializationContext(id, serviceLogger, this, getStateManager(id)));
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(this);
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(this, variableRegistry);
final ControllerServiceNode serviceNode = new StandardControllerServiceNode(proxiedService, originalService, id, validationContextFactory, this);
final ControllerServiceNode serviceNode = new StandardControllerServiceNode(proxiedService, originalService, id, validationContextFactory, this, variableRegistry);
serviceNodeHolder.set(serviceNode);
serviceNode.setName(rawClass.getSimpleName());
@ -258,7 +262,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
final String componentType = "(Missing) " + simpleClassName;
final ControllerServiceNode serviceNode = new StandardControllerServiceNode(proxiedService, proxiedService, id,
new StandardValidationContextFactory(this), this, componentType, type);
new StandardValidationContextFactory(this,variableRegistry), this, componentType, type, variableRegistry);
return serviceNode;
}

View File

@ -46,11 +46,12 @@ import org.apache.nifi.controller.state.config.StateProviderConfiguration;
import org.apache.nifi.framework.security.util.SslContextFactory;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.processor.StandardValidationContext;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.util.NiFiProperties;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class StandardStateManagerProvider implements StateManagerProvider {
public class StandardStateManagerProvider implements StateManagerProvider{
private static final Logger logger = LoggerFactory.getLogger(StandardStateManagerProvider.class);
private final ConcurrentMap<String, StateManager> stateManagers = new ConcurrentHashMap<>();
@ -62,12 +63,12 @@ public class StandardStateManagerProvider implements StateManagerProvider {
this.clusterStateProvider = clusterStateProvider;
}
public static StateManagerProvider create(final NiFiProperties properties) throws ConfigParseException, IOException {
final StateProvider localProvider = createLocalStateProvider(properties);
public static StateManagerProvider create(final NiFiProperties properties, final VariableRegistry variableRegistry) throws ConfigParseException, IOException {
final StateProvider localProvider = createLocalStateProvider(properties,variableRegistry);
final StateProvider clusterProvider;
if (properties.isNode()) {
clusterProvider = createClusteredStateProvider(properties);
clusterProvider = createClusteredStateProvider(properties,variableRegistry);
} else {
clusterProvider = null;
}
@ -75,19 +76,20 @@ public class StandardStateManagerProvider implements StateManagerProvider {
return new StandardStateManagerProvider(localProvider, clusterProvider);
}
private static StateProvider createLocalStateProvider(final NiFiProperties properties) throws IOException, ConfigParseException {
private static StateProvider createLocalStateProvider(final NiFiProperties properties, final VariableRegistry variableRegistry) throws IOException, ConfigParseException {
final File configFile = properties.getStateManagementConfigFile();
return createStateProvider(configFile, Scope.LOCAL, properties);
return createStateProvider(configFile, Scope.LOCAL, properties, variableRegistry);
}
private static StateProvider createClusteredStateProvider(final NiFiProperties properties) throws IOException, ConfigParseException {
private static StateProvider createClusteredStateProvider(final NiFiProperties properties, final VariableRegistry variableRegistry) throws IOException, ConfigParseException {
final File configFile = properties.getStateManagementConfigFile();
return createStateProvider(configFile, Scope.CLUSTER, properties);
return createStateProvider(configFile, Scope.CLUSTER, properties, variableRegistry);
}
private static StateProvider createStateProvider(final File configFile, final Scope scope, final NiFiProperties properties) throws ConfigParseException, IOException {
private static StateProvider createStateProvider(final File configFile, final Scope scope, final NiFiProperties properties,
final VariableRegistry variableRegistry) throws ConfigParseException, IOException {
final String providerId;
final String providerIdPropertyName;
final String providerDescription;
@ -166,17 +168,18 @@ public class StandardStateManagerProvider implements StateManagerProvider {
+ " is configured to use scope " + scope);
}
//create variable registry
final Map<PropertyDescriptor, PropertyValue> propertyMap = new HashMap<>();
final Map<PropertyDescriptor, String> propertyStringMap = new HashMap<>();
for (final PropertyDescriptor descriptor : provider.getPropertyDescriptors()) {
propertyMap.put(descriptor, new StandardPropertyValue(descriptor.getDefaultValue(), null));
propertyMap.put(descriptor, new StandardPropertyValue(descriptor.getDefaultValue(),null, variableRegistry));
propertyStringMap.put(descriptor, descriptor.getDefaultValue());
}
for (final Map.Entry<String, String> entry : providerConfig.getProperties().entrySet()) {
final PropertyDescriptor descriptor = provider.getPropertyDescriptor(entry.getKey());
propertyStringMap.put(descriptor, entry.getValue());
propertyMap.put(descriptor, new StandardPropertyValue(entry.getValue(), null));
propertyMap.put(descriptor, new StandardPropertyValue(entry.getValue(),null, variableRegistry));
}
final SSLContext sslContext = SslContextFactory.createSslContext(properties, false);
@ -186,7 +189,7 @@ public class StandardStateManagerProvider implements StateManagerProvider {
provider.initialize(initContext);
}
final ValidationContext validationContext = new StandardValidationContext(null, propertyStringMap, null, null, null);
final ValidationContext validationContext = new StandardValidationContext(null, propertyStringMap, null, null, null,variableRegistry);
final Collection<ValidationResult> results = provider.validate(validationContext);
final StringBuilder validationFailures = new StringBuilder();

View File

@ -54,6 +54,7 @@ import org.apache.nifi.encrypt.StringEncryptor;
import org.apache.nifi.logging.LogRepositoryFactory;
import org.apache.nifi.nar.NarCloseable;
import org.apache.nifi.processor.StandardProcessContext;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.remote.RemoteGroupPort;
import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.util.NiFiProperties;
@ -101,6 +102,7 @@ public final class StandardProcessGroup implements ProcessGroup {
private final Map<String, ControllerServiceNode> controllerServices = new HashMap<>();
private final Map<String, Template> templates = new HashMap<>();
private final StringEncryptor encryptor;
private final VariableRegistry variableRegistry;
private final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
private final Lock readLock = rwLock.readLock();
@ -109,7 +111,8 @@ public final class StandardProcessGroup implements ProcessGroup {
private static final Logger LOG = LoggerFactory.getLogger(StandardProcessGroup.class);
public StandardProcessGroup(final String id, final ControllerServiceProvider serviceProvider, final StandardProcessScheduler scheduler,
final NiFiProperties nifiProps, final StringEncryptor encryptor, final FlowController flowController) {
final NiFiProperties nifiProps, final StringEncryptor encryptor, final FlowController flowController,
final VariableRegistry variableRegistry) {
this.id = id;
this.controllerServiceProvider = serviceProvider;
this.parent = new AtomicReference<>();
@ -117,6 +120,7 @@ public final class StandardProcessGroup implements ProcessGroup {
this.comments = new AtomicReference<>("");
this.encryptor = encryptor;
this.flowController = flowController;
this.variableRegistry = variableRegistry;
name = new AtomicReference<>();
position = new AtomicReference<>(new Position(0D, 0D));
@ -345,7 +349,7 @@ public final class StandardProcessGroup implements ProcessGroup {
private void shutdown(final ProcessGroup procGroup) {
for (final ProcessorNode node : procGroup.getProcessors()) {
try (final NarCloseable x = NarCloseable.withNarLoader()) {
final StandardProcessContext processContext = new StandardProcessContext(node, controllerServiceProvider, encryptor, getStateManager(node.getIdentifier()));
final StandardProcessContext processContext = new StandardProcessContext(node, controllerServiceProvider, encryptor, getStateManager(node.getIdentifier()),variableRegistry);
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, node.getProcessor(), processContext);
}
}
@ -710,7 +714,7 @@ public final class StandardProcessGroup implements ProcessGroup {
}
try (final NarCloseable x = NarCloseable.withNarLoader()) {
final StandardProcessContext processContext = new StandardProcessContext(processor, controllerServiceProvider, encryptor, getStateManager(processor.getIdentifier()));
final StandardProcessContext processContext = new StandardProcessContext(processor, controllerServiceProvider, encryptor, getStateManager(processor.getIdentifier()),variableRegistry);
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, processor.getProcessor(), processContext);
} catch (final Exception e) {
throw new ComponentLifeCycleException("Failed to invoke 'OnRemoved' methods of " + processor, e);
@ -1839,7 +1843,7 @@ public final class StandardProcessGroup implements ProcessGroup {
service.verifyCanDelete();
try (final NarCloseable x = NarCloseable.withNarLoader()) {
final ConfigurationContext configurationContext = new StandardConfigurationContext(service, controllerServiceProvider, null);
final ConfigurationContext configurationContext = new StandardConfigurationContext(service, controllerServiceProvider, null, variableRegistry);
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, service.getControllerServiceImplementation(), configurationContext);
}

View File

@ -36,6 +36,7 @@ import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.encrypt.StringEncryptor;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.util.Connectables;
public class StandardProcessContext implements ProcessContext, ControllerServiceLookup {
@ -45,12 +46,15 @@ public class StandardProcessContext implements ProcessContext, ControllerService
private final Map<PropertyDescriptor, PreparedQuery> preparedQueries;
private final StringEncryptor encryptor;
private final StateManager stateManager;
private final VariableRegistry variableRegistry;
public StandardProcessContext(final ProcessorNode processorNode, final ControllerServiceProvider controllerServiceProvider, final StringEncryptor encryptor, final StateManager stateManager) {
public StandardProcessContext(final ProcessorNode processorNode, final ControllerServiceProvider controllerServiceProvider, final StringEncryptor encryptor, final StateManager stateManager,
final VariableRegistry variableRegistry) {
this.procNode = processorNode;
this.controllerServiceProvider = controllerServiceProvider;
this.encryptor = encryptor;
this.stateManager = stateManager;
this.variableRegistry = variableRegistry;
preparedQueries = new HashMap<>();
for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getProperties().entrySet()) {
@ -86,12 +90,12 @@ public class StandardProcessContext implements ProcessContext, ControllerService
final String setPropertyValue = procNode.getProperty(descriptor);
final String propValue = (setPropertyValue == null) ? descriptor.getDefaultValue() : setPropertyValue;
return new StandardPropertyValue(propValue, this, preparedQueries.get(descriptor));
return new StandardPropertyValue(propValue, this, preparedQueries.get(descriptor), variableRegistry);
}
@Override
public PropertyValue newPropertyValue(final String rawValue) {
return new StandardPropertyValue(rawValue, this, Query.prepare(rawValue));
return new StandardPropertyValue(rawValue, this, Query.prepare(rawValue), variableRegistry);
}
@Override
@ -221,4 +225,5 @@ public class StandardProcessContext implements ProcessContext, ControllerService
public String getName() {
return procNode.getName();
}
}

View File

@ -1,3 +1,4 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -36,6 +37,7 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.expression.ExpressionLanguageCompiler;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.registry.VariableRegistry;
public class StandardValidationContext implements ValidationContext {
@ -45,12 +47,13 @@ public class StandardValidationContext implements ValidationContext {
private final Map<String, Boolean> expressionLanguageSupported;
private final String annotationData;
private final Set<String> serviceIdentifiersToNotValidate;
private final VariableRegistry variableRegistry;
private final String groupId;
private final String componentId;
public StandardValidationContext(final ControllerServiceProvider controllerServiceProvider, final Map<PropertyDescriptor, String> properties,
final String annotationData, final String groupId, final String componentId) {
this(controllerServiceProvider, Collections.<String> emptySet(), properties, annotationData, groupId, componentId);
final String annotationData, final String groupId, final String componentId, VariableRegistry variableRegistry) {
this(controllerServiceProvider, Collections.<String> emptySet(), properties, annotationData, groupId, componentId,variableRegistry);
}
public StandardValidationContext(
@ -59,11 +62,12 @@ public class StandardValidationContext implements ValidationContext {
final Map<PropertyDescriptor, String> properties,
final String annotationData,
final String groupId,
final String componentId) {
final String componentId, VariableRegistry variableRegistry) {
this.controllerServiceProvider = controllerServiceProvider;
this.properties = new HashMap<>(properties);
this.annotationData = annotationData;
this.serviceIdentifiersToNotValidate = serviceIdentifiersToNotValidate;
this.variableRegistry = variableRegistry;
this.groupId = groupId;
this.componentId = componentId;
@ -87,12 +91,12 @@ public class StandardValidationContext implements ValidationContext {
@Override
public PropertyValue newPropertyValue(final String rawValue) {
return new StandardPropertyValue(rawValue, controllerServiceProvider, Query.prepare(rawValue));
return new StandardPropertyValue(rawValue, controllerServiceProvider, Query.prepare(rawValue), variableRegistry);
}
@Override
public ExpressionLanguageCompiler newExpressionLanguageCompiler() {
return new StandardExpressionLanguageCompiler();
return new StandardExpressionLanguageCompiler(variableRegistry);
}
@Override
@ -100,13 +104,13 @@ public class StandardValidationContext implements ValidationContext {
final ControllerServiceNode serviceNode = controllerServiceProvider.getControllerServiceNode(controllerService.getIdentifier());
final ProcessGroup serviceGroup = serviceNode.getProcessGroup();
final String serviceGroupId = serviceGroup == null ? null : serviceGroup.getIdentifier();
return new StandardValidationContext(controllerServiceProvider, serviceNode.getProperties(), serviceNode.getAnnotationData(), serviceGroupId, serviceNode.getIdentifier());
return new StandardValidationContext(controllerServiceProvider, serviceNode.getProperties(), serviceNode.getAnnotationData(), serviceGroupId, serviceNode.getIdentifier(),variableRegistry);
}
@Override
public PropertyValue getProperty(final PropertyDescriptor property) {
final String configuredValue = properties.get(property);
return new StandardPropertyValue(configuredValue == null ? property.getDefaultValue() : configuredValue, controllerServiceProvider, preparedQueries.get(property));
return new StandardPropertyValue(configuredValue == null ? property.getDefaultValue() : configuredValue, controllerServiceProvider, preparedQueries.get(property), variableRegistry);
}
@Override

View File

@ -23,23 +23,26 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.controller.ValidationContextFactory;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.registry.VariableRegistry;
public class StandardValidationContextFactory implements ValidationContextFactory {
private final ControllerServiceProvider serviceProvider;
private final VariableRegistry variableRegistry;
public StandardValidationContextFactory(final ControllerServiceProvider serviceProvider) {
public StandardValidationContextFactory(final ControllerServiceProvider serviceProvider, final VariableRegistry variableRegistry) {
this.serviceProvider = serviceProvider;
this.variableRegistry = variableRegistry;
}
@Override
public ValidationContext newValidationContext(final Map<PropertyDescriptor, String> properties, final String annotationData, final String groupId, final String componentId) {
return new StandardValidationContext(serviceProvider, properties, annotationData, groupId, componentId);
return new StandardValidationContext(serviceProvider, properties, annotationData, groupId, componentId,variableRegistry);
}
@Override
public ValidationContext newValidationContext(final Set<String> serviceIdentifiersToNotValidate,
final Map<PropertyDescriptor, String> properties, final String annotationData, final String groupId, String componentId) {
return new StandardValidationContext(serviceProvider, serviceIdentifiersToNotValidate, properties, annotationData, groupId, componentId);
return new StandardValidationContext(serviceProvider, serviceIdentifiersToNotValidate, properties, annotationData, groupId, componentId,variableRegistry);
}
}

View File

@ -24,8 +24,11 @@ import org.apache.nifi.cluster.protocol.NodeProtocolSender;
import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.repository.FlowFileEventRepository;
import org.apache.nifi.encrypt.StringEncryptor;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.util.NiFiProperties;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.BeansException;
import org.springframework.beans.factory.FactoryBean;
import org.springframework.context.ApplicationContext;
@ -37,6 +40,8 @@ import org.springframework.context.ApplicationContextAware;
@SuppressWarnings("rawtypes")
public class FlowControllerFactoryBean implements FactoryBean, ApplicationContextAware {
private static final Logger LOG = LoggerFactory.getLogger(FlowControllerFactoryBean.class);
private ApplicationContext applicationContext;
private FlowController flowController;
private NiFiProperties properties;
@ -45,6 +50,7 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex
private StringEncryptor encryptor;
private BulletinRepository bulletinRepository;
private ClusterCoordinator clusterCoordinator;
private VariableRegistry variableRegistry;
@Override
public Object getObject() throws Exception {
@ -63,7 +69,7 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex
nodeProtocolSender,
bulletinRepository,
clusterCoordinator,
heartbeatMonitor);
heartbeatMonitor, variableRegistry);
} else {
flowController = FlowController.createStandaloneInstance(
flowFileEventRepository,
@ -71,7 +77,7 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex
authorizer,
auditService,
encryptor,
bulletinRepository);
bulletinRepository, variableRegistry);
}
}
@ -79,6 +85,8 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex
return flowController;
}
@Override
public Class getObjectType() {
return FlowController.class;
@ -114,6 +122,10 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex
this.bulletinRepository = bulletinRepository;
}
public void setVariableRegistry(VariableRegistry variableRegistry) {
this.variableRegistry = variableRegistry;
}
public void setClusterCoordinator(final ClusterCoordinator clusterCoordinator) {
this.clusterCoordinator = clusterCoordinator;
}

View File

@ -27,6 +27,11 @@
<!-- nifi properties created via getInstance using a file path specified as a system property -->
<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">
<constructor-arg type="java.nio.file.Path[]" value="#{nifiProperties.getVariableRegistryPropertiesPaths()}" />
</bean>
<!-- flow file event repository -->
<bean id="flowFileEventRepository" class="org.apache.nifi.spring.RingBufferEventRepositoryBean">
</bean>
@ -41,6 +46,7 @@
<property name="encryptor" ref="stringEncryptor" />
<property name="bulletinRepository" ref="bulletinRepository" />
<property name="clusterCoordinator" ref="clusterCoordinator" />
<property name="variableRegistry" ref="variableRegistry"/>
</bean>
<!-- flow service -->

View File

@ -26,6 +26,8 @@ import org.apache.nifi.controller.serialization.FlowSerializer;
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;
@ -63,6 +65,7 @@ public class StandardFlowServiceTest {
private AuditService mockAuditService;
private StringEncryptor mockEncryptor;
private RevisionManager revisionManager;
private VariableRegistry variableRegistry;
@BeforeClass
public static void setupSuite() {
@ -72,11 +75,13 @@ public class StandardFlowServiceTest {
@Before
public void setup() throws Exception {
properties = NiFiProperties.getInstance();
variableRegistry = VariableRegistryUtils.createCustomVariableRegistry(properties.getVariableRegistryPropertiesPaths());
mockFlowFileEventRepository = mock(FlowFileEventRepository.class);
authorizer = mock(Authorizer.class);
mockAuditService = mock(AuditService.class);
revisionManager = mock(RevisionManager.class);
flowController = FlowController.createStandaloneInstance(mockFlowFileEventRepository, properties, authorizer, mockAuditService, mockEncryptor, new VolatileBulletinRepository());
flowController = FlowController.createStandaloneInstance(mockFlowFileEventRepository, properties, authorizer, mockAuditService, mockEncryptor,
new VolatileBulletinRepository(), variableRegistry);
flowService = StandardFlowService.createStandaloneInstance(flowController, properties, mockEncryptor, revisionManager, authorizer);
}

View File

@ -33,6 +33,8 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
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;
@ -61,6 +63,7 @@ public class TestFlowController {
private StringEncryptor encryptor;
private NiFiProperties properties;
private BulletinRepository bulletinRepo;
private VariableRegistry variableRegistry;
@Before
public void setup() {
@ -111,9 +114,10 @@ public class TestFlowController {
policies1.add(policy2);
authorizer = new MockPolicyBasedAuthorizer(groups1, users1, policies1);
variableRegistry = VariableRegistryUtils.createCustomVariableRegistry(properties.getVariableRegistryPropertiesPaths());
bulletinRepo = Mockito.mock(BulletinRepository.class);
controller = FlowController.createStandaloneInstance(flowFileEventRepo, properties, authorizer, auditService, encryptor, bulletinRepo);
controller = FlowController.createStandaloneInstance(flowFileEventRepo, properties, authorizer, auditService, encryptor, bulletinRepo,variableRegistry);
standardFlowSynchronizer = new StandardFlowSynchronizer(StringEncryptor.createEncryptor());
}
@ -165,7 +169,7 @@ public class TestFlowController {
assertNotEquals(authFingerprint, authorizer.getFingerprint());
controller.shutdown(true);
controller = FlowController.createStandaloneInstance(flowFileEventRepo, properties, authorizer, auditService, encryptor, bulletinRepo);
controller = FlowController.createStandaloneInstance(flowFileEventRepo, properties, authorizer, auditService, encryptor, bulletinRepo,variableRegistry);
controller.synchronize(standardFlowSynchronizer, proposedDataFlow);
assertEquals(authFingerprint, authorizer.getFingerprint());
}

View File

@ -43,6 +43,7 @@ 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;
@ -661,7 +662,8 @@ public class TestProcessorLifecycle {
properties.setProperty("nifi.remote.input.secure", "");
return FlowController.createStandaloneInstance(mock(FlowFileEventRepository.class), properties,
mock(Authorizer.class), mock(AuditService.class), null, new VolatileBulletinRepository());
mock(Authorizer.class), mock(AuditService.class), null, new VolatileBulletinRepository(),
VariableRegistryUtils.createCustomVariableRegistry(properties.getVariableRegistryPropertiesPaths()));
}
/**

View File

@ -61,6 +61,8 @@ import org.apache.nifi.processor.ProcessSession;
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;
@ -76,6 +78,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 FlowController controller;
private ProcessGroup rootGroup;
@ -83,7 +86,7 @@ public class TestStandardProcessScheduler {
public void setup() throws InitializationException {
System.setProperty("nifi.properties.file.path", "src/test/resources/nifi.properties");
this.refreshNiFiProperties();
scheduler = new StandardProcessScheduler(Mockito.mock(ControllerServiceProvider.class), null, stateMgrProvider);
scheduler = new StandardProcessScheduler(Mockito.mock(ControllerServiceProvider.class), null, stateMgrProvider,variableRegistry);
scheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, Mockito.mock(SchedulingAgent.class));
reportingTask = new TestReportingTask();
@ -91,8 +94,8 @@ public class TestStandardProcessScheduler {
Mockito.mock(ComponentLog.class), null);
reportingTask.initialize(config);
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(null);
taskNode = new StandardReportingTaskNode(reportingTask, UUID.randomUUID().toString(), null, scheduler, validationContextFactory);
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(null, variableRegistry);
taskNode = new StandardReportingTaskNode(reportingTask, UUID.randomUUID().toString(), null, scheduler, validationContextFactory, variableRegistry);
controller = Mockito.mock(FlowController.class);
rootGroup = new MockProcessGroup();
@ -127,12 +130,12 @@ public class TestStandardProcessScheduler {
public void testDisableControllerServiceWithProcessorTryingToStartUsingIt() throws InterruptedException {
final Processor proc = new ServiceReferencingProcessor();
final StandardControllerServiceProvider serviceProvider = new StandardControllerServiceProvider(controller, scheduler, null, Mockito.mock(StateManagerProvider.class));
final StandardControllerServiceProvider serviceProvider = new StandardControllerServiceProvider(controller, scheduler, null, Mockito.mock(StateManagerProvider.class),variableRegistry);
final ControllerServiceNode service = serviceProvider.createControllerService(NoStartServiceImpl.class.getName(), "service", true);
rootGroup.addControllerService(service);
final ProcessorNode procNode = new StandardProcessorNode(proc, UUID.randomUUID().toString(),
new StandardValidationContextFactory(serviceProvider), scheduler, serviceProvider);
new StandardValidationContextFactory(serviceProvider, variableRegistry), scheduler, serviceProvider);
rootGroup.addProcessor(procNode);
procNode.setProperty(ServiceReferencingProcessor.SERVICE_DESC.getName(), service.getIdentifier());
@ -211,7 +214,7 @@ public class TestStandardProcessScheduler {
@Test
public void validateServiceEnablementLogicHappensOnlyOnce() throws Exception {
final ProcessScheduler scheduler = createScheduler();
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider);
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider,variableRegistry);
final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(),
"1", false);
assertFalse(serviceNode.isActive());
@ -250,7 +253,7 @@ public class TestStandardProcessScheduler {
@Test
public void validateDisabledServiceCantBeDisabled() throws Exception {
final ProcessScheduler scheduler = createScheduler();
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider);
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry);
final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(),
"1", false);
final SimpleTestService ts = (SimpleTestService) serviceNode.getControllerServiceImplementation();
@ -288,7 +291,7 @@ public class TestStandardProcessScheduler {
@Test
public void validateEnabledServiceCanOnlyBeDisabledOnce() throws Exception {
final ProcessScheduler scheduler = createScheduler();
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider);
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider,variableRegistry);
final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(),
"1", false);
final SimpleTestService ts = (SimpleTestService) serviceNode.getControllerServiceImplementation();
@ -322,7 +325,7 @@ public class TestStandardProcessScheduler {
@Test
public void validateDisablingOfTheFailedService() throws Exception {
final ProcessScheduler scheduler = createScheduler();
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider);
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry);
final ControllerServiceNode serviceNode = provider.createControllerService(FailingService.class.getName(),
"1", false);
scheduler.enableControllerService(serviceNode);
@ -353,7 +356,7 @@ public class TestStandardProcessScheduler {
@Test
public void validateEnabledDisableMultiThread() throws Exception {
final ProcessScheduler scheduler = createScheduler();
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider);
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry);
final ExecutorService executor = Executors.newCachedThreadPool();
for (int i = 0; i < 200; i++) {
final ControllerServiceNode serviceNode = provider
@ -396,7 +399,7 @@ public class TestStandardProcessScheduler {
@Test
public void validateNeverEnablingServiceCanStillBeDisabled() throws Exception {
final ProcessScheduler scheduler = createScheduler();
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider);
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider,variableRegistry);
final ControllerServiceNode serviceNode = provider.createControllerService(LongEnablingService.class.getName(),
"1", false);
final LongEnablingService ts = (LongEnablingService) serviceNode.getControllerServiceImplementation();
@ -421,7 +424,7 @@ public class TestStandardProcessScheduler {
@Test
public void validateLongEnablingServiceCanStillBeDisabled() throws Exception {
final ProcessScheduler scheduler = createScheduler();
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider);
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry);
final ControllerServiceNode serviceNode = provider.createControllerService(LongEnablingService.class.getName(),
"1", false);
final LongEnablingService ts = (LongEnablingService) serviceNode.getControllerServiceImplementation();
@ -518,6 +521,6 @@ public class TestStandardProcessScheduler {
}
private ProcessScheduler createScheduler() {
return new StandardProcessScheduler(null, null, stateMgrProvider);
return new StandardProcessScheduler(null, null, stateMgrProvider, variableRegistry);
}
}

View File

@ -22,6 +22,8 @@ import org.apache.nifi.controller.ControllerService;
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.NiFiProperties;
import org.junit.Before;
@ -33,6 +35,7 @@ public class StandardControllerServiceProviderTest {
private ControllerService proxied;
private ControllerService implementation;
private static VariableRegistry variableRegistry;
@BeforeClass
public static void setupSuite() throws Exception {
@ -40,6 +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());
}
@Before
@ -67,7 +71,7 @@ public class StandardControllerServiceProviderTest {
@Override
public void onComponentRemoved(String componentId) {
}
});
}, variableRegistry);
ControllerServiceNode node = provider.createControllerService(clazz, id, true);
proxied = node.getProxiedControllerService();
implementation = node.getControllerServiceImplementation();

View File

@ -1,3 +1,4 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -45,6 +46,8 @@ import org.apache.nifi.controller.service.mock.ServiceC;
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;
@ -74,13 +77,15 @@ public class TestStandardControllerServiceProvider {
}
};
private static VariableRegistry variableRegistry = VariableRegistryUtils.createSystemVariableRegistry();
@BeforeClass
public static void setNiFiProps() {
System.setProperty("nifi.properties.file.path", "src/test/resources/nifi.properties");
}
private StandardProcessScheduler createScheduler() {
return new StandardProcessScheduler(null, null, stateManagerProvider);
return new StandardProcessScheduler(null, null, stateManagerProvider,variableRegistry);
}
@Test
@ -90,7 +95,7 @@ public class TestStandardControllerServiceProvider {
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
final ProcessScheduler scheduler = createScheduler();
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider);
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider,variableRegistry);
final ControllerServiceNode serviceNode = provider.createControllerService(ServiceB.class.getName(), "B", false);
provider.enableControllerService(serviceNode);
@ -104,7 +109,7 @@ public class TestStandardControllerServiceProvider {
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(group);
final ProcessScheduler scheduler = createScheduler();
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider);
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider,variableRegistry);
final ControllerServiceNode serviceNodeB = provider.createControllerService(ServiceB.class.getName(), "B", false);
final ControllerServiceNode serviceNodeA = provider.createControllerService(ServiceA.class.getName(), "A", false);
@ -163,7 +168,7 @@ public class TestStandardControllerServiceProvider {
final FlowController controller = Mockito.mock(FlowController.class);
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider);
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider,variableRegistry);
// build a graph of controller services with dependencies as such:
//
@ -215,7 +220,7 @@ public class TestStandardControllerServiceProvider {
final FlowController controller = Mockito.mock(FlowController.class);
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, null, null, stateManagerProvider);
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, null, null, stateManagerProvider,variableRegistry);
final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1", false);
final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceB.class.getName(), "2", false);
@ -359,9 +364,9 @@ public class TestStandardControllerServiceProvider {
private ProcessorNode createProcessor(final StandardProcessScheduler scheduler, final ControllerServiceProvider serviceProvider) {
final ProcessorNode procNode = new StandardProcessorNode(new DummyProcessor(), UUID.randomUUID().toString(),
new StandardValidationContextFactory(serviceProvider), scheduler, serviceProvider);
new StandardValidationContextFactory(serviceProvider, null), scheduler, serviceProvider);
final ProcessGroup group = new StandardProcessGroup(UUID.randomUUID().toString(), serviceProvider, scheduler, null, null, null);
final ProcessGroup group = new StandardProcessGroup(UUID.randomUUID().toString(), serviceProvider, scheduler, null, null, null, variableRegistry);
group.addProcessor(procNode);
procNode.setProcessGroup(group);
@ -375,7 +380,7 @@ public class TestStandardControllerServiceProvider {
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
final StandardProcessScheduler scheduler = createScheduler();
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, null, null, stateManagerProvider);
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, null, null, stateManagerProvider,variableRegistry);
final ControllerServiceNode serviceNode = provider.createControllerService(ServiceA.class.getName(), "1", false);
final ProcessorNode procNode = createProcessor(scheduler, provider);
@ -394,7 +399,7 @@ public class TestStandardControllerServiceProvider {
public void validateEnableServices() {
StandardProcessScheduler scheduler = createScheduler();
FlowController controller = Mockito.mock(FlowController.class);
StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider);
StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider,variableRegistry);
ProcessGroup procGroup = new MockProcessGroup();
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
@ -439,7 +444,7 @@ public class TestStandardControllerServiceProvider {
StandardProcessScheduler scheduler = createScheduler();
FlowController controller = Mockito.mock(FlowController.class);
StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null,
stateManagerProvider);
stateManagerProvider, variableRegistry);
ProcessGroup procGroup = new MockProcessGroup();
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
@ -475,7 +480,7 @@ public class TestStandardControllerServiceProvider {
public void validateEnableServicesWithDisabledMissingService() {
StandardProcessScheduler scheduler = createScheduler();
FlowController controller = Mockito.mock(FlowController.class);
StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider);
StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider,variableRegistry);
ProcessGroup procGroup = new MockProcessGroup();
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);

View File

@ -32,6 +32,8 @@ 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;
@ -43,9 +45,9 @@ 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));
properties.put(WriteAheadLocalStateProvider.PATH, new StandardPropertyValue("target/local-state-provider/" + UUID.randomUUID().toString(), null, variableRegistry));
provider.initialize(new StateProviderInitializationContext() {
@Override
@ -62,7 +64,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);
return new StandardPropertyValue(null, null, variableRegistry);
}
return prop;
}

View File

@ -31,6 +31,8 @@ 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;
@ -40,6 +42,7 @@ public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
private StateProvider provider;
private TestingServer zkServer;
private VariableRegistry variableRegistry;
private static final Map<PropertyDescriptor, String> defaultProperties = new HashMap<>();
@ -58,6 +61,7 @@ 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 {
@ -71,7 +75,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));
propValueMap.put(entry.getKey(), new StandardPropertyValue(entry.getValue(), null, variableRegistry));
}
return propValueMap;
}
@ -79,7 +83,7 @@ public class TestZooKeeperStateProvider extends AbstractTestStateProvider {
@Override
public PropertyValue getProperty(final PropertyDescriptor property) {
final String prop = properties.get(property);
return new StandardPropertyValue(prop, null);
return new StandardPropertyValue(prop, null, variableRegistry);
}
@Override

View File

@ -29,15 +29,19 @@ 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);
final PropertyValue value = new StandardPropertyValue("Hello, ${audience}!", lookup, variableRegistry);
final Map<String, String> attributes = new HashMap<>();
attributes.put("audience", "World");
assertEquals("Hello, World!", value.evaluateAttributeExpressions(createFlowFile(attributes)).getValue());
@ -45,7 +49,7 @@ public class TestStandardPropertyValue {
@Test
public void testMissingEndBraceEvaluatesToStringLiteral() {
final PropertyValue value = new StandardPropertyValue("Hello, ${audience!", lookup);
final PropertyValue value = new StandardPropertyValue("Hello, ${audience!", lookup, variableRegistry);
final Map<String, String> attributes = new HashMap<>();
attributes.put("audience", "World");
assertEquals("Hello, ${audience!", value.evaluateAttributeExpressions(createFlowFile(attributes)).getValue());
@ -53,7 +57,7 @@ public class TestStandardPropertyValue {
@Test
public void testEscaped() {
final PropertyValue value = new StandardPropertyValue("Hello, $${audience}!", lookup);
final PropertyValue value = new StandardPropertyValue("Hello, $${audience}!", lookup, variableRegistry);
final Map<String, String> attributes = new HashMap<>();
attributes.put("audience", "World");
assertEquals("Hello, ${audience}!", value.evaluateAttributeExpressions(createFlowFile(attributes)).getValue());
@ -61,7 +65,7 @@ public class TestStandardPropertyValue {
@Test
public void testSubstituteAttributesWithMultipleMatchingArgs() {
final PropertyValue value = new StandardPropertyValue("Hello, ${audience}${comma}${question}!", lookup);
final PropertyValue value = new StandardPropertyValue("Hello, ${audience}${comma}${question}!", lookup, variableRegistry);
final Map<String, String> attributes = new HashMap<>();
attributes.put("audience", "World");
attributes.put("comma", ",");
@ -71,14 +75,14 @@ public class TestStandardPropertyValue {
@Test
public void testSubstituteAttributesWithNoMatch() {
final PropertyValue value = new StandardPropertyValue("Hello, ${audience}${comma}${question:replaceNull('')}!", lookup);
final PropertyValue value = new StandardPropertyValue("Hello, ${audience}${comma}${question:replaceNull('')}!", lookup, variableRegistry);
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);
final PropertyValue value = new StandardPropertyValue("Hello, ${'${a}${b}'}!", lookup, variableRegistry);
final Map<String, String> attributes = new HashMap<>();
attributes.put("a", "b");
attributes.put("b", "World");
@ -88,7 +92,7 @@ public class TestStandardPropertyValue {
@Test
public void testGetValueAsIntegerAfterSubstitute() {
final PropertyValue value = new StandardPropertyValue("1${value}", lookup);
final PropertyValue value = new StandardPropertyValue("1${value}", lookup, variableRegistry);
final Map<String, String> attributes = new HashMap<>();
attributes.put("value", "39");
assertEquals(139, value.evaluateAttributeExpressions(createFlowFile(attributes)).asInteger().intValue());
@ -96,7 +100,7 @@ public class TestStandardPropertyValue {
@Test(expected = NumberFormatException.class)
public void testGetValueAsIntegerAfterSubstitutingWithNonInteger() {
final PropertyValue value = new StandardPropertyValue("1${value}", lookup);
final PropertyValue value = new StandardPropertyValue("1${value}", lookup, variableRegistry);
final Map<String, String> attributes = new HashMap<>();
attributes.put("value", "Yes");
final PropertyValue substituted = value.evaluateAttributeExpressions(createFlowFile(attributes));
@ -105,7 +109,7 @@ public class TestStandardPropertyValue {
@Test
public void testFileSize() {
final PropertyValue value = new StandardPropertyValue("${fileSize}", lookup);
final PropertyValue value = new StandardPropertyValue("${fileSize}", lookup, variableRegistry);
final FlowFile flowFile = new StandardFlowFileRecord.Builder().size(1024 * 1024L).build();
final long val = value.evaluateAttributeExpressions(flowFile).asLong().longValue();
assertEquals(1024 * 1024L, val);
@ -115,7 +119,7 @@ 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);
final PropertyValue value = new StandardPropertyValue("${entryDate:toNumber():toDate():format('yyyy')}", lookup, variableRegistry);
final FlowFile flowFile = new StandardFlowFileRecord.Builder().entryDate(now.getTimeInMillis()).build();
final int val = value.evaluateAttributeExpressions(flowFile).asInteger().intValue();
assertEquals(year, val);
@ -125,7 +129,7 @@ public class TestStandardPropertyValue {
public void testSystemProperty() {
System.setProperty("Prop1", "Foo");
System.setProperty("Prop2", "Bar");
final PropertyValue value = new StandardPropertyValue("${Prop1}${Prop2}${abc}", lookup);
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);
@ -171,5 +175,6 @@ public class TestStandardPropertyValue {
public boolean isControllerServiceEnabling(final String serviceIdentifier) {
return false;
}
}
}

View File

@ -184,4 +184,8 @@ nifi.zookeeper.root.node=${nifi.zookeeper.root.node}
nifi.kerberos.krb5.file=${nifi.kerberos.krb5.file}
nifi.kerberos.service.principal=${nifi.kerberos.service.principal}
nifi.kerberos.keytab.location=${nifi.kerberos.keytab.location}
nifi.kerberos.authentication.expiration=${nifi.kerberos.authentication.expiration}
nifi.kerberos.authentication.expiration=${nifi.kerberos.authentication.expiration}
# external properties files for variable registry
# supports a comma delimited list of file locations
nifi.variable.registry.properties=

View File

@ -44,6 +44,7 @@ import org.apache.nifi.cluster.manager.exception.NoClusterCoordinatorException;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.reporting.ReportingTaskProvider;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
@ -91,6 +92,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
private ReportingTaskProvider reportingTaskProvider;
private AuditService auditService;
private Authorizer authorizer;
private VariableRegistry variableRegistry;
private void authorizeFlowAccess(final NiFiUser user) {
// authorize access
@ -287,6 +289,11 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
}
@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) {
@ -885,4 +892,8 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
public void setAuthorizer(final Authorizer authorizer) {
this.authorizer = authorizer;
}
public void setVariableRegistry(final VariableRegistry variableRegistry){
this.variableRegistry = variableRegistry;
}
}

View File

@ -78,6 +78,7 @@ import org.apache.nifi.provenance.search.QuerySubmission;
import org.apache.nifi.provenance.search.SearchTerm;
import org.apache.nifi.provenance.search.SearchTerms;
import org.apache.nifi.provenance.search.SearchableField;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.reporting.ReportingTask;
@ -147,7 +148,7 @@ public class ControllerFacade implements Authorizable {
// properties
private NiFiProperties properties;
private DtoFactory dtoFactory;
private VariableRegistry variableRegistry;
/**
* Returns the group id that contains the specified processor.
@ -1503,6 +1504,8 @@ public class ControllerFacade implements Authorizable {
return dto;
}
private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
final List<String> matches = new ArrayList<>();
final Processor processor = procNode.getProcessor();
@ -1572,8 +1575,7 @@ public class ControllerFacade implements Authorizable {
if (processor instanceof Searchable) {
final Searchable searchable = (Searchable) processor;
// prepare the search context
final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController);
final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController, variableRegistry);
// search the processor using the appropriate thread context classloader
try (final NarCloseable x = NarCloseable.withNarLoader()) {
@ -1781,4 +1783,8 @@ public class ControllerFacade implements Authorizable {
public void setBulletinRepository(BulletinRepository bulletinRepository) {
this.bulletinRepository = bulletinRepository;
}
public void setVariableRegistry(VariableRegistry variableRegistry) {
this.variableRegistry = variableRegistry;
}
}

View File

@ -23,6 +23,7 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.search.SearchContext;
/**
@ -33,11 +34,13 @@ public class StandardSearchContext implements SearchContext {
private final String searchTerm;
private final ProcessorNode processorNode;
private final ControllerServiceLookup controllerServiceLookup;
private final VariableRegistry variableRegistry;
public StandardSearchContext(final String searchTerm, final ProcessorNode processorNode, final ControllerServiceLookup controllerServiceLookup) {
public StandardSearchContext(final String searchTerm, final ProcessorNode processorNode, final ControllerServiceLookup controllerServiceLookup, VariableRegistry variableRegistry) {
this.searchTerm = searchTerm;
this.processorNode = processorNode;
this.controllerServiceLookup = controllerServiceLookup;
this.variableRegistry = variableRegistry;
}
@Override
@ -53,7 +56,7 @@ public class StandardSearchContext implements SearchContext {
@Override
public PropertyValue getProperty(PropertyDescriptor property) {
final String configuredValue = processorNode.getProperty(property);
return new StandardPropertyValue(configuredValue == null ? property.getDefaultValue() : configuredValue, controllerServiceLookup, null);
return new StandardPropertyValue(configuredValue == null ? property.getDefaultValue() : configuredValue, controllerServiceLookup,variableRegistry);
}
@Override

View File

@ -119,6 +119,7 @@
<property name="authorizer" ref="authorizer"/>
<property name="dtoFactory" ref="dtoFactory"/>
<property name="bulletinRepository" ref="bulletinRepository"/>
<property name="variableRegistry" ref="variableRegistry"/>
</bean>
<bean id="authorizableLookup" class="org.apache.nifi.authorization.StandardAuthorizableLookup">
<property name="controllerFacade" ref="controllerFacade"/>
@ -176,6 +177,7 @@
<property name="controllerServiceProvider" ref="controllerServiceProvider"/>
<property name="reportingTaskProvider" ref="reportingTaskProvider"/>
<property name="authorizer" ref="authorizer"/>
<property name="variableRegistry" ref="variableRegistry"/>
</bean>
<!-- rest endpoints -->

View File

@ -29,6 +29,8 @@ 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;
@ -64,6 +66,7 @@ public class TestRangerNiFiAuthorizer {
private RangerBasePluginWithPolicies rangerBasePlugin;
private AuthorizerConfigurationContext configurationContext;
private NiFiProperties nifiProperties;
private VariableRegistry variableRegistry;
private String serviceType = "nifiService";
private String appId = "nifiAppId";
@ -95,22 +98,23 @@ 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));
.thenReturn(new MockPropertyValue("src/test/resources/ranger/ranger-nifi-security.xml", null,variableRegistry));
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_AUDIT_PATH_PROP)))
.thenReturn(new MockPropertyValue("src/test/resources/ranger/ranger-nifi-audit.xml", null));
.thenReturn(new MockPropertyValue("src/test/resources/ranger/ranger-nifi-audit.xml", null,variableRegistry));
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_APP_ID_PROP)))
.thenReturn(new MockPropertyValue(appId, null));
.thenReturn(new MockPropertyValue(appId, null,variableRegistry));
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_SERVICE_TYPE_PROP)))
.thenReturn(new MockPropertyValue(serviceType, null));
.thenReturn(new MockPropertyValue(serviceType, null,variableRegistry));
return configurationContext;
}
@ -126,7 +130,7 @@ public class TestRangerNiFiAuthorizer {
@Test
public void testKerberosEnabledWithoutKeytab() {
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_KERBEROS_ENABLED_PROP)))
.thenReturn(new MockPropertyValue("true", null));
.thenReturn(new MockPropertyValue("true", null,variableRegistry));
nifiProperties = Mockito.mock(NiFiProperties.class);
when(nifiProperties.getKerberosServicePrincipal()).thenReturn("");
@ -146,7 +150,7 @@ public class TestRangerNiFiAuthorizer {
@Test
public void testKerberosEnabledWithoutPrincipal() {
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_KERBEROS_ENABLED_PROP)))
.thenReturn(new MockPropertyValue("true", null));
.thenReturn(new MockPropertyValue("true", null,variableRegistry));
nifiProperties = Mockito.mock(NiFiProperties.class);
when(nifiProperties.getKerberosKeytabLocation()).thenReturn("");
@ -166,7 +170,7 @@ public class TestRangerNiFiAuthorizer {
@Test
public void testKerberosEnabledWithoutKeytabOrPrincipal() {
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_KERBEROS_ENABLED_PROP)))
.thenReturn(new MockPropertyValue("true", null));
.thenReturn(new MockPropertyValue("true", null,variableRegistry));
nifiProperties = Mockito.mock(NiFiProperties.class);
when(nifiProperties.getKerberosKeytabLocation()).thenReturn("");
@ -200,7 +204,7 @@ public class TestRangerNiFiAuthorizer {
@Test
public void testKerberosEnabled() {
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_KERBEROS_ENABLED_PROP)))
.thenReturn(new MockPropertyValue("true", null));
.thenReturn(new MockPropertyValue("true", null,variableRegistry));
nifiProperties = Mockito.mock(NiFiProperties.class);
when(nifiProperties.getKerberosKeytabLocation()).thenReturn("test");
@ -398,7 +402,7 @@ public class TestRangerNiFiAuthorizer {
final String rangerAdminIdentity = "ranger-admin";
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_ADMIN_IDENTITY_PROP)))
.thenReturn(new MockPropertyValue(rangerAdminIdentity, null));
.thenReturn(new MockPropertyValue(rangerAdminIdentity, null,variableRegistry));
rangerBasePlugin = Mockito.mock(RangerBasePluginWithPolicies.class);
authorizer = new MockRangerNiFiAuthorizer(rangerBasePlugin);
@ -446,10 +450,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));
.thenReturn(new MockPropertyValue("src/test/resources/ranger/ranger-nifi-security.xml", null,variableRegistry));
when(configurationContext.getProperty(eq(RangerNiFiAuthorizer.RANGER_AUDIT_PATH_PROP)))
.thenReturn(new MockPropertyValue("src/test/resources/ranger/ranger-nifi-audit.xml", null));
.thenReturn(new MockPropertyValue("src/test/resources/ranger/ranger-nifi-audit.xml", null,variableRegistry));
Authorizer authorizer = new RangerNiFiAuthorizer();
try {

View File

@ -18,6 +18,8 @@ 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;
@ -37,9 +39,12 @@ 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();
}
/**
@ -72,7 +77,7 @@ public class TestInvokeGroovy extends BaseScriptTest {
@Test
public void testScriptDefinedAttribute() throws Exception {
InvokeScriptedProcessor processor = new InvokeScriptedProcessor();
MockProcessContext context = new MockProcessContext(processor);
MockProcessContext context = new MockProcessContext(processor, variableRegistry);
MockProcessorInitializationContext initContext = new MockProcessorInitializationContext(processor, context);
processor.initialize(initContext);
@ -81,7 +86,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));
processor.customValidate(new MockValidationContext(context, null, variableRegistry));
processor.setup(context);
List<PropertyDescriptor> descriptors = processor.getSupportedPropertyDescriptors();
@ -106,7 +111,7 @@ public class TestInvokeGroovy extends BaseScriptTest {
@Test
public void testScriptDefinedRelationship() throws Exception {
InvokeScriptedProcessor processor = new InvokeScriptedProcessor();
MockProcessContext context = new MockProcessContext(processor);
MockProcessContext context = new MockProcessContext(processor, variableRegistry);
MockProcessorInitializationContext initContext = new MockProcessorInitializationContext(processor, context);
processor.initialize(initContext);
@ -114,7 +119,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));
processor.customValidate(new MockValidationContext(context, null, variableRegistry));
processor.setup(context);
Set<Relationship> relationships = processor.getRelationships();

View File

@ -28,6 +28,7 @@ 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;
@ -94,7 +95,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);
return new MockPropertyValue(properties.get(descriptor), null, VariableRegistryUtils.createSystemVariableRegistry());
}
}).when(context).getProperty(Mockito.any(PropertyDescriptor.class));

View File

@ -26,6 +26,8 @@ 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;
@ -36,10 +38,12 @@ 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));
final MockProcessContext ctx = new MockProcessContext(proc, new MockStateManager(proc), variableRegistry);
final ValidationResult validationResult = ctx.setProperty("RouteA", "${a:equals('b')"); // Missing closing brace
assertFalse(validationResult.isValid());
}
@ -47,7 +51,7 @@ public class TestRouteOnAttribute {
@Test
public void testInvalidOnNonBooleanProperty() {
final RouteOnAttribute proc = new RouteOnAttribute();
final MockProcessContext ctx = new MockProcessContext(proc, new MockStateManager(proc));
final MockProcessContext ctx = new MockProcessContext(proc, new MockStateManager(proc), variableRegistry);
final ValidationResult validationResult = ctx.setProperty("RouteA", "${a:length()"); // Should be boolean
assertFalse(validationResult.isValid());
}

View File

@ -21,6 +21,7 @@ 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;
@ -136,6 +137,6 @@ public class MonitorMemoryTest {
properties.setProperty("nifi.remote.input.secure", "");
return FlowController.createStandaloneInstance(mock(FlowFileEventRepository.class), properties,
mock(Authorizer.class), mock(AuditService.class), null, null);
mock(Authorizer.class), mock(AuditService.class), null, null, VariableRegistryUtils.createCustomVariableRegistry(properties.getVariableRegistryPropertiesPaths()));
}
}

View File

@ -41,6 +41,8 @@ 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;
@ -55,6 +57,7 @@ import org.slf4j.LoggerFactory;
public class TestServerAndClient {
private static Logger LOGGER;
private static VariableRegistry variableRegistry;
static {
System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
@ -64,6 +67,7 @@ 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
@ -329,7 +333,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());
MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup(), variableRegistry);
client.cacheConfig(clientContext);
final Serializer<String> valueSerializer = new StringSerializer();
final Serializer<String> keySerializer = new StringSerializer();
@ -375,7 +379,7 @@ public class TestServerAndClient {
client2.initialize(clientInitContext2);
MockConfigurationContext clientContext2 = new MockConfigurationContext(clientProperties,
clientInitContext2.getControllerServiceLookup());
clientInitContext2.getControllerServiceLookup(), variableRegistry);
client2.cacheConfig(clientContext2);
assertFalse(client2.putIfAbsent("testKey", "test", keySerializer, valueSerializer));
assertTrue(client2.containsKey("testKey", keySerializer));
@ -408,7 +412,7 @@ public class TestServerAndClient {
server.initialize(serverInitContext);
final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup(), variableRegistry);
server.startServer(serverContext);
DistributedMapCacheClientService client = new DistributedMapCacheClientService();
@ -418,7 +422,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());
MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup(), variableRegistry);
client.cacheConfig(clientContext);
final Serializer<String> valueSerializer = new StringSerializer();
final Serializer<String> keySerializer = new StringSerializer();
@ -465,7 +469,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());
final MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup(), variableRegistry);
client.onConfigured(clientContext);
return client;

View File

@ -23,6 +23,7 @@ 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;
@ -32,6 +33,12 @@ import org.apache.nifi.update.attributes.dto.RuleDTO;
*/
public class UpdateAttributeModelFactory {
private final VariableRegistry variableRegistry;
public UpdateAttributeModelFactory(VariableRegistry variableRegistry) {
this.variableRegistry = variableRegistry;
}
public Rule createRule(final RuleDTO dto) {
if (dto == null) {
throw new IllegalArgumentException("Rule must be specified.");
@ -72,7 +79,7 @@ public class UpdateAttributeModelFactory {
}
// validate the condition's expression
final StandardExpressionLanguageCompiler elCompiler = new StandardExpressionLanguageCompiler();
final StandardExpressionLanguageCompiler elCompiler = new StandardExpressionLanguageCompiler(variableRegistry);
final String syntaxError = elCompiler.validateExpression(dto.getExpression(), false);
if (syntaxError != null) {
throw new IllegalArgumentException(syntaxError);

View File

@ -46,6 +46,7 @@ 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;
@ -187,6 +188,7 @@ 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) {
@ -219,7 +221,7 @@ public class RuleResource {
// load the criteria
final Criteria criteria = getCriteria(configurationContext, requestContext);
final UpdateAttributeModelFactory factory = new UpdateAttributeModelFactory();
final UpdateAttributeModelFactory factory = new UpdateAttributeModelFactory(variableRegistry);
// create the new rule
final Rule rule;
@ -261,10 +263,14 @@ 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();
final UpdateAttributeModelFactory factory = new UpdateAttributeModelFactory(variableRegistry);
condition = factory.createCondition(requestEntity.getCondition());
condition.setId(uuid);
} catch (final IllegalArgumentException iae) {
@ -295,10 +301,14 @@ 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();
final UpdateAttributeModelFactory factory = new UpdateAttributeModelFactory(variableRegistry);
action = factory.createAction(requestEntity.getAction());
action.setId(uuid);
} catch (final IllegalArgumentException iae) {
@ -461,6 +471,8 @@ 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) {
@ -497,7 +509,7 @@ public class RuleResource {
requestEntity.getProcessorId(), requestEntity.getRevision(), requestEntity.getClientId());
// load the criteria
final UpdateAttributeModelFactory factory = new UpdateAttributeModelFactory();
final UpdateAttributeModelFactory factory = new UpdateAttributeModelFactory(variableRegistry);
final Criteria criteria = getCriteria(nifiWebContext, requestContext);
// attempt to locate the rule