mirror of https://github.com/apache/nifi.git
NIFI-9003 Added framework support for Parameter Providers
- Added Environment Variable and File Parameter Provider implementations - Added Parameter Provider commands to CLI - Added component search handling for Parameter Providers - Added documentation for integrating Parameter Providers This closes #5369 Signed-off-by: David Handermann <exceptionfactory@apache.org>
This commit is contained in:
parent
e81466d4f3
commit
de7793e69b
|
@ -32,6 +32,7 @@ public enum Component {
|
|||
ControllerService,
|
||||
ReportingTask,
|
||||
ParameterContext,
|
||||
ParameterProvider,
|
||||
AccessPolicy,
|
||||
User,
|
||||
UserGroup;
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.nifi.components.state.StateManager;
|
|||
|
||||
/**
|
||||
* <p>
|
||||
* Annotation that a Processor, ReportingTask, or Controller Service can use to indicate
|
||||
* Annotation that a Processor, ReportingTask, ParameterProvider, or Controller Service can use to indicate
|
||||
* that the component makes use of the {@link StateManager}. This annotation provides the
|
||||
* user with a description of what information is being stored so that the user is able to
|
||||
* understand what is shown to them and know what they are clearing should they choose to
|
||||
|
|
|
@ -25,7 +25,8 @@ import java.lang.annotation.Target;
|
|||
|
||||
/**
|
||||
* Annotation that may be placed on a {@link org.apache.nifi.processor.Processor Processor},
|
||||
* {@link org.apache.nifi.controller.ControllerService ControllerService}, or
|
||||
* {@link org.apache.nifi.controller.ControllerService ControllerService},
|
||||
* {@link org.apache.nifi.parameter.ParameterProvider ParameterProvider}, or
|
||||
* {@link org.apache.nifi.reporting.ReportingTask ReportingTask} allowing for a
|
||||
* description to be provided. This description can be provided to a user in
|
||||
* logs, UI, etc.
|
||||
|
|
|
@ -26,7 +26,8 @@ import org.apache.nifi.components.ConfigurableComponent;
|
|||
|
||||
/**
|
||||
* Annotation that may be placed on a null {@link org.apache.nifi.processor.Processor Processor},
|
||||
* {@link org.apache.nifi.controller.ControllerService ControllerService}, or
|
||||
* {@link org.apache.nifi.controller.ControllerService ControllerService},
|
||||
* {@link org.apache.nifi.parameter.ParameterProvider ParameterProvider}, or
|
||||
* {@link org.apache.nifi.reporting.ReportingTask ReportingTask} that indicates
|
||||
* this component is related to the components listed.
|
||||
*
|
||||
|
|
|
@ -25,7 +25,8 @@ import java.lang.annotation.Target;
|
|||
|
||||
/**
|
||||
* Annotation that can be applied to a {@link org.apache.nifi.processor.Processor Processor},
|
||||
* {@link org.apache.nifi.controller.ControllerService ControllerService}, or
|
||||
* {@link org.apache.nifi.controller.ControllerService ControllerService},
|
||||
* {@link org.apache.nifi.parameter.ParameterProvider ParameterProvider}, or
|
||||
* {@link org.apache.nifi.reporting.ReportingTask ReportingTask} in order to
|
||||
* associate tags (keywords) with the component. These tags do not affect the
|
||||
* component in any way but serve as additional documentation and can be used to
|
||||
|
|
|
@ -26,7 +26,8 @@ import java.lang.annotation.Target;
|
|||
/**
|
||||
* <p>
|
||||
* Marker annotation a {@link org.apache.nifi.processor.Processor Processor},
|
||||
* {@link org.apache.nifi.controller.ControllerService ControllerService}, or
|
||||
* {@link org.apache.nifi.controller.ControllerService ControllerService},
|
||||
* {@link org.apache.nifi.parameter.ParameterProvider ParameterProvider}, or
|
||||
* {@link org.apache.nifi.reporting.ReportingTask ReportingTask} implementation
|
||||
* can use to indicate a method should be called whenever the component is added
|
||||
* to the flow. This method will be called once for the entire life of a
|
||||
|
|
|
@ -43,7 +43,7 @@ import org.apache.nifi.controller.ConfigurationContext;
|
|||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* Note that this annotation will be ignored if applied to a ReportingTask or
|
||||
* Note that this annotation will be ignored if applied to a ReportingTask, ParameterProvider or
|
||||
* Processor. For a Controller Service, enabling and disabling are considered
|
||||
* lifecycle events, as the action makes them usable or unusable by other
|
||||
* components. However, for a Processor and a Reporting Task, these are not
|
||||
|
|
|
@ -50,7 +50,7 @@ import java.lang.annotation.Target;
|
|||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* Note that this annotation will be ignored if applied to a ReportingTask or
|
||||
* Note that this annotation will be ignored if applied to a ReportingTask, ParameterProvider or
|
||||
* Processor. For a Controller Service, enabling and disabling are considered
|
||||
* lifecycle events, as the action makes them usable or unusable by other
|
||||
* components. However, for a Processor and a Reporting Task, these are not
|
||||
|
|
|
@ -28,7 +28,8 @@ import org.apache.nifi.processor.ProcessContext;
|
|||
/**
|
||||
* <p>
|
||||
* Marker annotation a {@link org.apache.nifi.processor.Processor Processor},
|
||||
* {@link org.apache.nifi.controller.ControllerService ControllerService}, or
|
||||
* {@link org.apache.nifi.controller.ControllerService ControllerService},
|
||||
* {@link org.apache.nifi.parameter.ParameterProvider ParameterProvider}, or
|
||||
* {@link org.apache.nifi.reporting.ReportingTask ReportingTask} implementation
|
||||
* can use to indicate a method should be called whenever the component is
|
||||
* removed from the flow. This method will be called once for the entire life of
|
||||
|
@ -40,7 +41,7 @@ import org.apache.nifi.processor.ProcessContext;
|
|||
* <p>
|
||||
* Methods with this annotation are permitted to take no arguments or to take a
|
||||
* single argument. If using a single argument, that argument must be of type
|
||||
* {@link ConfigurationContext} if the component is a ReportingTask or a
|
||||
* {@link ConfigurationContext} if the component is a ReportingTask, a ParameterProvider, or a
|
||||
* ControllerService. If the component is a Processor, then the argument must be
|
||||
* of type {@link ProcessContext}.
|
||||
* </p>
|
||||
|
|
|
@ -28,7 +28,8 @@ import org.apache.nifi.processor.ProcessContext;
|
|||
/**
|
||||
* <p>
|
||||
* Marker annotation a {@link org.apache.nifi.processor.Processor Processor},
|
||||
* {@link org.apache.nifi.controller.ControllerService ControllerService}, or
|
||||
* {@link org.apache.nifi.controller.ControllerService ControllerService},
|
||||
* {@link org.apache.nifi.parameter.ParameterProvider ParameterProvider}, or
|
||||
* {@link org.apache.nifi.reporting.ReportingTask ReportingTask} implementation
|
||||
* can use to indicate a method should be called whenever the flow is being
|
||||
* shutdown. This will be called at most once for each component in a JVM
|
||||
|
|
|
@ -59,7 +59,7 @@ public interface ConfigurationContext extends PropertyContext {
|
|||
Long getSchedulingPeriod(TimeUnit timeUnit);
|
||||
|
||||
/**
|
||||
* Returns the component's (ControllerService, ReportingTask, e.g.) name
|
||||
* Returns the component's (ControllerService, ReportingTask, ParameterProvider, e.g.) name
|
||||
* @return the String name of this component
|
||||
*/
|
||||
String getName();
|
||||
|
|
|
@ -20,6 +20,7 @@ import org.apache.nifi.annotation.behavior.Stateful;
|
|||
import org.apache.nifi.components.ConfigurableComponent;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.PropertyValue;
|
||||
import org.apache.nifi.parameter.ParameterProvider;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSessionFactory;
|
||||
import org.apache.nifi.processor.Processor;
|
||||
|
@ -29,7 +30,7 @@ import org.apache.nifi.reporting.ReportingTask;
|
|||
/**
|
||||
* <p>
|
||||
* This interface provides a mechanism for creating services that are shared
|
||||
* among all {@link Processor}s, {@link ReportingTask}s, and other
|
||||
* among all {@link Processor}s, {@link ReportingTask}s, {@link ParameterProvider}s and other
|
||||
* {@code ControllerService}s.
|
||||
* </p>
|
||||
*
|
||||
|
|
|
@ -45,8 +45,10 @@ import org.apache.nifi.components.ConfigurableComponent;
|
|||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.controller.ControllerService;
|
||||
import org.apache.nifi.documentation.init.DocumentationControllerServiceInitializationContext;
|
||||
import org.apache.nifi.documentation.init.DocumentationParameterProviderInitializationContext;
|
||||
import org.apache.nifi.documentation.init.DocumentationProcessorInitializationContext;
|
||||
import org.apache.nifi.documentation.init.DocumentationReportingInitializationContext;
|
||||
import org.apache.nifi.parameter.ParameterProvider;
|
||||
import org.apache.nifi.processor.Processor;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
|
@ -86,6 +88,8 @@ public abstract class AbstractDocumentationWriter implements ExtensionDocumentat
|
|||
initialize((ControllerService) component);
|
||||
} else if (component instanceof ReportingTask) {
|
||||
initialize((ReportingTask) component);
|
||||
} else if (component instanceof ParameterProvider) {
|
||||
initialize((ParameterProvider) component);
|
||||
}
|
||||
} catch (final InitializationException ie) {
|
||||
throw new RuntimeException("Failed to initialize " + component, ie);
|
||||
|
@ -104,6 +108,10 @@ public abstract class AbstractDocumentationWriter implements ExtensionDocumentat
|
|||
reportingTask.initialize(new DocumentationReportingInitializationContext());
|
||||
}
|
||||
|
||||
protected void initialize(final ParameterProvider parameterProvider) throws InitializationException {
|
||||
parameterProvider.initialize(new DocumentationParameterProviderInitializationContext());
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void write(final ConfigurableComponent component) throws IOException {
|
||||
write(component, Collections.emptyList(), Collections.emptyMap());
|
||||
|
@ -254,6 +262,9 @@ public abstract class AbstractDocumentationWriter implements ExtensionDocumentat
|
|||
if (component instanceof ReportingTask) {
|
||||
return ExtensionType.REPORTING_TASK;
|
||||
}
|
||||
if (component instanceof ParameterProvider) {
|
||||
return ExtensionType.PARAMETER_PROVIDER;
|
||||
}
|
||||
throw new AssertionError("Encountered unknown Configurable Component Type for " + component);
|
||||
}
|
||||
|
||||
|
|
|
@ -21,5 +21,7 @@ public enum ExtensionType {
|
|||
|
||||
CONTROLLER_SERVICE,
|
||||
|
||||
REPORTING_TASK;
|
||||
REPORTING_TASK,
|
||||
|
||||
PARAMETER_PROVIDER;
|
||||
}
|
||||
|
|
|
@ -0,0 +1,66 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.documentation.init;
|
||||
|
||||
import org.apache.nifi.controller.NodeTypeProvider;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.parameter.ParameterProviderInitializationContext;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.UUID;
|
||||
|
||||
public class DocumentationParameterProviderInitializationContext implements ParameterProviderInitializationContext {
|
||||
private final String id = UUID.randomUUID().toString();
|
||||
private final ComponentLog componentLog = new NopComponentLog();
|
||||
private final NodeTypeProvider nodeTypeProvider = new StandaloneNodeTypeProvider();
|
||||
private final String name = "name";
|
||||
|
||||
@Override
|
||||
public String getIdentifier() {
|
||||
return id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ComponentLog getLogger() {
|
||||
return componentLog;
|
||||
}
|
||||
|
||||
@Override
|
||||
public NodeTypeProvider getNodeTypeProvider() {
|
||||
return nodeTypeProvider;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getKerberosServicePrincipal() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getKerberosServiceKeytab() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getKerberosConfigurationFile() {
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -32,6 +32,7 @@ public enum ComponentType {
|
|||
CONTROLLER_SERVICE("Controller Service"),
|
||||
REPORTING_TASK("Reporting Task"),
|
||||
PARAMETER_CONTEXT("Parameter Context"),
|
||||
PARAMETER_PROVIDER("Parameter Provider"),
|
||||
TEMPLATE("Template");
|
||||
|
||||
|
||||
|
|
|
@ -0,0 +1,66 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.flow;
|
||||
|
||||
import io.swagger.annotations.ApiModel;
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
import org.apache.nifi.flow.Bundle;
|
||||
|
||||
@ApiModel
|
||||
public class ParameterProviderReference {
|
||||
|
||||
private String identifier;
|
||||
private String name;
|
||||
private String type;
|
||||
private Bundle bundle;
|
||||
|
||||
@ApiModelProperty("The fully qualified name of the parameter provider class.")
|
||||
public String getType() {
|
||||
return type;
|
||||
}
|
||||
|
||||
public void setType(final String type) {
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The details of the artifact that bundled this parameter provider.")
|
||||
public Bundle getBundle() {
|
||||
return bundle;
|
||||
}
|
||||
|
||||
public void setBundle(final Bundle bundle) {
|
||||
this.bundle = bundle;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The identifier of the parameter provider")
|
||||
public String getIdentifier() {
|
||||
return identifier;
|
||||
}
|
||||
|
||||
public void setIdentifier(final String identifier) {
|
||||
this.identifier = identifier;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The name of the parameter provider")
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
public void setName(final String name) {
|
||||
this.name = name;
|
||||
}
|
||||
}
|
|
@ -23,6 +23,7 @@ public class VersionedExternalFlow {
|
|||
private VersionedProcessGroup flowContents;
|
||||
private Map<String, ExternalControllerServiceReference> externalControllerServices;
|
||||
private Map<String, VersionedParameterContext> parameterContexts;
|
||||
private Map<String, ParameterProviderReference> parameterProviders;
|
||||
private VersionedExternalFlowMetadata metadata;
|
||||
|
||||
public VersionedProcessGroup getFlowContents() {
|
||||
|
@ -56,4 +57,12 @@ public class VersionedExternalFlow {
|
|||
public void setMetadata(final VersionedExternalFlowMetadata metadata) {
|
||||
this.metadata = metadata;
|
||||
}
|
||||
|
||||
public Map<String, ParameterProviderReference> getParameterProviders() {
|
||||
return parameterProviders;
|
||||
}
|
||||
|
||||
public void setParameterProviders(final Map<String, ParameterProviderReference> parameterProviders) {
|
||||
this.parameterProviders = parameterProviders;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ public class VersionedParameter {
|
|||
private String name;
|
||||
private String description;
|
||||
private boolean sensitive;
|
||||
private boolean provided;
|
||||
private String value;
|
||||
|
||||
@ApiModelProperty("The name of the parameter")
|
||||
|
@ -54,6 +55,15 @@ public class VersionedParameter {
|
|||
this.sensitive = sensitive;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Whether or not the parameter value is provided by a ParameterProvider")
|
||||
public boolean isProvided() {
|
||||
return provided;
|
||||
}
|
||||
|
||||
public void setProvided(boolean provided) {
|
||||
this.provided = provided;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The value of the parameter")
|
||||
public String getValue() {
|
||||
return value;
|
||||
|
|
|
@ -25,6 +25,9 @@ public class VersionedParameterContext extends VersionedComponent {
|
|||
private Set<VersionedParameter> parameters;
|
||||
private List<String> inheritedParameterContexts;
|
||||
private String description;
|
||||
private String parameterProvider;
|
||||
private String parameterGroupName;
|
||||
private Boolean isSynchronized;
|
||||
|
||||
@ApiModelProperty("The description of the parameter context")
|
||||
public String getDescription() {
|
||||
|
@ -57,4 +60,31 @@ public class VersionedParameterContext extends VersionedComponent {
|
|||
public ComponentType getComponentType() {
|
||||
return ComponentType.PARAMETER_CONTEXT;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The identifier of an optional parameter provider")
|
||||
public String getParameterProvider() {
|
||||
return parameterProvider;
|
||||
}
|
||||
|
||||
public void setParameterProvider(String parameterProvider) {
|
||||
this.parameterProvider = parameterProvider;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The corresponding parameter group name fetched from the parameter provider, if applicable")
|
||||
public String getParameterGroupName() {
|
||||
return parameterGroupName;
|
||||
}
|
||||
|
||||
public void setParameterGroupName(String parameterGroupName) {
|
||||
this.parameterGroupName = parameterGroupName;
|
||||
}
|
||||
|
||||
@ApiModelProperty("True if the parameter provider is set and the context should receive updates when its parameters are next fetched")
|
||||
public Boolean isSynchronized() {
|
||||
return isSynchronized;
|
||||
}
|
||||
|
||||
public void setSynchronized(Boolean aSynchronized) {
|
||||
isSynchronized = aSynchronized;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,89 @@
|
|||
/*
|
||||
* 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.flow;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
public class VersionedParameterProvider extends VersionedComponent implements VersionedConfigurableComponent, VersionedExtensionComponent {
|
||||
private String type;
|
||||
private Bundle bundle;
|
||||
private Map<String, String> properties;
|
||||
private Map<String, VersionedPropertyDescriptor> propertyDescriptors;
|
||||
private String annotationData;
|
||||
|
||||
|
||||
@Override
|
||||
@ApiModelProperty(value = "The type of the parameter provider.")
|
||||
public String getType() {
|
||||
return type;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setType(final String type) {
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
@Override
|
||||
@ApiModelProperty(value = "The details of the artifact that bundled this parameter provider type.")
|
||||
public Bundle getBundle() {
|
||||
return bundle;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBundle(Bundle bundle) {
|
||||
this.bundle = bundle;
|
||||
}
|
||||
|
||||
@Override
|
||||
@ApiModelProperty(value = "The properties of the parameter provider.")
|
||||
public Map<String, String> getProperties() {
|
||||
return properties;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setProperties(Map<String, String> properties) {
|
||||
this.properties = properties;
|
||||
}
|
||||
|
||||
@Override
|
||||
@ApiModelProperty("The property descriptors for the parameter provider.")
|
||||
public Map<String, VersionedPropertyDescriptor> getPropertyDescriptors() {
|
||||
return propertyDescriptors;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPropertyDescriptors(Map<String, VersionedPropertyDescriptor> propertyDescriptors) {
|
||||
this.propertyDescriptors = propertyDescriptors;
|
||||
}
|
||||
|
||||
@ApiModelProperty(value = "The annotation for the parameter provider. This is how the custom UI relays configuration to the parameter provider.")
|
||||
public String getAnnotationData() {
|
||||
return annotationData;
|
||||
}
|
||||
|
||||
public void setAnnotationData(String annotationData) {
|
||||
this.annotationData = annotationData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ComponentType getComponentType() {
|
||||
return ComponentType.PARAMETER_PROVIDER;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* 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.parameter;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Objects;
|
||||
|
||||
public abstract class AbstractParameterGroup<T> {
|
||||
|
||||
private final String groupName;
|
||||
|
||||
private final Collection<T> items;
|
||||
|
||||
/**
|
||||
* Creates a named parameter group.
|
||||
* @param groupName The parameter group name
|
||||
* @param items A collection of grouped items
|
||||
*/
|
||||
protected AbstractParameterGroup(final String groupName, final Collection<T> items) {
|
||||
this.groupName = Objects.requireNonNull(groupName, "Group name is required");
|
||||
this.items = items;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The group name
|
||||
*/
|
||||
public String getGroupName() {
|
||||
return groupName;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The collection of grouped items
|
||||
*/
|
||||
public Collection<T> getItems() {
|
||||
return items;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,79 @@
|
|||
/*
|
||||
* 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.parameter;
|
||||
|
||||
import org.apache.nifi.components.AbstractConfigurableComponent;
|
||||
import org.apache.nifi.controller.NodeTypeProvider;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
|
||||
public abstract class AbstractParameterProvider extends AbstractConfigurableComponent implements ParameterProvider {
|
||||
private String identifier;
|
||||
private String name;
|
||||
private ComponentLog logger;
|
||||
private NodeTypeProvider nodeTypeProvider;
|
||||
|
||||
@Override
|
||||
public final void initialize(final ParameterProviderInitializationContext config) throws InitializationException {
|
||||
identifier = config.getIdentifier();
|
||||
logger = config.getLogger();
|
||||
name = config.getName();
|
||||
nodeTypeProvider = config.getNodeTypeProvider();
|
||||
init(config);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the identifier of this Parameter Provider
|
||||
*/
|
||||
@Override
|
||||
public String getIdentifier() {
|
||||
return identifier;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the {@link NodeTypeProvider} that was passed to the
|
||||
* {@link #initialize(ParameterProviderInitializationContext)} method
|
||||
*/
|
||||
protected final NodeTypeProvider getNodeTypeProvider() {
|
||||
return nodeTypeProvider;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the name of this Parameter Provider
|
||||
*/
|
||||
protected String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the logger that has been provided to the component by the
|
||||
* framework in its initialize method
|
||||
*/
|
||||
protected ComponentLog getLogger() {
|
||||
return logger;
|
||||
}
|
||||
|
||||
/**
|
||||
* Provides a mechanism by which subclasses can perform initialization of
|
||||
* the Parameter Provider before its parameters are fetched
|
||||
*
|
||||
* @param config context
|
||||
* @throws InitializationException if failure to init
|
||||
*/
|
||||
protected void init(final ParameterProviderInitializationContext config) throws InitializationException {
|
||||
}
|
||||
}
|
|
@ -22,19 +22,21 @@ public class Parameter {
|
|||
private final ParameterDescriptor descriptor;
|
||||
private final String value;
|
||||
private final String parameterContextId;
|
||||
private final boolean provided;
|
||||
|
||||
private Parameter(final ParameterDescriptor descriptor, final String value, final String parameterContextId) {
|
||||
public Parameter(final ParameterDescriptor descriptor, final String value, final String parameterContextId, final Boolean provided) {
|
||||
this.descriptor = descriptor;
|
||||
this.value = value;
|
||||
this.parameterContextId = parameterContextId;
|
||||
this.provided = provided == null ? false : provided.booleanValue();
|
||||
}
|
||||
|
||||
public Parameter(final Parameter parameter, final String parameterContextId) {
|
||||
this(parameter.getDescriptor(), parameter.getValue(), parameterContextId);
|
||||
this(parameter.getDescriptor(), parameter.getValue(), parameterContextId, parameter.isProvided());
|
||||
}
|
||||
|
||||
public Parameter(final ParameterDescriptor descriptor, final String value) {
|
||||
this(descriptor, value, null);
|
||||
this(descriptor, value, null, false);
|
||||
}
|
||||
|
||||
public ParameterDescriptor getDescriptor() {
|
||||
|
@ -67,4 +69,12 @@ public class Parameter {
|
|||
public int hashCode() {
|
||||
return Objects.hash(descriptor, value);
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @return True if this parameter is provided by a ParameterProvider.
|
||||
*/
|
||||
public boolean isProvided() {
|
||||
return provided;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,54 @@
|
|||
/*
|
||||
* 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.parameter;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Encapsulates a named group of externally fetched parameters that can be provided to referencing Parameter Contexts.
|
||||
*/
|
||||
public class ParameterGroup {
|
||||
|
||||
private final String groupName;
|
||||
|
||||
private final List<Parameter> parameters;
|
||||
|
||||
/**
|
||||
* Creates a named parameter group.
|
||||
* @param groupName The parameter group name
|
||||
* @param parameters A list of parameters
|
||||
*/
|
||||
public ParameterGroup(final String groupName, final List<Parameter> parameters) {
|
||||
this.groupName = groupName;
|
||||
this.parameters = Collections.unmodifiableList(parameters);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The group name
|
||||
*/
|
||||
public String getGroupName() {
|
||||
return groupName;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The provided parameters
|
||||
*/
|
||||
public List<Parameter> getParameters() {
|
||||
return parameters;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,100 @@
|
|||
/*
|
||||
* 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.parameter;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* User-provided configuration for a group of parameters fetched from a ParameterProvider.
|
||||
*/
|
||||
public class ParameterGroupConfiguration implements Comparable<ParameterGroupConfiguration> {
|
||||
|
||||
private final String groupName;
|
||||
|
||||
private final String parameterContextName;
|
||||
|
||||
private final Map<String, ParameterSensitivity> parameterSensitivities;
|
||||
|
||||
private final Boolean isSynchronized;
|
||||
|
||||
/**
|
||||
* Creates a named group of parameter names.
|
||||
* @param groupName The parameter group name
|
||||
* @param parameterContextName The parameter context name to which parameters will be applied
|
||||
* @param parameterSensitivities A map from parameter name to desired sensitivity. Any parameter not included in this map will not be included
|
||||
* when applied to the parameter context.
|
||||
* @param isSynchronized If true, indicates that a ParameterContext should be created if not already existing, or updated if existing
|
||||
*/
|
||||
public ParameterGroupConfiguration(final String groupName, final String parameterContextName, final Map<String, ParameterSensitivity> parameterSensitivities,
|
||||
final Boolean isSynchronized) {
|
||||
this.groupName = Objects.requireNonNull(groupName, "Parameter group name is required");
|
||||
this.parameterContextName = Optional.ofNullable(parameterContextName).orElse(groupName);
|
||||
this.parameterSensitivities = Collections.unmodifiableMap(Objects.requireNonNull(parameterSensitivities, "Parameter sensitivity map is required"));
|
||||
this.isSynchronized = isSynchronized;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The external parameter group name
|
||||
*/
|
||||
public String getGroupName() {
|
||||
return groupName;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The name of the ParameterContext that maps to this group
|
||||
*/
|
||||
public String getParameterContextName() {
|
||||
return parameterContextName;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return A map from parameter name to desired sensitivity. If the sensitivity is null, this indicates that the parameter
|
||||
* has not yet been configured by the user.
|
||||
*/
|
||||
public Map<String, ParameterSensitivity> getParameterSensitivities() {
|
||||
return parameterSensitivities;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return True if this group should be synchronized with a parameter context. If null, this indicates that
|
||||
* it has not yet been configured by the user.
|
||||
*/
|
||||
public Boolean isSynchronized() {
|
||||
return isSynchronized;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(final ParameterGroupConfiguration other) {
|
||||
if (other == null) {
|
||||
return -1;
|
||||
}
|
||||
|
||||
final String groupName = getGroupName();
|
||||
final String otherGroupName = other.getGroupName();
|
||||
|
||||
if (groupName == null) {
|
||||
return otherGroupName == null ? 0 : -1;
|
||||
}
|
||||
if (otherGroupName == null) {
|
||||
return 1;
|
||||
}
|
||||
return groupName.compareTo(otherGroupName);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,88 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.parameter;
|
||||
|
||||
import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored;
|
||||
import org.apache.nifi.components.ConfigurableComponent;
|
||||
import org.apache.nifi.controller.ConfigurationContext;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Defines a provider that is responsible for fetching from an external source Parameters with
|
||||
* which a ParameterContext can be populated.
|
||||
*
|
||||
* <p>
|
||||
* <code>ParameterProvider</code>s are discovered following Java's
|
||||
* <code>ServiceLoader</code> pattern. As a result, all implementations must
|
||||
* follow these rules:
|
||||
*
|
||||
* <ul>
|
||||
* <li>The implementation must implement this interface.</li>
|
||||
* <li>The implementation must have a file named
|
||||
* org.apache.nifi.parameter.ParameterProvider located within the jar's
|
||||
* <code>META-INF/services</code> directory. This file contains a list of
|
||||
* fully-qualified class names of all <code>ParameterProvider</code>s in the jar,
|
||||
* one-per-line.
|
||||
* <li>The implementation must support a default constructor.</li>
|
||||
* </ul>
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* All implementations of this interface must be thread-safe.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* Parameter Providers may choose to annotate a method with the
|
||||
* {@link OnConfigurationRestored @OnConfigurationRestored} annotation. If this is done, that method
|
||||
* will be invoked after all properties have been set for the ParameterProvider and
|
||||
* before its parameters are fetched.
|
||||
* </p>
|
||||
*/
|
||||
public interface ParameterProvider extends ConfigurableComponent {
|
||||
|
||||
/**
|
||||
* Provides the Parameter Provider with access to objects that may be of use
|
||||
* throughout the life of the provider
|
||||
*
|
||||
* @param config of initialization context
|
||||
* @throws org.apache.nifi.reporting.InitializationException if unable to init
|
||||
*/
|
||||
void initialize(ParameterProviderInitializationContext config) throws InitializationException;
|
||||
|
||||
/**
|
||||
* Fetches named groups of parameters from an external source.
|
||||
*
|
||||
* Any referencing Parameter Context will only receive the Parameters from a group if the Parameter Context name matches
|
||||
* the group name (ignoring case) and the reference sensitivity matches the group sensitivity.
|
||||
*
|
||||
* If group name is null, all referencing Parameter Contexts will receive the Parameters in that group,
|
||||
* regardless of their name.
|
||||
*
|
||||
* If more than one ProvidedParameterGroup matches a given ParameterContext, all parameters from these groups will be
|
||||
* applied. However, if any parameters among these matching groups have the same name but different value, the framework
|
||||
* will throw a <code>RuntimeException</code>
|
||||
*
|
||||
* @param context The <code>ConfigurationContext</code>for the provider
|
||||
* @return A list of fetched Parameter groups. The framework will set the sensitivity appropriately based on how the ParameterProvider
|
||||
* is referenced in a ParameterContext.
|
||||
* @throws IOException if there is an I/O problem while fetching the Parameters
|
||||
*/
|
||||
List<ParameterGroup> fetchParameters(ConfigurationContext context) throws IOException;
|
||||
}
|
|
@ -0,0 +1,47 @@
|
|||
/*
|
||||
* 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.parameter;
|
||||
|
||||
import org.apache.nifi.controller.NodeTypeProvider;
|
||||
import org.apache.nifi.kerberos.KerberosContext;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
|
||||
public interface ParameterProviderInitializationContext extends KerberosContext {
|
||||
|
||||
/**
|
||||
* @return the identifier associated with the {@link ParameterProvider} with
|
||||
* which this context is associated
|
||||
*/
|
||||
String getIdentifier();
|
||||
|
||||
/**
|
||||
* @return the configured name for this ParameterProvider
|
||||
*/
|
||||
String getName();
|
||||
|
||||
/**
|
||||
* @return a logger that can be used to log important events in a standard
|
||||
* way and generate bulletins when appropriate
|
||||
*/
|
||||
ComponentLog getLogger();
|
||||
|
||||
/**
|
||||
* @return the {@link NodeTypeProvider} which can be used to detect the node
|
||||
* type of this NiFi instance.
|
||||
*/
|
||||
NodeTypeProvider getNodeTypeProvider();
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.parameter;
|
||||
|
||||
/**
|
||||
* Indicates the sensitivity of a parameter.
|
||||
*/
|
||||
public enum ParameterSensitivity {
|
||||
SENSITIVE("Sensitive"),
|
||||
NON_SENSITIVE("Non-Sensitive");
|
||||
|
||||
private String name;
|
||||
|
||||
ParameterSensitivity(final String name) {
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,63 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.parameter;
|
||||
|
||||
import org.apache.nifi.components.ConfigVerificationResult;
|
||||
import org.apache.nifi.controller.ConfigurationContext;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Any Parameter Provider that implements this interface will be provided the opportunity to verify
|
||||
* a given configuration of the Parameter Provider. This allows the Parameter Provider to provide meaningful feedback
|
||||
* to users when configuring the dataflow.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* Generally speaking, verification differs from validation in that validation is expected to be very
|
||||
* quick and run often. If a Parameter Provider is not valid, its parameters cannot be fetched. However, verification may be
|
||||
* more expensive or time-consuming to complete. For example, validation may ensure that a username is
|
||||
* provided for connecting to an external service but should not perform any sort of network connection
|
||||
* in order to verify that the username is accurate. Verification, on the other hand, may create resources
|
||||
* such as network connections, may be more expensive to complete, and may be run only when a user invokes
|
||||
* the action (though verification may later occur at other stages, such as when fetching the parameters).
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* The framework is not responsible for triggering the Lifecycle management stages, such as @OnScheduled before triggering the verification. Such
|
||||
* methods should be handled by the {@link #verify(ConfigurationContext, ComponentLog)} itself.
|
||||
* The {@link #verify(ConfigurationContext, ComponentLog)} method will only be called if the configuration is valid according to the
|
||||
* validation rules (i.e., all Property Descriptors' validators and customValidate methods have indicated that the configuration is valid).
|
||||
* </p>
|
||||
*/
|
||||
public interface VerifiableParameterProvider {
|
||||
|
||||
/**
|
||||
* Verifies that the configuration defined by the given ConfigurationContext is valid.
|
||||
*
|
||||
* @param context the Configuration Context that contains the necessary configuration
|
||||
* @param verificationLogger a logger that can be used during verification. While the typical logger can be used, doing so may result
|
||||
* in producing bulletins, which can be confusing.
|
||||
*
|
||||
* @return a List of ConfigVerificationResults, each illustrating one step of the verification process that was completed
|
||||
*/
|
||||
List<ConfigVerificationResult> verify(ConfigurationContext context, ComponentLog verificationLogger);
|
||||
|
||||
}
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.reporting;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.components.AbstractConfigurableComponent;
|
||||
import org.apache.nifi.controller.ConfigurationContext;
|
||||
|
@ -24,6 +23,8 @@ import org.apache.nifi.controller.ControllerServiceLookup;
|
|||
import org.apache.nifi.controller.NodeTypeProvider;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public abstract class AbstractReportingTask extends AbstractConfigurableComponent implements ReportingTask {
|
||||
|
||||
private String identifier;
|
||||
|
|
|
@ -46,6 +46,11 @@ public enum ComponentType {
|
|||
*/
|
||||
REPORTING_TASK,
|
||||
|
||||
/**
|
||||
* Bulletin is associated with a Parameter Provider
|
||||
*/
|
||||
PARAMETER_PROVIDER,
|
||||
|
||||
/**
|
||||
* Bulletin is associated with a Controller Service
|
||||
*/
|
||||
|
|
|
@ -62,9 +62,13 @@ high-level description of the most common extension points:
|
|||
monitoring information, and internal NiFi state to be published to external
|
||||
endpoints, such as log files, e-mail, and remote web services.
|
||||
|
||||
- ParameterProvider
|
||||
* The ParameterProvider interface allows Parameters to be provided by external sources. Provided parameters
|
||||
are still stored in Parameter Contexts, but this mechanism allows them to be provided and managed externally.
|
||||
|
||||
- ControllerService
|
||||
* A ControllerService provides shared state and functionality across Processors, other ControllerServices,
|
||||
and ReportingTasks within a single JVM. An example use case may include loading a very
|
||||
Parameter Providers, and ReportingTasks within a single JVM. An example use case may include loading a very
|
||||
large dataset into memory. By performing this work in a ControllerService, the data
|
||||
can be loaded once and be exposed to all Processors via this service, rather than requiring
|
||||
many different Processors to load the dataset themselves.
|
||||
|
@ -177,7 +181,7 @@ Processors to run without consuming resources unnecessarily.
|
|||
[[property_descriptor]]
|
||||
==== PropertyDescriptor
|
||||
PropertyDescriptor defines a property that is to be used by a
|
||||
Processor, ReportingTask, or ControllerService.
|
||||
Processor, ReportingTask, ParameterProvider, or ControllerService.
|
||||
The definition of a property includes its name, a description of the
|
||||
property, an optional default value,
|
||||
validation logic, and an indicator as to whether or not the property
|
||||
|
@ -511,7 +515,7 @@ Annotations may be applied to Java methods in a NiFi component to
|
|||
indicate to
|
||||
the framework when the methods should be called. For the discussion of
|
||||
Component Lifecycle, we will define a NiFi component as a
|
||||
Processor, ControllerServices, or ReportingTask.
|
||||
Processor, ControllerServices, ParameterProviders, or ReportingTask.
|
||||
|
||||
==== @OnAdded
|
||||
|
||||
|
@ -546,9 +550,9 @@ without throwing anything.
|
|||
Methods using this annotation must take either 0 arguments or a single argument
|
||||
of type `org.apache.nifi.controller.ConfigurationContext`.
|
||||
|
||||
Note that this annotation will be ignored if applied to a ReportingTask or
|
||||
Processor. For a Controller Service, enabling and disabling are considered
|
||||
lifecycle events, as the action makes them usable or unusable by other
|
||||
Note that this annotation will be ignored if applied to a ReportingTask,
|
||||
Parameter Provider, or Processor. For a Controller Service, enabling and disabling
|
||||
are considered lifecycle events, as the action makes them usable or unusable by other
|
||||
components. However, for a Processor and a Reporting Task, these are not
|
||||
lifecycle events but rather a mechanism to allow a component to be excluded
|
||||
when starting or stopping a group of components.
|
||||
|
@ -910,7 +914,7 @@ Often Processors and ControllerServices are related to one another. Sometimes i
|
|||
Sometimes a Processor uses a ControllerService like `InvokeHTTP` and `StandardSSLContextService`. Sometimes one ControllerService uses another
|
||||
like `DistributedMapCacheClientService` and `DistributedMapCacheServer`. Developers of these extension points may relate these
|
||||
different components using the `SeeAlso` tag. This annotation links these components in the documentation.
|
||||
`SeeAlso` can be applied to Processors, ControllerServices and ReportingTasks. An example of how to do this is listed below:
|
||||
`SeeAlso` can be applied to Processors, ControllerServices, ParameterProviders, and ReportingTasks. An example of how to do this is listed below:
|
||||
|
||||
[source, java]
|
||||
----
|
||||
|
@ -939,7 +943,7 @@ to provide a rich explanation of what this Processor is doing, what kind of
|
|||
data it expects and produces, and what FlowFile attributes it expects and produces.
|
||||
Because this documentation is in an HTML format, you may include images and tables
|
||||
to best describe this component. The same methods can be used to provide advanced
|
||||
documentation for Processors, ControllerServices and ReportingTasks.
|
||||
documentation for Processors, ControllerServices, ParameterProviders, and ReportingTasks.
|
||||
|
||||
[[provenance_events]]
|
||||
== Provenance Events
|
||||
|
@ -1928,14 +1932,15 @@ Controller Service's interface. See <<nars>> for more information.
|
|||
=== Interacting with a ControllerService
|
||||
|
||||
ControllerServices may be obtained by a Processor, another
|
||||
ControllerService, or a ReportingTask
|
||||
ControllerService, a ParameterProvider, or a ReportingTask
|
||||
by means of the ControllerServiceLookup or by using the
|
||||
`identifiesControllerService` method of the
|
||||
PropertyDescriptor's Builder class. The ControllerServiceLookup can be
|
||||
obtained by a Processor from the
|
||||
ProcessorInitializationContext that is passed to the `initialize`
|
||||
method. Likewise, it is obtained by
|
||||
a ControllerService from the ControllerServiceInitializationContext
|
||||
a ControllerService from the ControllerServiceInitializationContext,
|
||||
a ParameterProvider from the ParameterProviderInitializationContext,
|
||||
and by a ReportingTask via the
|
||||
ReportingConfiguration object passed to the `initialize` method.
|
||||
|
||||
|
@ -2005,7 +2010,7 @@ ReportingTask interface exposes methods for
|
|||
configuration, validation, and initialization. These methods are all
|
||||
identical to those of the
|
||||
Processor and ControllerService interfaces except that the
|
||||
`initialize` method is passed a `ReportingConfiguration`
|
||||
`initialize` method is passed a `ReportingInitializationContext`
|
||||
object, as opposed to the initialization objects received by the other
|
||||
Components. The ReportingTask also has
|
||||
an `onTrigger` method that is invoked by the framework to trigger the
|
||||
|
@ -2049,6 +2054,44 @@ ReportingTasks, allowing reports to be generated
|
|||
in many different ways to expose metrics and monitoring capabilities
|
||||
needed for any number of operational concerns.
|
||||
|
||||
== Parameter Providers
|
||||
|
||||
Although Parameter Contexts cannot be extended, the `ParameterProvider`
|
||||
interface allows an extension point for providing parameters to
|
||||
Parameter Contexts. Parameter Providers can fetch groups of parameters
|
||||
that can be mapped to new Parameter Contexts, which can then be kept
|
||||
up to date by re-fetching the parameters.
|
||||
|
||||
This extensibility allows parameters to be both initially provided and
|
||||
managed external to NiFi.
|
||||
|
||||
|
||||
=== Developing a Parameter Provider
|
||||
|
||||
The ParameterProvider interface exposes methods for
|
||||
configuration, validation, and initialization. These methods are all
|
||||
identical to those of the Processor and ControllerService interfaces except that the
|
||||
`initialize` method is passed a `ParameterProviderInitializationContext`
|
||||
object, as opposed to the initialization objects received by the other
|
||||
Components. The ParameterProvider also has a `fetchParameters` method that is invoked
|
||||
by the framework to fetch the parameters from its source.
|
||||
|
||||
Fetched parameters are returned in `ParameterGroup` s, each of which has
|
||||
a group name and a list of parameters. Parameter Groups may be mapped by the user
|
||||
to Parameter Contexts. This allows a single `ParameterProvider` instance to provide different
|
||||
parameters to multiple Parameter Contexts based on their mappings.
|
||||
|
||||
Within the `fetchParameters` method, the ParameterProvider is given access to a
|
||||
`ConfigurationContext`, from which configuration
|
||||
and information about the NiFi instance can be obtained. The
|
||||
ControllerServiceLookup that is accessible via the Context provides
|
||||
access to ControllerServices that have been
|
||||
configured. However, this method of obtaining Controller Services is
|
||||
not the preferred method. Rather, the
|
||||
preferred method for obtaining a Controller Service is to reference
|
||||
the Controller Service in a PropertyDescriptor,
|
||||
as is discussed in the <<interacting-with-controller-service>> section.
|
||||
|
||||
== UI Extensions
|
||||
|
||||
There are two UI extension points that are available in NiFi:
|
||||
|
|
|
@ -84,6 +84,7 @@ The following are available commands:
|
|||
nifi pg-import
|
||||
nifi pg-start
|
||||
nifi pg-stop
|
||||
nifi pg-create
|
||||
nifi pg-get-vars
|
||||
nifi pg-set-var
|
||||
nifi pg-get-version
|
||||
|
@ -92,40 +93,55 @@ The following are available commands:
|
|||
nifi pg-list
|
||||
nifi pg-status
|
||||
nifi pg-get-services
|
||||
nifi pg-create-service
|
||||
nifi pg-enable-services
|
||||
nifi pg-disable-services
|
||||
nifi pg-create-service
|
||||
nifi create-user
|
||||
nifi pg-get-param-context
|
||||
nifi pg-set-param-context
|
||||
nifi pg-replace
|
||||
nifi get-services
|
||||
nifi get-service
|
||||
nifi create-service
|
||||
nifi enable-services
|
||||
nifi disable-services
|
||||
nifi get-reporting-tasks
|
||||
nifi get-reporting-task
|
||||
nifi create-reporting-task
|
||||
nifi delete-reporting-task
|
||||
nifi start-reporting-tasks
|
||||
nifi stop-reporting-tasks
|
||||
nifi list-users
|
||||
nifi create-user-group
|
||||
nifi create-user
|
||||
nifi list-user-groups
|
||||
nifi create-user-group
|
||||
nifi update-user-group
|
||||
nifi get-policy
|
||||
nifi update-policy
|
||||
nifi create-service
|
||||
nifi get-services
|
||||
nifi get-service
|
||||
nifi disable-services
|
||||
nifi enable-services
|
||||
nifi get-reporting-task
|
||||
nifi get-reporting-tasks
|
||||
nifi create-reporting-task
|
||||
nifi delete-reporting-task
|
||||
nifi set-param
|
||||
nifi delete-param
|
||||
nifi list-templates
|
||||
nifi download-template
|
||||
nifi upload-template
|
||||
nifi list-param-contexts
|
||||
nifi get-param-context
|
||||
nifi create-param-context
|
||||
nifi delete-param-context
|
||||
nifi merge-param-context
|
||||
nifi set-inherited-param-contexts
|
||||
nifi remove-inherited-param-contexts
|
||||
nifi set-param-provider-reference
|
||||
nifi remove-param-provider-reference
|
||||
nifi set-param
|
||||
nifi delete-param
|
||||
nifi export-param-context
|
||||
nifi import-param-context
|
||||
nifi pg-get-param-context
|
||||
nifi pg-set-param-context
|
||||
nifi list-templates
|
||||
nifi download-template
|
||||
nifi upload-template
|
||||
nifi start-reporting-tasks
|
||||
nifi stop-reporting-tasks
|
||||
nifi merge-param-context
|
||||
nifi list-param-providers
|
||||
nifi get-param-provider
|
||||
nifi create-param-provider
|
||||
nifi delete-param-provider
|
||||
nifi fetch-params
|
||||
nifi set-param-provider-property
|
||||
nifi get-access-token
|
||||
nifi get-access-token-spnego
|
||||
nifi logout-access-token
|
||||
registry current-user
|
||||
registry list-buckets
|
||||
registry create-bucket
|
||||
|
@ -157,6 +173,9 @@ The following are available commands:
|
|||
registry get-policy
|
||||
registry update-policy
|
||||
registry update-bucket-policy
|
||||
registry get-access-token
|
||||
registry get-access-token-spnego
|
||||
registry logout-access-token
|
||||
session keys
|
||||
session show
|
||||
session get
|
||||
|
|
|
@ -93,6 +93,8 @@ UI may become unavailable.
|
|||
|
||||
*Reporting Task*: Reporting Tasks run in the background to provide statistical reports about what is happening in the NiFi instance. The DFM adds and configures Reporting Tasks in the User Interface as desired. Common reporting tasks include the ControllerStatusReportingTask, MonitorDiskUsage reporting task, MonitorMemory reporting task, and the StandardGangliaReporter.
|
||||
|
||||
*Parameter Provider*: Parameter Providers can provide parameters from an external source to Parameter Contexts. The parameters of a Parameter Provider may be fetched and applied to all referencing Parameter Contexts.
|
||||
|
||||
*Funnel*: A funnel is a NiFi component that is used to combine the data from several Connections into a single Connection.
|
||||
|
||||
*Process Group*: When a dataflow becomes complex, it often is beneficial to reason about the dataflow at a higher, more abstract level.
|
||||
|
|
|
@ -28,5 +28,6 @@ public enum UiExtensionType {
|
|||
ContentViewer,
|
||||
ProcessorConfiguration,
|
||||
ControllerServiceConfiguration,
|
||||
ReportingTaskConfiguration
|
||||
ReportingTaskConfiguration,
|
||||
ParameterProviderConfiguration
|
||||
}
|
||||
|
|
|
@ -0,0 +1,96 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.util;
|
||||
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.controller.ControllerServiceLookup;
|
||||
import org.apache.nifi.controller.NodeTypeProvider;
|
||||
import org.apache.nifi.kerberos.KerberosContext;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.parameter.ParameterProviderInitializationContext;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class MockParameterProviderInitializationContext extends MockControllerServiceLookup implements ParameterProviderInitializationContext, ControllerServiceLookup {
|
||||
|
||||
private final String identifier;
|
||||
private final String name;
|
||||
private final Map<PropertyDescriptor, String> properties = new HashMap<>();
|
||||
private final ComponentLog logger;
|
||||
private final KerberosContext kerberosContext;
|
||||
|
||||
public MockParameterProviderInitializationContext(final String identifier, final String name, final ComponentLog logger) {
|
||||
this(identifier, name, logger, null);
|
||||
}
|
||||
|
||||
public MockParameterProviderInitializationContext(final String identifier, final String name, final ComponentLog logger, final KerberosContext kerberosContext) {
|
||||
this.identifier = identifier;
|
||||
this.name = name;
|
||||
this.logger = logger;
|
||||
this.kerberosContext = kerberosContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getIdentifier() {
|
||||
return identifier;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
public void setProperty(final String propertyName, final String value) {
|
||||
setProperty(new PropertyDescriptor.Builder().name(propertyName).build(), value);
|
||||
}
|
||||
|
||||
public void setProperty(final PropertyDescriptor propertyName, final String value) {
|
||||
this.properties.put(propertyName, value);
|
||||
}
|
||||
|
||||
public void setProperties(final Map<PropertyDescriptor, String> properties) {
|
||||
this.properties.clear();
|
||||
this.properties.putAll(properties);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NodeTypeProvider getNodeTypeProvider() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ComponentLog getLogger() {
|
||||
return logger;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getKerberosServicePrincipal() {
|
||||
return kerberosContext != null ? kerberosContext.getKerberosServicePrincipal() : null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getKerberosServiceKeytab() {
|
||||
return kerberosContext != null ? kerberosContext.getKerberosServiceKeytab() : null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getKerberosConfigurationFile() {
|
||||
return kerberosContext != null ? kerberosContext.getKerberosConfigurationFile() : null;
|
||||
}
|
||||
}
|
|
@ -568,7 +568,8 @@ public class StandardActionDAO implements ActionDAO {
|
|||
|
||||
// get the component details if appropriate
|
||||
ComponentDetails componentDetails = null;
|
||||
if (Component.Processor.equals(component) || Component.ControllerService.equals(component) || Component.ReportingTask.equals(component)) {
|
||||
if (Component.Processor.equals(component) || Component.ControllerService.equals(component) || Component.ReportingTask.equals(component)
|
||||
|| Component.ParameterProvider.equals(component)) {
|
||||
componentDetails = getExtensionDetails(actionId);
|
||||
} else if (Component.RemoteProcessGroup.equals(component)) {
|
||||
componentDetails = getRemoteProcessGroupDetails(actionId);
|
||||
|
@ -641,7 +642,8 @@ public class StandardActionDAO implements ActionDAO {
|
|||
|
||||
// get the component details if appropriate
|
||||
ComponentDetails componentDetails = null;
|
||||
if (Component.Processor.equals(component) || Component.ControllerService.equals(component) || Component.ReportingTask.equals(component)) {
|
||||
if (Component.Processor.equals(component) || Component.ControllerService.equals(component) || Component.ReportingTask.equals(component)
|
||||
|| Component.ParameterProvider.equals(component)) {
|
||||
componentDetails = getExtensionDetails(actionId);
|
||||
} else if (Component.RemoteProcessGroup.equals(component)) {
|
||||
componentDetails = getRemoteProcessGroupDetails(actionId);
|
||||
|
|
|
@ -120,7 +120,7 @@ public class ControllerServiceReferencingComponentDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return type of reference this is (Processor, ControllerService, or ReportingTask)
|
||||
* @return type of reference this is (Processor, ControllerService, ParameterProvider, or ReportingTask)
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The type of reference this is.",
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.nifi.web.api.dto;
|
|||
import io.swagger.annotations.ApiModelProperty;
|
||||
import org.apache.nifi.web.api.entity.ParameterContextReferenceEntity;
|
||||
import org.apache.nifi.web.api.entity.ParameterEntity;
|
||||
import org.apache.nifi.web.api.entity.ParameterProviderConfigurationEntity;
|
||||
import org.apache.nifi.web.api.entity.ProcessGroupEntity;
|
||||
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
|
@ -33,6 +34,7 @@ public class ParameterContextDTO {
|
|||
private Set<ParameterEntity> parameters;
|
||||
private Set<ProcessGroupEntity> boundProcessGroups;
|
||||
private List<ParameterContextReferenceEntity> inheritedParameterContexts;
|
||||
private ParameterProviderConfigurationEntity parameterProviderConfiguration;
|
||||
|
||||
public void setId(String id) {
|
||||
this.identifier = id;
|
||||
|
@ -88,6 +90,15 @@ public class ParameterContextDTO {
|
|||
return boundProcessGroups;
|
||||
}
|
||||
|
||||
@ApiModelProperty(value = "Optional configuration for a Parameter Provider")
|
||||
public ParameterProviderConfigurationEntity getParameterProviderConfiguration() {
|
||||
return parameterProviderConfiguration;
|
||||
}
|
||||
|
||||
public void setParameterProviderConfiguration(final ParameterProviderConfigurationEntity parameterProviderConfiguration) {
|
||||
this.parameterProviderConfiguration = parameterProviderConfiguration;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "ParameterContext[id=" + identifier + ", name=" + name + ", parameters=" + parameters + "]";
|
||||
|
|
|
@ -30,6 +30,7 @@ public class ParameterDTO {
|
|||
private Boolean sensitive;
|
||||
private String value;
|
||||
private Boolean valueRemoved;
|
||||
private Boolean provided;
|
||||
private Set<AffectedComponentEntity> referencingComponents;
|
||||
private ParameterContextReferenceEntity parameterContext;
|
||||
private Boolean inherited;
|
||||
|
@ -70,6 +71,15 @@ public class ParameterDTO {
|
|||
this.sensitive = sensitive;
|
||||
}
|
||||
|
||||
@ApiModelProperty("Whether or not the Parameter is provided by a ParameterProvider")
|
||||
public Boolean getProvided() {
|
||||
return provided;
|
||||
}
|
||||
|
||||
public void setProvided(final Boolean provided) {
|
||||
this.provided = provided;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The value of the Parameter")
|
||||
public String getValue() {
|
||||
return value;
|
||||
|
@ -109,6 +119,6 @@ public class ParameterDTO {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "ParameterDTO[name=" + name + ", sensitive=" + sensitive + ", value=" + (sensitive ? "********" : value) + "]";
|
||||
return "ParameterDTO[name=" + name + ", sensitive=" + sensitive + ", value=" + (sensitive ? "********" : value) + (provided ? " (provided)" : "") + "]";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,61 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.dto;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
import org.apache.nifi.web.api.entity.AffectedComponentEntity;
|
||||
import org.apache.nifi.web.api.entity.ParameterContextUpdateEntity;
|
||||
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
@XmlType(name = "parameterProviderApplyParametersRequest")
|
||||
public class ParameterProviderApplyParametersRequestDTO extends AsynchronousRequestDTO<ParameterProviderApplyParametersUpdateStepDTO> {
|
||||
private ParameterProviderDTO parameterProvider;
|
||||
private List<ParameterContextUpdateEntity> parameterContextUpdates;
|
||||
private Set<AffectedComponentEntity> referencingComponents;
|
||||
|
||||
@ApiModelProperty(value = "The Parameter Contexts updated by this Parameter Provider. This may not be populated until the request has successfully completed.",
|
||||
accessMode = ApiModelProperty.AccessMode.READ_ONLY)
|
||||
public List<ParameterContextUpdateEntity> getParameterContextUpdates() {
|
||||
return parameterContextUpdates;
|
||||
}
|
||||
|
||||
public void setParameterContextUpdates(final List<ParameterContextUpdateEntity> parameterContextUpdates) {
|
||||
this.parameterContextUpdates = parameterContextUpdates;
|
||||
}
|
||||
|
||||
@ApiModelProperty(value = "The Parameter Provider that is being operated on. This may not be populated until the request has successfully completed.",
|
||||
accessMode = ApiModelProperty.AccessMode.READ_ONLY)
|
||||
public ParameterProviderDTO getParameterProvider() {
|
||||
return parameterProvider;
|
||||
}
|
||||
|
||||
public void setParameterProvider(final ParameterProviderDTO parameterProvider) {
|
||||
this.parameterProvider = parameterProvider;
|
||||
}
|
||||
|
||||
@ApiModelProperty(value = "The components that are referenced by the update.", accessMode = ApiModelProperty.AccessMode.READ_ONLY)
|
||||
public Set<AffectedComponentEntity> getReferencingComponents() {
|
||||
return referencingComponents;
|
||||
}
|
||||
|
||||
public void setReferencingComponents(final Set<AffectedComponentEntity> referencingComponents) {
|
||||
this.referencingComponents = referencingComponents;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,23 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.dto;
|
||||
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
|
||||
@XmlType(name = "parameterProviderApplyParametersUpdateStep")
|
||||
public class ParameterProviderApplyParametersUpdateStepDTO extends UpdateStepDTO {
|
||||
}
|
|
@ -0,0 +1,65 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.dto;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
|
||||
@XmlType(name = "parameterProviderConfiguration")
|
||||
public class ParameterProviderConfigurationDTO {
|
||||
private String parameterProviderId;
|
||||
private String parameterProviderName;
|
||||
private String parameterGroupName;
|
||||
private Boolean isSynchronized;
|
||||
|
||||
@ApiModelProperty("The ID of the Parameter Provider")
|
||||
public String getParameterProviderId() {
|
||||
return parameterProviderId;
|
||||
}
|
||||
|
||||
public void setParameterProviderId(String parameterProviderId) {
|
||||
this.parameterProviderId = parameterProviderId;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The name of the Parameter Provider")
|
||||
public String getParameterProviderName() {
|
||||
return parameterProviderName;
|
||||
}
|
||||
|
||||
public void setParameterProviderName(String parameterProviderName) {
|
||||
this.parameterProviderName = parameterProviderName;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The Parameter Group name that maps to the Parameter Context")
|
||||
public String getParameterGroupName() {
|
||||
return parameterGroupName;
|
||||
}
|
||||
|
||||
public void setParameterGroupName(String parameterGroupName) {
|
||||
this.parameterGroupName = parameterGroupName;
|
||||
}
|
||||
|
||||
@ApiModelProperty("True if the Parameter Context should receive the parameters from the mapped Parameter Group")
|
||||
public Boolean getSynchronized() {
|
||||
return isSynchronized;
|
||||
}
|
||||
|
||||
public void setSynchronized(Boolean isSynchronized) {
|
||||
this.isSynchronized = isSynchronized;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,301 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.dto;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
import org.apache.nifi.web.api.entity.AffectedComponentEntity;
|
||||
import org.apache.nifi.web.api.entity.ParameterProviderReferencingComponentEntity;
|
||||
import org.apache.nifi.web.api.entity.ParameterGroupConfigurationEntity;
|
||||
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
import java.util.Collection;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Component that is capable of providing parameters to NiFi from an external source
|
||||
*/
|
||||
@XmlType(name = "parameterProvider")
|
||||
public class ParameterProviderDTO extends ComponentDTO {
|
||||
public static final String VALID = "VALID";
|
||||
public static final String INVALID = "INVALID";
|
||||
public static final String VALIDATING = "VALIDATING";
|
||||
|
||||
private String name;
|
||||
private String type;
|
||||
private BundleDTO bundle;
|
||||
private String comments;
|
||||
private Boolean persistsState;
|
||||
private Boolean restricted;
|
||||
private Boolean deprecated;
|
||||
private Boolean isExtensionMissing;
|
||||
private Boolean multipleVersionsAvailable;
|
||||
|
||||
private Map<String, String> properties;
|
||||
private Map<String, PropertyDescriptorDTO> descriptors;
|
||||
private Collection<ParameterGroupConfigurationEntity> parameterGroupConfigurations;
|
||||
private Set<AffectedComponentEntity> affectedComponents;
|
||||
private Set<ParameterProviderReferencingComponentEntity> referencingParameterContexts;
|
||||
|
||||
private String customUiUrl;
|
||||
private String annotationData;
|
||||
|
||||
private Collection<String> validationErrors;
|
||||
private String validationStatus;
|
||||
|
||||
/**
|
||||
* @return user-defined name of the parameter provider
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The name of the parameter provider."
|
||||
)
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
public void setName(final String name) {
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
@ApiModelProperty("The set of all components in the flow that are referencing Parameters provided by this provider")
|
||||
public Set<AffectedComponentEntity> getAffectedComponents() {
|
||||
return affectedComponents;
|
||||
}
|
||||
|
||||
public void setAffectedComponents(final Set<AffectedComponentEntity> affectedComponents) {
|
||||
this.affectedComponents = affectedComponents;
|
||||
}
|
||||
|
||||
public void setReferencingParameterContexts(final Set<ParameterProviderReferencingComponentEntity> referencingParameterContexts) {
|
||||
this.referencingParameterContexts = referencingParameterContexts;
|
||||
}
|
||||
|
||||
@ApiModelProperty(value = "The Parameter Contexts that reference this Parameter Provider", accessMode = ApiModelProperty.AccessMode.READ_ONLY)
|
||||
public Set<ParameterProviderReferencingComponentEntity> getReferencingParameterContexts() {
|
||||
return referencingParameterContexts;
|
||||
}
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "Configuration for any fetched parameter groups."
|
||||
)
|
||||
public Collection<ParameterGroupConfigurationEntity> getParameterGroupConfigurations() {
|
||||
return parameterGroupConfigurations;
|
||||
}
|
||||
|
||||
public void setParameterGroupConfigurations(final Collection<ParameterGroupConfigurationEntity> parameterGroupConfigurations) {
|
||||
this.parameterGroupConfigurations = parameterGroupConfigurations;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return user-defined comments for the parameter provider
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The comments of the parameter provider."
|
||||
)
|
||||
public String getComments() {
|
||||
return comments;
|
||||
}
|
||||
|
||||
public void setComments(final String comments) {
|
||||
this.comments = comments;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return type of parameter provider
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The fully qualified type of the parameter provider."
|
||||
)
|
||||
public String getType() {
|
||||
return type;
|
||||
}
|
||||
|
||||
public void setType(final String type) {
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
/**
|
||||
* The details of the artifact that bundled this parameter provider type.
|
||||
*
|
||||
* @return The bundle details
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The details of the artifact that bundled this parameter provider type."
|
||||
)
|
||||
public BundleDTO getBundle() {
|
||||
return bundle;
|
||||
}
|
||||
|
||||
public void setBundle(final BundleDTO bundle) {
|
||||
this.bundle = bundle;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return whether this parameter provider persists state
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "Whether the parameter provider persists state."
|
||||
)
|
||||
public Boolean getPersistsState() {
|
||||
return persistsState;
|
||||
}
|
||||
|
||||
public void setPersistsState(final Boolean persistsState) {
|
||||
this.persistsState = persistsState;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return whether this parameter provider requires elevated privileges
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "Whether the parameter provider requires elevated privileges."
|
||||
)
|
||||
public Boolean getRestricted() {
|
||||
return restricted;
|
||||
}
|
||||
|
||||
public void setRestricted(final Boolean restricted) {
|
||||
this.restricted = restricted;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Whether the parameter provider has been deprecated.
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "Whether the parameter provider has been deprecated."
|
||||
)
|
||||
public Boolean getDeprecated() {
|
||||
return deprecated;
|
||||
}
|
||||
|
||||
public void setDeprecated(final Boolean deprecated) {
|
||||
this.deprecated = deprecated;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return whether the underlying extension is missing
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "Whether the underlying extension is missing."
|
||||
)
|
||||
public Boolean getExtensionMissing() {
|
||||
return isExtensionMissing;
|
||||
}
|
||||
|
||||
public void setExtensionMissing(final Boolean extensionMissing) {
|
||||
isExtensionMissing = extensionMissing;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return whether this parameter provider has multiple versions available
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "Whether the parameter provider has multiple versions available."
|
||||
)
|
||||
public Boolean getMultipleVersionsAvailable() {
|
||||
return multipleVersionsAvailable;
|
||||
}
|
||||
|
||||
public void setMultipleVersionsAvailable(final Boolean multipleVersionsAvailable) {
|
||||
this.multipleVersionsAvailable = multipleVersionsAvailable;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return parameter provider's properties
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The properties of the parameter provider."
|
||||
)
|
||||
public Map<String, String> getProperties() {
|
||||
return properties;
|
||||
}
|
||||
|
||||
public void setProperties(Map<String, String> properties) {
|
||||
this.properties = properties;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Map of property name to descriptor
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The descriptors for the parameter providers properties."
|
||||
)
|
||||
public Map<String, PropertyDescriptorDTO> getDescriptors() {
|
||||
return descriptors;
|
||||
}
|
||||
|
||||
public void setDescriptors(Map<String, PropertyDescriptorDTO> descriptors) {
|
||||
this.descriptors = descriptors;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the URL for this parameter provider custom configuration UI if applicable. Null otherwise
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The URL for the custom configuration UI for the parameter provider."
|
||||
)
|
||||
public String getCustomUiUrl() {
|
||||
return customUiUrl;
|
||||
}
|
||||
|
||||
public void setCustomUiUrl(String customUiUrl) {
|
||||
this.customUiUrl = customUiUrl;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return currently configured annotation data for the parameter provider
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The annotation data for the parameter provider. This is how the custom UI relays configuration to the parameter provider."
|
||||
)
|
||||
public String getAnnotationData() {
|
||||
return annotationData;
|
||||
}
|
||||
|
||||
public void setAnnotationData(String annotationData) {
|
||||
this.annotationData = annotationData;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the validation errors from this parameter provider. These validation errors represent the problems with the parameter provider that must be resolved before it can be scheduled to run.
|
||||
*
|
||||
* @return The validation errors
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "Gets the validation errors from the parameter provider. These validation errors represent the problems with the parameter provider that must be resolved before "
|
||||
+ "it can be scheduled to run."
|
||||
)
|
||||
public Collection<String> getValidationErrors() {
|
||||
return validationErrors;
|
||||
}
|
||||
|
||||
public void setValidationErrors(Collection<String> validationErrors) {
|
||||
this.validationErrors = validationErrors;
|
||||
}
|
||||
|
||||
@ApiModelProperty(value = "Indicates whether the Parameter Provider is valid, invalid, or still in the process of validating (i.e., it is unknown whether or not the Parameter Provider is valid)",
|
||||
accessMode = ApiModelProperty.AccessMode.READ_ONLY,
|
||||
allowableValues = VALID + ", " + INVALID + ", " + VALIDATING)
|
||||
public String getValidationStatus() {
|
||||
return validationStatus;
|
||||
}
|
||||
|
||||
public void setValidationStatus(String validationStatus) {
|
||||
this.validationStatus = validationStatus;
|
||||
}
|
||||
|
||||
}
|
|
@ -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.web.api.dto;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
|
||||
/**
|
||||
* A component referencing a parameter provider. The only allowed component at this time is a ParameterContext.
|
||||
*/
|
||||
@XmlType(name = "parameterProviderReferencingComponent")
|
||||
public class ParameterProviderReferencingComponentDTO {
|
||||
|
||||
private String id;
|
||||
private String name;
|
||||
|
||||
/**
|
||||
* @return id for this component referencing a parameter provider
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The id of the component referencing a parameter provider."
|
||||
)
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public void setId(String id) {
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return name for this component referencing a parameter provider
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The name of the component referencing a parameter provider."
|
||||
)
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
public void setName(String name) {
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
}
|
|
@ -102,12 +102,12 @@ public class ReportingTaskDTO extends ComponentDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* The details of the artifact that bundled this processor type.
|
||||
* The details of the artifact that bundled this reporting task type.
|
||||
*
|
||||
* @return The bundle details
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The details of the artifact that bundled this processor type."
|
||||
value = "The details of the artifact that bundled this reporting task type."
|
||||
)
|
||||
public BundleDTO getBundle() {
|
||||
return bundle;
|
||||
|
@ -123,7 +123,7 @@ public class ReportingTaskDTO extends ComponentDTO {
|
|||
* @return The scheduling period
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The frequency with which to schedule the reporting task. The format of the value willd epend on the valud of the schedulingStrategy."
|
||||
value = "The frequency with which to schedule the reporting task. The format of the value will depend on the value of the schedulingStrategy."
|
||||
)
|
||||
public String getSchedulingPeriod() {
|
||||
return schedulingPeriod;
|
||||
|
@ -333,7 +333,7 @@ public class ReportingTaskDTO extends ComponentDTO {
|
|||
this.validationErrors = validationErrors;
|
||||
}
|
||||
|
||||
@ApiModelProperty(value = "Indicates whether the Processor is valid, invalid, or still in the process of validating (i.e., it is unknown whether or not the Processor is valid)",
|
||||
@ApiModelProperty(value = "Indicates whether the Reporting Task is valid, invalid, or still in the process of validating (i.e., it is unknown whether or not the Reporting Task is valid)",
|
||||
accessMode = ApiModelProperty.AccessMode.READ_ONLY,
|
||||
allowableValues = VALID + ", " + INVALID + ", " + VALIDATING)
|
||||
public String getValidationStatus() {
|
||||
|
|
|
@ -38,6 +38,7 @@ public class SearchResultsDTO {
|
|||
private List<ComponentSearchResultDTO> labelResults = new ArrayList<>();
|
||||
private List<ComponentSearchResultDTO> controllerServiceNodeResults = new ArrayList<>();
|
||||
private List<ComponentSearchResultDTO> parameterContextResults = new ArrayList<>();
|
||||
private List<ComponentSearchResultDTO> parameterProviderNodeResults = new ArrayList<>();
|
||||
private List<ComponentSearchResultDTO> parameterResults = new ArrayList<>();
|
||||
|
||||
/**
|
||||
|
@ -166,6 +167,20 @@ public class SearchResultsDTO {
|
|||
this.controllerServiceNodeResults = controllerServiceNodeResults;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the parameter provider nodes that matched the search
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The parameter provider nodes that matched the search"
|
||||
)
|
||||
public List<ComponentSearchResultDTO> getParameterProviderNodeResults() {
|
||||
return parameterProviderNodeResults;
|
||||
}
|
||||
|
||||
public void setParameterProviderNodeResults(List<ComponentSearchResultDTO> parameterProviderNodeResults) {
|
||||
this.parameterProviderNodeResults = parameterProviderNodeResults;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return parameter contexts that matched the search.
|
||||
*/
|
||||
|
|
|
@ -31,6 +31,7 @@ public class ControllerBulletinsEntity extends Entity {
|
|||
private List<BulletinEntity> bulletins;
|
||||
private List<BulletinEntity> controllerServiceBulletins;
|
||||
private List<BulletinEntity> reportingTaskBulletins;
|
||||
private List<BulletinEntity> parameterProviderBulletins;
|
||||
|
||||
/**
|
||||
* @return System bulletins to be reported to the user
|
||||
|
@ -68,12 +69,25 @@ public class ControllerBulletinsEntity extends Entity {
|
|||
this.reportingTaskBulletins = reportingTaskBulletins;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Parameter provider bulletins to be reported to the user
|
||||
*/
|
||||
@ApiModelProperty("Parameter provider bulletins to be reported to the user.")
|
||||
public List<BulletinEntity> getParameterProviderBulletins() {
|
||||
return parameterProviderBulletins;
|
||||
}
|
||||
|
||||
public void setParameterProviderBulletins(List<BulletinEntity> parameterProviderBulletins) {
|
||||
this.parameterProviderBulletins = parameterProviderBulletins;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerBulletinsEntity clone() {
|
||||
final ControllerBulletinsEntity other = new ControllerBulletinsEntity();
|
||||
other.setBulletins(getBulletins() == null ? null : new ArrayList<>(getBulletins()));
|
||||
other.setControllerServiceBulletins(getControllerServiceBulletins() == null ? null : new ArrayList<>(getControllerServiceBulletins()));
|
||||
other.setReportingTaskBulletins(getReportingTaskBulletins() == null ? null : new ArrayList<>(getReportingTaskBulletins()));
|
||||
other.setParameterProviderBulletins(getParameterProviderBulletins() == null ? null : new ArrayList<>(getParameterProviderBulletins()));
|
||||
return other;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,59 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.entity;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
import org.apache.nifi.web.api.dto.ParameterContextDTO;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
import java.util.Set;
|
||||
|
||||
@XmlRootElement(name = "parameterContextUpdateEntity")
|
||||
public class ParameterContextUpdateEntity extends Entity {
|
||||
private RevisionDTO parameterContextRevision;
|
||||
private ParameterContextDTO parameterContext;
|
||||
private Set<AffectedComponentEntity> referencingComponents;
|
||||
|
||||
@ApiModelProperty("The Revision of the Parameter Context")
|
||||
public RevisionDTO getParameterContextRevision() {
|
||||
return parameterContextRevision;
|
||||
}
|
||||
|
||||
public void setParameterContextRevision(final RevisionDTO parameterContextRevision) {
|
||||
this.parameterContextRevision = parameterContextRevision;
|
||||
}
|
||||
|
||||
@ApiModelProperty(value = "The Parameter Context that is being operated on. This may not be populated until the request has successfully completed.",
|
||||
accessMode = ApiModelProperty.AccessMode.READ_ONLY)
|
||||
public ParameterContextDTO getParameterContext() {
|
||||
return parameterContext;
|
||||
}
|
||||
|
||||
public void setParameterContext(final ParameterContextDTO parameterContext) {
|
||||
this.parameterContext = parameterContext;
|
||||
}
|
||||
|
||||
@ApiModelProperty(value = "The components that are referenced by the update.", accessMode = ApiModelProperty.AccessMode.READ_ONLY)
|
||||
public Set<AffectedComponentEntity> getReferencingComponents() {
|
||||
return referencingComponents;
|
||||
}
|
||||
|
||||
public void setReferencingComponents(final Set<AffectedComponentEntity> referencingComponents) {
|
||||
this.referencingComponents = referencingComponents;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,100 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.entity;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
import org.apache.nifi.parameter.ParameterSensitivity;
|
||||
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Entity encapsulating the configuration for a single parameter group.
|
||||
*/
|
||||
@XmlType(name = "parameterGroupConfiguration")
|
||||
public class ParameterGroupConfigurationEntity extends Entity implements Comparable<ParameterGroupConfigurationEntity> {
|
||||
|
||||
private String groupName;
|
||||
private String parameterContextName;
|
||||
private Boolean isSynchronized;
|
||||
private Map<String, ParameterSensitivity> parameterSensitivities;
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "The name of the external parameter group to which the provided parameter names apply."
|
||||
)
|
||||
public String getGroupName() {
|
||||
return groupName;
|
||||
}
|
||||
|
||||
public void setGroupName(final String groupName) {
|
||||
this.groupName = groupName;
|
||||
}
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "The name of the ParameterContext that receives the parameters in this group"
|
||||
)
|
||||
public String getParameterContextName() {
|
||||
return parameterContextName;
|
||||
}
|
||||
|
||||
public void setParameterContextName(final String parameterContextName) {
|
||||
this.parameterContextName = parameterContextName;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return All fetched parameter names that should be applied.
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "All fetched parameter names that should be applied."
|
||||
)
|
||||
public Map<String, ParameterSensitivity> getParameterSensitivities() {
|
||||
return parameterSensitivities;
|
||||
}
|
||||
|
||||
public void setParameterSensitivities(Map<String, ParameterSensitivity> parameterSensitivities) {
|
||||
this.parameterSensitivities = parameterSensitivities;
|
||||
}
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "True if this group should be synchronized to a ParameterContext, including creating one if it does not exist."
|
||||
)
|
||||
public Boolean isSynchronized() {
|
||||
return isSynchronized;
|
||||
}
|
||||
|
||||
public void setSynchronized(Boolean aSynchronized) {
|
||||
isSynchronized = aSynchronized;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(final ParameterGroupConfigurationEntity other) {
|
||||
if (other == null) {
|
||||
return -1;
|
||||
}
|
||||
|
||||
final String groupName = getGroupName();
|
||||
final String otherGroupName = other.getGroupName();
|
||||
|
||||
if (groupName == null) {
|
||||
return otherGroupName == null ? 0 : -1;
|
||||
}
|
||||
if (otherGroupName == null) {
|
||||
return 1;
|
||||
}
|
||||
return groupName.compareTo(otherGroupName);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,36 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.entity;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
import org.apache.nifi.web.api.dto.ParameterProviderApplyParametersRequestDTO;
|
||||
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
|
||||
@XmlRootElement(name = "parameterProviderApplyParametersRequestEntity")
|
||||
public class ParameterProviderApplyParametersRequestEntity extends Entity {
|
||||
private ParameterProviderApplyParametersRequestDTO request;
|
||||
|
||||
@ApiModelProperty("The Apply Parameters Request")
|
||||
public ParameterProviderApplyParametersRequestDTO getRequest() {
|
||||
return request;
|
||||
}
|
||||
|
||||
public void setRequest(final ParameterProviderApplyParametersRequestDTO request) {
|
||||
this.request = request;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,80 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.entity;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
import org.apache.nifi.web.api.dto.ParameterProviderConfigurationDTO;
|
||||
import org.apache.nifi.web.api.dto.PermissionsDTO;
|
||||
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
|
||||
/**
|
||||
* A serialized representation of this class can be placed in the entity body of a request or response to or from the API.
|
||||
* This particular entity holds a reference to a ParameterProviderConfigurationDTO.
|
||||
*/
|
||||
@XmlRootElement(name = "parameterProviderConfigurationEntity")
|
||||
public class ParameterProviderConfigurationEntity implements Permissible<ParameterProviderConfigurationDTO> {
|
||||
|
||||
private String id;
|
||||
private PermissionsDTO permissions;
|
||||
private ParameterProviderConfigurationDTO component;
|
||||
|
||||
/**
|
||||
* The id for this component.
|
||||
*
|
||||
* @return The id
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The id of the component."
|
||||
)
|
||||
public String getId() {
|
||||
return this.id;
|
||||
}
|
||||
|
||||
public void setId(final String id) {
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
/**
|
||||
* The permissions for this component.
|
||||
*
|
||||
* @return The permissions
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The permissions for this component."
|
||||
)
|
||||
public PermissionsDTO getPermissions() {
|
||||
return permissions;
|
||||
}
|
||||
|
||||
public void setPermissions(PermissionsDTO permissions) {
|
||||
this.permissions = permissions;
|
||||
}
|
||||
|
||||
/**
|
||||
* The ParameterProviderConfigurationDTO that is being serialized.
|
||||
*
|
||||
* @return The ParameterProviderConfigurationDTO object
|
||||
*/
|
||||
public ParameterProviderConfigurationDTO getComponent() {
|
||||
return component;
|
||||
}
|
||||
|
||||
public void setComponent(ParameterProviderConfigurationDTO component) {
|
||||
this.component = component;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,63 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.entity;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
import org.apache.nifi.web.api.dto.ParameterProviderDTO;
|
||||
import org.apache.nifi.web.api.dto.PermissionsDTO;
|
||||
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
|
||||
|
||||
/**
|
||||
* A serialized representation of this class can be placed in the entity body of a response to the API. This particular entity holds a reference to a parameter provider.
|
||||
*/
|
||||
@XmlRootElement(name = "parameterProviderEntity")
|
||||
public class ParameterProviderEntity extends ComponentEntity implements Permissible<ParameterProviderDTO> {
|
||||
|
||||
private ParameterProviderDTO component;
|
||||
private PermissionsDTO permissions;
|
||||
|
||||
/**
|
||||
* @return parameter provider that is being serialized
|
||||
*/
|
||||
@Override
|
||||
public ParameterProviderDTO getComponent() {
|
||||
return component;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setComponent(ParameterProviderDTO component) {
|
||||
this.component = component;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The permissions for this component operations
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The permissions for this component."
|
||||
)
|
||||
@Override
|
||||
public PermissionsDTO getPermissions() {
|
||||
return permissions;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPermissions(PermissionsDTO permissions) {
|
||||
this.permissions = permissions;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,85 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.entity;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
import java.util.Collection;
|
||||
|
||||
/**
|
||||
* Request for applying fetched parameters from a Parameter Provider.
|
||||
*/
|
||||
@XmlType(name = "parameterProviderParameterApplication")
|
||||
public class ParameterProviderParameterApplicationEntity extends Entity {
|
||||
|
||||
private String id;
|
||||
private RevisionDTO revision;
|
||||
private Boolean disconnectedNodeAcknowledged;
|
||||
private Collection<ParameterGroupConfigurationEntity> parameterGroupConfigurations;
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "The id of the parameter provider."
|
||||
)
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public void setId(final String id) {
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return revision for this request/response
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The revision for this request/response. The revision is required for any mutable flow requests and is included in all responses."
|
||||
)
|
||||
public RevisionDTO getRevision() {
|
||||
return revision;
|
||||
}
|
||||
|
||||
public void setRevision(final RevisionDTO revision) {
|
||||
this.revision = revision;
|
||||
}
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed."
|
||||
)
|
||||
public Boolean isDisconnectedNodeAcknowledged() {
|
||||
return disconnectedNodeAcknowledged;
|
||||
}
|
||||
|
||||
public void setDisconnectedNodeAcknowledged(final Boolean disconnectedNodeAcknowledged) {
|
||||
this.disconnectedNodeAcknowledged = disconnectedNodeAcknowledged;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Specifies per group which parameter names should be applied to the Parameter Contexts.
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "Configuration for the fetched Parameter Groups"
|
||||
)
|
||||
public Collection<ParameterGroupConfigurationEntity> getParameterGroupConfigurations() {
|
||||
return parameterGroupConfigurations;
|
||||
}
|
||||
|
||||
public void setParameterGroupConfigurations(Collection<ParameterGroupConfigurationEntity> parameterGroupConfigurations) {
|
||||
this.parameterGroupConfigurations = parameterGroupConfigurations;
|
||||
}
|
||||
}
|
|
@ -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.web.api.entity;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
|
||||
/**
|
||||
* Request for fetching parameters from a Parameter Provider.
|
||||
*/
|
||||
@XmlType(name = "parameterProviderParameterFetch")
|
||||
public class ParameterProviderParameterFetchEntity extends Entity {
|
||||
|
||||
private String id;
|
||||
private RevisionDTO revision;
|
||||
private Boolean disconnectedNodeAcknowledged;
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "The id of the parameter provider."
|
||||
)
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public void setId(final String id) {
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return revision for this request/response
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The revision for this request/response. The revision is required for any mutable flow requests and is included in all responses."
|
||||
)
|
||||
public RevisionDTO getRevision() {
|
||||
return revision;
|
||||
}
|
||||
|
||||
public void setRevision(final RevisionDTO revision) {
|
||||
this.revision = revision;
|
||||
}
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "Acknowledges that this node is disconnected to allow for mutable requests to proceed."
|
||||
)
|
||||
public Boolean isDisconnectedNodeAcknowledged() {
|
||||
return disconnectedNodeAcknowledged;
|
||||
}
|
||||
|
||||
public void setDisconnectedNodeAcknowledged(final Boolean disconnectedNodeAcknowledged) {
|
||||
this.disconnectedNodeAcknowledged = disconnectedNodeAcknowledged;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,62 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.web.api.entity;
|
||||
|
||||
import io.swagger.annotations.ApiModelProperty;
|
||||
import org.apache.nifi.web.api.dto.ParameterProviderReferencingComponentDTO;
|
||||
import org.apache.nifi.web.api.dto.PermissionsDTO;
|
||||
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
|
||||
/**
|
||||
* A serialized representation of this class can be placed in the entity body of a response to the API.
|
||||
* This particular entity holds a reference to component that references a parameter providers.
|
||||
*/
|
||||
@XmlRootElement(name = "parameterProviderReferencingComponentEntity")
|
||||
public class ParameterProviderReferencingComponentEntity extends ComponentEntity {
|
||||
|
||||
private ParameterProviderReferencingComponentDTO component;
|
||||
private PermissionsDTO permissions;
|
||||
|
||||
/**
|
||||
* @return parameter provider referencing components that is being serialized
|
||||
*/
|
||||
public ParameterProviderReferencingComponentDTO getComponent() {
|
||||
return component;
|
||||
}
|
||||
|
||||
public void setComponent(ParameterProviderReferencingComponentDTO component) {
|
||||
this.component = component;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The permissions for this component
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The permissions for this component."
|
||||
)
|
||||
@Override
|
||||
public PermissionsDTO getPermissions() {
|
||||
return permissions;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPermissions(PermissionsDTO permissions) {
|
||||
this.permissions = permissions;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.entity;
|
||||
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* A serialized representation of this class can be placed in the entity body of a response to the API. This particular entity holds a reference to a list of parameter provider referencing
|
||||
* components.
|
||||
*/
|
||||
@XmlRootElement(name = "parameterProviderReferencingComponentsEntity")
|
||||
public class ParameterProviderReferencingComponentsEntity extends Entity {
|
||||
|
||||
private Set<ParameterProviderReferencingComponentEntity> parameterProviderReferencingComponents;
|
||||
|
||||
/**
|
||||
* @return set of parameter provider referencing components that are being serialized
|
||||
*/
|
||||
public Set<ParameterProviderReferencingComponentEntity> getParameterProviderReferencingComponents() {
|
||||
return parameterProviderReferencingComponents;
|
||||
}
|
||||
|
||||
public void setParameterProviderReferencingComponents(Set<ParameterProviderReferencingComponentEntity> parameterProviderReferencingComponents) {
|
||||
this.parameterProviderReferencingComponents = parameterProviderReferencingComponents;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,43 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.entity;
|
||||
|
||||
import org.apache.nifi.web.api.dto.DocumentedTypeDTO;
|
||||
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* A serialized representation of this class can be placed in the entity body of a response to the API. This particular entity holds a reference to a list of parameter provider types.
|
||||
*/
|
||||
@XmlRootElement(name = "parameterProviderTypesEntity")
|
||||
public class ParameterProviderTypesEntity extends Entity {
|
||||
|
||||
private Set<DocumentedTypeDTO> parameterProviderTypes;
|
||||
|
||||
/**
|
||||
* @return list of parameter provider types that are being serialized
|
||||
*/
|
||||
public Set<DocumentedTypeDTO> getParameterProviderTypes() {
|
||||
return parameterProviderTypes;
|
||||
}
|
||||
|
||||
public void setParameterProviderTypes(Set<DocumentedTypeDTO> parameterProviderTypes) {
|
||||
this.parameterProviderTypes = parameterProviderTypes;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.entity;
|
||||
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* A serialized representation of this class can be placed in the entity body of a response to the API. This particular entity holds a reference to a list of parameter providers.
|
||||
*/
|
||||
@XmlRootElement(name = "parameterProvidersEntity")
|
||||
public class ParameterProvidersEntity extends Entity {
|
||||
|
||||
private Set<ParameterProviderEntity> parameterProviders;
|
||||
|
||||
/**
|
||||
* @return list of parameter providers that are being serialized
|
||||
*/
|
||||
public Set<ParameterProviderEntity> getParameterProviders() {
|
||||
return parameterProviders;
|
||||
}
|
||||
|
||||
public void setParameterProviders(final Set<ParameterProviderEntity> parameterProviders) {
|
||||
this.parameterProviders = parameterProviders;
|
||||
}
|
||||
|
||||
}
|
|
@ -25,6 +25,7 @@ import org.apache.nifi.documentation.html.HtmlProcessorDocumentationWriter;
|
|||
import org.apache.nifi.nar.ExtensionDefinition;
|
||||
import org.apache.nifi.nar.ExtensionManager;
|
||||
import org.apache.nifi.nar.ExtensionMapping;
|
||||
import org.apache.nifi.parameter.ParameterProvider;
|
||||
import org.apache.nifi.processor.Processor;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.apache.nifi.reporting.ReportingTask;
|
||||
|
@ -41,7 +42,7 @@ import java.io.OutputStream;
|
|||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Uses the ExtensionManager to get a list of Processor, ControllerService, and
|
||||
* Uses the ExtensionManager to get a list of Processor, ControllerService, ParameterProvider, and
|
||||
* Reporting Task classes that were loaded and generate documentation for them.
|
||||
*
|
||||
*
|
||||
|
@ -65,6 +66,7 @@ public class DocGenerator {
|
|||
documentConfigurableComponent(extensionManager.getExtensions(Processor.class), explodedNiFiDocsDir, extensionManager);
|
||||
documentConfigurableComponent(extensionManager.getExtensions(ControllerService.class), explodedNiFiDocsDir, extensionManager);
|
||||
documentConfigurableComponent(extensionManager.getExtensions(ReportingTask.class), explodedNiFiDocsDir, extensionManager);
|
||||
documentConfigurableComponent(extensionManager.getExtensions(ParameterProvider.class), explodedNiFiDocsDir, extensionManager);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -145,7 +147,7 @@ public class DocGenerator {
|
|||
|
||||
/**
|
||||
* Returns the DocumentationWriter for the type of component. Currently
|
||||
* Processor, ControllerService, and ReportingTask are supported.
|
||||
* Processor, ControllerService, ParameterProvider, and ReportingTask are supported.
|
||||
*
|
||||
* @param componentClass the class that requires a DocumentationWriter
|
||||
* @return a DocumentationWriter capable of generating documentation for
|
||||
|
@ -159,6 +161,8 @@ public class DocGenerator {
|
|||
return new HtmlDocumentationWriter(extensionManager);
|
||||
} else if (ReportingTask.class.isAssignableFrom(componentClass)) {
|
||||
return new HtmlDocumentationWriter(extensionManager);
|
||||
} else if (ParameterProvider.class.isAssignableFrom(componentClass)) {
|
||||
return new HtmlDocumentationWriter(extensionManager);
|
||||
}
|
||||
|
||||
return null;
|
||||
|
|
|
@ -59,7 +59,7 @@ import java.util.Set;
|
|||
|
||||
/**
|
||||
* Generates HTML documentation for a ConfigurableComponent. This class is used
|
||||
* to generate documentation for ControllerService and ReportingTask because
|
||||
* to generate documentation for ControllerService, ParameterProvider, and ReportingTask because
|
||||
* they have no additional information.
|
||||
*
|
||||
*
|
||||
|
|
|
@ -53,7 +53,7 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
@WritesAttribute(attribute = "first", description = "this is the first attribute i write"),
|
||||
@WritesAttribute(attribute = "second")})
|
||||
@ReadsAttribute(attribute = "incoming", description = "this specifies the format of the thing")
|
||||
@SeeAlso(value = {FullyDocumentedControllerService.class, FullyDocumentedReportingTask.class}, classNames = {"org.apache.nifi.processor.ExampleProcessor"})
|
||||
@SeeAlso(value = {FullyDocumentedControllerService.class, FullyDocumentedReportingTask.class, FullyDocumentedParameterProvider.class}, classNames = {"org.apache.nifi.processor.ExampleProcessor"})
|
||||
@DynamicProperty(name = "Relationship Name", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
|
||||
value = "some XPath", description = "Routes FlowFiles to relationships based on XPath")
|
||||
@DynamicRelationship(name = "name from dynamic property", description = "all files that match the properties XPath")
|
||||
|
|
|
@ -0,0 +1,115 @@
|
|||
/*
|
||||
* 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.documentation.example;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.Restricted;
|
||||
import org.apache.nifi.annotation.behavior.SystemResource;
|
||||
import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnRemoved;
|
||||
import org.apache.nifi.annotation.lifecycle.OnShutdown;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.controller.ConfigurationContext;
|
||||
import org.apache.nifi.parameter.AbstractParameterProvider;
|
||||
import org.apache.nifi.parameter.ParameterProvider;
|
||||
import org.apache.nifi.parameter.ParameterGroup;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
@CapabilityDescription("A helper parameter provider to do...")
|
||||
@Tags({"first", "second", "third"})
|
||||
@Restricted("parameter provider restriction description")
|
||||
@SystemResourceConsideration(resource = SystemResource.CPU)
|
||||
@SystemResourceConsideration(resource = SystemResource.DISK, description = "Customized disk usage description")
|
||||
@SystemResourceConsideration(resource = SystemResource.MEMORY, description = "")
|
||||
public class FullyDocumentedParameterProvider extends AbstractParameterProvider implements ParameterProvider {
|
||||
|
||||
public static final PropertyDescriptor INCLUDE_REGEX = new PropertyDescriptor.Builder()
|
||||
.name("include-regex")
|
||||
.displayName("Include Regex")
|
||||
.description("A Regular Expression indicating what to include as parameters.")
|
||||
.addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
|
||||
.required(true)
|
||||
.defaultValue(".*")
|
||||
.build();
|
||||
public static final PropertyDescriptor EXCLUDE_REGEX = new PropertyDescriptor.Builder()
|
||||
.name("exclude-regex")
|
||||
.displayName("Exclude Regex")
|
||||
.description("A Regular Expression indicating what to exclude as parameters.")
|
||||
.addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
|
||||
.required(false)
|
||||
.build();
|
||||
|
||||
private int onRemovedNoArgs = 0;
|
||||
private int onRemovedArgs = 0;
|
||||
|
||||
private int onShutdownNoArgs = 0;
|
||||
private int onShutdownArgs = 0;
|
||||
|
||||
@Override
|
||||
public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
final List<PropertyDescriptor> descriptors = new ArrayList<>();
|
||||
descriptors.add(INCLUDE_REGEX);
|
||||
descriptors.add(EXCLUDE_REGEX);
|
||||
return descriptors;
|
||||
}
|
||||
|
||||
@OnRemoved
|
||||
public void onRemovedNoArgs() {
|
||||
onRemovedNoArgs++;
|
||||
}
|
||||
|
||||
@OnRemoved
|
||||
public void onRemovedArgs(ConfigurationContext context) {
|
||||
onRemovedArgs++;
|
||||
}
|
||||
|
||||
@OnShutdown
|
||||
public void onShutdownNoArgs() {
|
||||
onShutdownNoArgs++;
|
||||
}
|
||||
|
||||
@OnShutdown
|
||||
public void onShutdownArgs(ConfigurationContext context) {
|
||||
onShutdownArgs++;
|
||||
}
|
||||
|
||||
public int getOnRemovedNoArgs() {
|
||||
return onRemovedNoArgs;
|
||||
}
|
||||
|
||||
public int getOnRemovedArgs() {
|
||||
return onRemovedArgs;
|
||||
}
|
||||
|
||||
public int getOnShutdownNoArgs() {
|
||||
return onShutdownNoArgs;
|
||||
}
|
||||
|
||||
public int getOnShutdownArgs() {
|
||||
return onShutdownArgs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ParameterGroup> fetchParameters(final ConfigurationContext context) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
|
@ -22,10 +22,12 @@ import org.apache.nifi.controller.ControllerService;
|
|||
import org.apache.nifi.documentation.DocumentationWriter;
|
||||
import org.apache.nifi.documentation.example.ControllerServiceWithLogger;
|
||||
import org.apache.nifi.documentation.example.FullyDocumentedControllerService;
|
||||
import org.apache.nifi.documentation.example.FullyDocumentedParameterProvider;
|
||||
import org.apache.nifi.documentation.example.FullyDocumentedReportingTask;
|
||||
import org.apache.nifi.documentation.example.ReportingTaskWithLogger;
|
||||
import org.apache.nifi.init.ControllerServiceInitializer;
|
||||
import org.apache.nifi.init.ReportingTaskingInitializer;
|
||||
import org.apache.nifi.init.ParameterProviderInitializer;
|
||||
import org.apache.nifi.init.ReportingTaskInitializer;
|
||||
import org.apache.nifi.mock.MockControllerServiceInitializationContext;
|
||||
import org.apache.nifi.mock.MockReportingInitializationContext;
|
||||
import org.apache.nifi.nar.ExtensionManager;
|
||||
|
@ -110,11 +112,57 @@ public class HtmlDocumentationWriterTest {
|
|||
assertEquals(1, controllerService.getOnShutdownNoArgs());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDocumentParameterProvider() throws InitializationException, IOException {
|
||||
|
||||
FullyDocumentedParameterProvider parameterProvider = new FullyDocumentedParameterProvider();
|
||||
ParameterProviderInitializer initializer = new ParameterProviderInitializer(extensionManager);
|
||||
initializer.initialize(parameterProvider);
|
||||
|
||||
DocumentationWriter writer = new HtmlDocumentationWriter(extensionManager);
|
||||
|
||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
|
||||
writer.write(parameterProvider, baos, false);
|
||||
initializer.teardown(parameterProvider);
|
||||
|
||||
String results = new String(baos.toByteArray());
|
||||
XmlValidator.assertXmlValid(results);
|
||||
|
||||
// description
|
||||
assertContains(results, "A helper parameter provider to do...");
|
||||
|
||||
// tags
|
||||
assertContains(results, "first, second, third");
|
||||
|
||||
// properties
|
||||
assertContains(results, "Include Regex");
|
||||
assertContains(results, "A Regular Expression indicating what to include as parameters.");
|
||||
|
||||
// restricted
|
||||
assertContains(results, "parameter provider restriction description");
|
||||
|
||||
// verify system resource considerations
|
||||
assertContains(results, SystemResource.CPU.name());
|
||||
assertContains(results, SystemResourceConsideration.DEFAULT_DESCRIPTION);
|
||||
assertContains(results, SystemResource.DISK.name());
|
||||
assertContains(results, "Customized disk usage description");
|
||||
assertContains(results, SystemResource.MEMORY.name());
|
||||
assertContains(results, "Not Specified");
|
||||
|
||||
// verify the right OnRemoved and OnShutdown methods were called
|
||||
assertEquals(0, parameterProvider.getOnRemovedArgs());
|
||||
assertEquals(0, parameterProvider.getOnRemovedNoArgs());
|
||||
|
||||
assertEquals(1, parameterProvider.getOnShutdownArgs());
|
||||
assertEquals(1, parameterProvider.getOnShutdownNoArgs());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDocumentReportingTask() throws InitializationException, IOException {
|
||||
|
||||
FullyDocumentedReportingTask reportingTask = new FullyDocumentedReportingTask();
|
||||
ReportingTaskingInitializer initializer = new ReportingTaskingInitializer(extensionManager);
|
||||
ReportingTaskInitializer initializer = new ReportingTaskInitializer(extensionManager);
|
||||
initializer.initialize(reportingTask);
|
||||
|
||||
DocumentationWriter writer = new HtmlDocumentationWriter(extensionManager);
|
||||
|
|
|
@ -280,6 +280,24 @@ public final class ResourceFactory {
|
|||
}
|
||||
};
|
||||
|
||||
private final static Resource PARAMETER_PROVIDERS_RESOURCE = new Resource() {
|
||||
|
||||
@Override
|
||||
public String getIdentifier() {
|
||||
return ResourceType.ParameterProvider.getValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return "Parameter Providers";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getSafeDescription() {
|
||||
return "parameter providers";
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/**
|
||||
* Gets the Resource for accessing the Controller. This includes Controller level configuration, bulletins, reporting tasks, and the cluster.
|
||||
|
@ -316,6 +334,14 @@ public final class ResourceFactory {
|
|||
return PARAMETER_CONTEXTS_RESOURCE;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the Resource for accessing the Parameter Providers.
|
||||
* @return The resource for access Parameter Providers
|
||||
*/
|
||||
public static Resource getParameterProvidersResource() {
|
||||
return PARAMETER_PROVIDERS_RESOURCE;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the Resource for accessing provenance. Access to this Resource allows the user to access data provenance. However, additional authorization
|
||||
* is required based on the component that generated the event and the attributes of the event.
|
||||
|
@ -555,6 +581,9 @@ public final class ResourceFactory {
|
|||
case ParameterContext:
|
||||
componentType = "Parameter Context";
|
||||
break;
|
||||
case ParameterProvider:
|
||||
componentType = "Parameter Provider";
|
||||
break;
|
||||
default:
|
||||
componentType = "Component";
|
||||
break;
|
||||
|
|
|
@ -42,7 +42,8 @@ public enum ResourceType {
|
|||
Operation("/operation"),
|
||||
Template("/templates"),
|
||||
Tenant("/tenants"),
|
||||
ParameterContext("/parameter-contexts");
|
||||
ParameterContext("/parameter-contexts"),
|
||||
ParameterProvider("/parameter-providers");
|
||||
|
||||
final String value;
|
||||
|
||||
|
|
|
@ -49,6 +49,9 @@ import org.apache.nifi.cluster.coordination.http.endpoints.ParameterContextEndpo
|
|||
import org.apache.nifi.cluster.coordination.http.endpoints.ParameterContextUpdateEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ParameterContextValidationMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ParameterContextsEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ParameterProviderEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ParameterProviderFetchRequestsEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ParameterProvidersEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.PortEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.PortStatusEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.PrioritizerTypesEndpointMerger;
|
||||
|
@ -175,6 +178,9 @@ public class StandardHttpResponseMapper implements HttpResponseMapper {
|
|||
endpointMergers.add(new VerifyConfigEndpointMerger());
|
||||
endpointMergers.add(new RuntimeManifestEndpointMerger());
|
||||
endpointMergers.add(new ReplayLastEventEndpointMerger());
|
||||
endpointMergers.add(new ParameterProviderEndpointMerger());
|
||||
endpointMergers.add(new ParameterProvidersEndpointMerger());
|
||||
endpointMergers.add(new ParameterProviderFetchRequestsEndpointMerger());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -57,6 +57,7 @@ public class ControllerBulletinsEndpointMerger extends AbstractSingleEntityEndpo
|
|||
final Map<NodeIdentifier, List<BulletinEntity>> bulletinDtos = new HashMap<>();
|
||||
final Map<NodeIdentifier, List<BulletinEntity>> controllerServiceBulletinDtos = new HashMap<>();
|
||||
final Map<NodeIdentifier, List<BulletinEntity>> reportingTaskBulletinDtos = new HashMap<>();
|
||||
final Map<NodeIdentifier, List<BulletinEntity>> parameterProviderBulletinDtos = new HashMap<>();
|
||||
for (final Map.Entry<NodeIdentifier, ControllerBulletinsEntity> entry : entityMap.entrySet()) {
|
||||
final NodeIdentifier nodeIdentifier = entry.getKey();
|
||||
final ControllerBulletinsEntity entity = entry.getValue();
|
||||
|
@ -90,6 +91,15 @@ public class ControllerBulletinsEndpointMerger extends AbstractSingleEntityEndpo
|
|||
reportingTaskBulletinDtos.computeIfAbsent(nodeIdentifier, nodeId -> new ArrayList<>()).add(bulletin);
|
||||
});
|
||||
}
|
||||
if (entity.getParameterProviderBulletins() != null) {
|
||||
entity.getParameterProviderBulletins().forEach(bulletin -> {
|
||||
if (bulletin.getNodeAddress() == null) {
|
||||
bulletin.setNodeAddress(nodeAddress);
|
||||
}
|
||||
|
||||
parameterProviderBulletinDtos.computeIfAbsent(nodeIdentifier, nodeId -> new ArrayList<>()).add(bulletin);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
clientEntity.setBulletins(BulletinMerger.mergeBulletins(bulletinDtos, entityMap.size()));
|
||||
|
@ -100,6 +110,7 @@ public class ControllerBulletinsEndpointMerger extends AbstractSingleEntityEndpo
|
|||
Collections.sort(clientEntity.getBulletins(), BULLETIN_COMPARATOR);
|
||||
Collections.sort(clientEntity.getControllerServiceBulletins(), BULLETIN_COMPARATOR);
|
||||
Collections.sort(clientEntity.getReportingTaskBulletins(), BULLETIN_COMPARATOR);
|
||||
Collections.sort(clientEntity.getParameterProviderBulletins(), BULLETIN_COMPARATOR);
|
||||
|
||||
// prune the response to only include the max number of bulletins
|
||||
if (clientEntity.getBulletins().size() > MAX_BULLETINS_FOR_CONTROLLER) {
|
||||
|
@ -111,6 +122,9 @@ public class ControllerBulletinsEndpointMerger extends AbstractSingleEntityEndpo
|
|||
if (clientEntity.getReportingTaskBulletins().size() > MAX_BULLETINS_PER_COMPONENT) {
|
||||
clientEntity.setReportingTaskBulletins(clientEntity.getReportingTaskBulletins().subList(0, MAX_BULLETINS_PER_COMPONENT));
|
||||
}
|
||||
if (clientEntity.getParameterProviderBulletins().size() > MAX_BULLETINS_PER_COMPONENT) {
|
||||
clientEntity.setParameterProviderBulletins(clientEntity.getParameterProviderBulletins().subList(0, MAX_BULLETINS_PER_COMPONENT));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ParameterProviderEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
public class ParameterProviderEndpointMerger extends AbstractSingleEntityEndpoint<ParameterProviderEntity> implements EndpointResponseMerger {
|
||||
private static final Pattern PARAMETER_PROVIDER_URI = Pattern.compile("/nifi-api/parameter-providers/[a-f0-9\\-]{36}");
|
||||
|
||||
@Override
|
||||
public boolean canHandle(final URI uri, final String method) {
|
||||
return ("GET".equalsIgnoreCase(method) || "DELETE".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && PARAMETER_PROVIDER_URI.matcher(uri.getPath()).matches();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Class<ParameterProviderEntity> getEntityClass() {
|
||||
return ParameterProviderEntity.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void mergeResponses(final ParameterProviderEntity clientEntity, final Map<NodeIdentifier, ParameterProviderEntity> entityMap, final Set<NodeResponse> successfulResponses,
|
||||
final Set<NodeResponse> problematicResponses) {
|
||||
for(final ParameterProviderEntity entity : entityMap.values()) {
|
||||
ParameterProviderMerger.merge(clientEntity, entity);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ParameterProviderEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
public class ParameterProviderFetchRequestsEndpointMerger extends AbstractSingleEntityEndpoint<ParameterProviderEntity> implements EndpointResponseMerger {
|
||||
private static final Pattern PARAMETER_PROVIDER_URI = Pattern.compile("/nifi-api/parameter-providers/[a-f0-9\\-]{36}/parameters/fetch-requests");
|
||||
|
||||
@Override
|
||||
public boolean canHandle(final URI uri, final String method) {
|
||||
return "PUT".equalsIgnoreCase(method) && PARAMETER_PROVIDER_URI.matcher(uri.getPath()).matches();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Class<ParameterProviderEntity> getEntityClass() {
|
||||
return ParameterProviderEntity.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void mergeResponses(final ParameterProviderEntity clientEntity, final Map<NodeIdentifier, ParameterProviderEntity> entityMap, final Set<NodeResponse> successfulResponses,
|
||||
final Set<NodeResponse> problematicResponses) {
|
||||
for(final ParameterProviderEntity entity : entityMap.values()) {
|
||||
ParameterProviderMerger.merge(clientEntity, entity);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,59 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.parameter.ParameterSensitivity;
|
||||
import org.apache.nifi.web.api.entity.ParameterGroupConfigurationEntity;
|
||||
import org.apache.nifi.web.api.entity.ParameterProviderEntity;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class ParameterProviderMerger {
|
||||
|
||||
public static void merge(final ParameterProviderEntity target, final ParameterProviderEntity otherEntity) {
|
||||
final Collection<ParameterGroupConfigurationEntity> targetParameterGroupConfigurations = target.getComponent().getParameterGroupConfigurations();
|
||||
if (targetParameterGroupConfigurations != null) {
|
||||
if (otherEntity.getComponent().getParameterGroupConfigurations() != null) {
|
||||
final Iterator<ParameterGroupConfigurationEntity> otherGroupIterator = otherEntity.getComponent().getParameterGroupConfigurations().iterator();
|
||||
for (final ParameterGroupConfigurationEntity parameterGroupConfiguration : targetParameterGroupConfigurations) {
|
||||
if (!otherGroupIterator.hasNext()) {
|
||||
continue;
|
||||
}
|
||||
ParameterGroupConfigurationEntity otherConfiguration = otherGroupIterator.next();
|
||||
if (!StringUtils.equals(parameterGroupConfiguration.getGroupName(), otherConfiguration.getGroupName())) {
|
||||
continue;
|
||||
}
|
||||
final Map<String, ParameterSensitivity> targetParameterSensitivities = parameterGroupConfiguration.getParameterSensitivities();
|
||||
if (targetParameterSensitivities != null) {
|
||||
if (otherConfiguration.getGroupName() != null) {
|
||||
targetParameterSensitivities.keySet().retainAll(otherConfiguration.getParameterSensitivities().keySet());
|
||||
}
|
||||
parameterGroupConfiguration.setParameterSensitivities(new LinkedHashMap<>());
|
||||
targetParameterSensitivities.keySet().stream()
|
||||
.sorted()
|
||||
.forEach(paramName -> parameterGroupConfiguration.getParameterSensitivities()
|
||||
.put(paramName, targetParameterSensitivities.get(paramName)));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.manager.DocumentedTypesMerger;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.DocumentedTypeDTO;
|
||||
import org.apache.nifi.web.api.entity.ParameterProviderTypesEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class ParameterProviderTypesEndpointMerger extends AbstractNodeStatusEndpoint<ParameterProviderTypesEntity, Set<DocumentedTypeDTO>> {
|
||||
public static final String PARAMETER_PROVIDER_TYPES_URI_PATTERN = "/nifi-api/flow/parameter-provider-types";
|
||||
|
||||
@Override
|
||||
public boolean canHandle(URI uri, String method) {
|
||||
return "GET".equalsIgnoreCase(method) && PARAMETER_PROVIDER_TYPES_URI_PATTERN.equals(uri.getPath());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Class<ParameterProviderTypesEntity> getEntityClass() {
|
||||
return ParameterProviderTypesEntity.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Set<DocumentedTypeDTO> getDto(ParameterProviderTypesEntity entity) {
|
||||
return entity.getParameterProviderTypes();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void mergeResponses(Set<DocumentedTypeDTO> clientDto, Map<NodeIdentifier, Set<DocumentedTypeDTO>> dtoMap, NodeIdentifier selectedNodeId) {
|
||||
DocumentedTypesMerger.mergeDocumentedTypes(clientDto, dtoMap);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,65 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ParameterProviderEntity;
|
||||
import org.apache.nifi.web.api.entity.ParameterProvidersEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
public class ParameterProvidersEndpointMerger extends AbstractSingleEntityEndpoint<ParameterProvidersEntity> implements EndpointResponseMerger {
|
||||
private static final Pattern PARAMETER_PROVIDER_URI = Pattern.compile("/nifi-api/flow/parameter-providers");
|
||||
|
||||
@Override
|
||||
public boolean canHandle(final URI uri, final String method) {
|
||||
return "GET".equalsIgnoreCase(method) && PARAMETER_PROVIDER_URI.matcher(uri.getPath()).matches();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Class<ParameterProvidersEntity> getEntityClass() {
|
||||
return ParameterProvidersEntity.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void mergeResponses(final ParameterProvidersEntity clientEntity, final Map<NodeIdentifier, ParameterProvidersEntity> entityMap, final Set<NodeResponse> successfulResponses,
|
||||
final Set<NodeResponse> problematicResponses) {
|
||||
|
||||
final Map<String, ParameterProviderEntity> providerEntities = new HashMap<>();
|
||||
for (final ParameterProvidersEntity providersEntity : entityMap.values()) {
|
||||
for (final ParameterProviderEntity entity : providersEntity.getParameterProviders()) {
|
||||
final ParameterProviderEntity mergedEntity = providerEntities.get(entity.getId());
|
||||
if (mergedEntity == null) {
|
||||
providerEntities.put(entity.getId(), entity);
|
||||
continue;
|
||||
}
|
||||
|
||||
ParameterProviderMerger.merge(mergedEntity, entity);
|
||||
}
|
||||
}
|
||||
|
||||
clientEntity.setParameterProviders(new HashSet<>(providerEntities.values()));
|
||||
}
|
||||
|
||||
}
|
|
@ -33,6 +33,7 @@ public class VerifyConfigEndpointMerger extends AbstractSingleEntityEndpoint<Ver
|
|||
public static final Pattern VERIFY_PROCESSOR_CONFIG_URI_PATTERN = Pattern.compile("/nifi-api/processors/[a-f0-9\\-]{36}/config/verification-requests(/[a-f0-9\\-]{36})?");
|
||||
public static final Pattern VERIFY_CONTROLLER_SERVICE_CONFIG_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/[a-f0-9\\-]{36}/config/verification-requests(/[a-f0-9\\-]{36})?");
|
||||
public static final Pattern VERIFY_REPORTING_TASK_CONFIG_URI_PATTERN = Pattern.compile("/nifi-api/reporting-tasks/[a-f0-9\\-]{36}/config/verification-requests(/[a-f0-9\\-]{36})?");
|
||||
public static final Pattern VERIFY_PARAMETER_PROVIDER_CONFIG_URI_PATTERN = Pattern.compile("/nifi-api/parameter-providers/[a-f0-9\\-]{36}/config/verification-requests(/[a-f0-9\\-]{36})?");
|
||||
|
||||
@Override
|
||||
protected Class<VerifyConfigRequestEntity> getEntityClass() {
|
||||
|
@ -43,7 +44,8 @@ public class VerifyConfigEndpointMerger extends AbstractSingleEntityEndpoint<Ver
|
|||
public boolean canHandle(final URI uri, final String method) {
|
||||
return VERIFY_PROCESSOR_CONFIG_URI_PATTERN.matcher(uri.getPath()).matches()
|
||||
|| VERIFY_CONTROLLER_SERVICE_CONFIG_URI_PATTERN.matcher(uri.getPath()).matches()
|
||||
|| VERIFY_REPORTING_TASK_CONFIG_URI_PATTERN.matcher(uri.getPath()).matches();
|
||||
|| VERIFY_REPORTING_TASK_CONFIG_URI_PATTERN.matcher(uri.getPath()).matches()
|
||||
|| VERIFY_PARAMETER_PROVIDER_CONFIG_URI_PATTERN.matcher(uri.getPath()).matches();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.nifi.connectable.Connectable;
|
|||
import org.apache.nifi.connectable.Connection;
|
||||
import org.apache.nifi.connectable.Funnel;
|
||||
import org.apache.nifi.connectable.Port;
|
||||
import org.apache.nifi.controller.ParameterProviderNode;
|
||||
import org.apache.nifi.controller.ProcessScheduler;
|
||||
import org.apache.nifi.controller.ProcessorNode;
|
||||
import org.apache.nifi.controller.ReportingTaskNode;
|
||||
|
@ -38,6 +39,7 @@ import org.apache.nifi.nar.NarCloseable;
|
|||
import org.apache.nifi.parameter.Parameter;
|
||||
import org.apache.nifi.parameter.ParameterContext;
|
||||
import org.apache.nifi.parameter.ParameterContextManager;
|
||||
import org.apache.nifi.parameter.ParameterProviderConfiguration;
|
||||
import org.apache.nifi.parameter.ParameterReferenceManager;
|
||||
import org.apache.nifi.parameter.ReferenceOnlyParameterContext;
|
||||
import org.apache.nifi.parameter.StandardParameterContext;
|
||||
|
@ -71,6 +73,7 @@ public abstract class AbstractFlowManager implements FlowManager {
|
|||
private final ConcurrentMap<String, Port> allOutputPorts = new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<String, Funnel> allFunnels = new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<String, ReportingTaskNode> allReportingTasks = new ConcurrentHashMap<>();
|
||||
private final ConcurrentMap<String, ParameterProviderNode> allParameterProviders = new ConcurrentHashMap<>();
|
||||
|
||||
private final FlowFileEventRepository flowFileEventRepository;
|
||||
private final ParameterContextManager parameterContextManager;
|
||||
|
@ -217,6 +220,7 @@ public abstract class AbstractFlowManager implements FlowManager {
|
|||
componentCounts.put("Reporting Tasks", getAllReportingTasks().size());
|
||||
componentCounts.put("Process Groups", allProcessGroups.size() - 2); // -2 to account for the root group because we don't want it in our counts and the 'root group alias' key.
|
||||
componentCounts.put("Remote Process Groups", getRootGroup().findAllRemoteProcessGroups().size());
|
||||
componentCounts.put("Parameter Providers", getAllParameterProviders().size());
|
||||
|
||||
int localInputPorts = 0;
|
||||
int publicInputPorts = 0;
|
||||
|
@ -273,6 +277,7 @@ public abstract class AbstractFlowManager implements FlowManager {
|
|||
|
||||
getRootControllerServices().forEach(this::removeRootControllerService);
|
||||
getAllReportingTasks().forEach(this::removeReportingTask);
|
||||
getAllParameterProviders().forEach(this::removeParameterProvider);
|
||||
|
||||
for (final String registryId : flowRegistryClient.getRegistryIdentifiers()) {
|
||||
flowRegistryClient.removeFlowRegistry(registryId);
|
||||
|
@ -294,6 +299,10 @@ public abstract class AbstractFlowManager implements FlowManager {
|
|||
reportingTask.verifyCanDelete();
|
||||
}
|
||||
|
||||
for (final ParameterProviderNode parameterProvider : getAllParameterProviders()) {
|
||||
parameterProvider.verifyCanDelete();
|
||||
}
|
||||
|
||||
final ProcessGroup rootGroup = getRootGroup();
|
||||
rootGroup.verifyCanDelete(true, true);
|
||||
}
|
||||
|
@ -376,6 +385,11 @@ public abstract class AbstractFlowManager implements FlowManager {
|
|||
return allReportingTasks.get(taskId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ParameterProviderNode createParameterProvider(final String type, final String id, final BundleCoordinate bundleCoordinate, final boolean firstTimeAdded) {
|
||||
return createParameterProvider(type, id, bundleCoordinate, Collections.emptySet(), firstTimeAdded, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeReportingTask(final ReportingTaskNode reportingTaskNode) {
|
||||
final ReportingTaskNode existing = allReportingTasks.get(reportingTaskNode.getIdentifier());
|
||||
|
@ -414,6 +428,51 @@ public abstract class AbstractFlowManager implements FlowManager {
|
|||
allReportingTasks.put(taskNode.getIdentifier(), taskNode);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ParameterProviderNode getParameterProvider(final String id) {
|
||||
return id == null ? null : allParameterProviders.get(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeParameterProvider(final ParameterProviderNode parameterProvider) {
|
||||
final ParameterProviderNode existing = allParameterProviders.get(parameterProvider.getIdentifier());
|
||||
if (existing == null || existing != parameterProvider) {
|
||||
throw new IllegalStateException("Parameter Provider " + parameterProvider + " does not exist in this Flow");
|
||||
}
|
||||
|
||||
final Class<?> taskClass = parameterProvider.getParameterProvider().getClass();
|
||||
try (final NarCloseable x = NarCloseable.withComponentNarLoader(getExtensionManager(), taskClass, parameterProvider.getParameterProvider().getIdentifier())) {
|
||||
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, parameterProvider.getParameterProvider(), parameterProvider.getConfigurationContext());
|
||||
}
|
||||
|
||||
for (final Map.Entry<PropertyDescriptor, String> entry : parameterProvider.getEffectivePropertyValues().entrySet()) {
|
||||
final PropertyDescriptor descriptor = entry.getKey();
|
||||
if (descriptor.getControllerServiceDefinition() != null) {
|
||||
final String value = entry.getValue() == null ? descriptor.getDefaultValue() : entry.getValue();
|
||||
if (value != null) {
|
||||
final ControllerServiceNode serviceNode = controllerServiceProvider.getControllerServiceNode(value);
|
||||
if (serviceNode != null) {
|
||||
serviceNode.removeReference(parameterProvider, descriptor);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
allParameterProviders.remove(parameterProvider.getIdentifier());
|
||||
LogRepositoryFactory.removeRepository(parameterProvider.getIdentifier());
|
||||
|
||||
getExtensionManager().removeInstanceClassLoader(parameterProvider.getIdentifier());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<ParameterProviderNode> getAllParameterProviders() {
|
||||
return new HashSet<>(allParameterProviders.values());
|
||||
}
|
||||
|
||||
public void onParameterProviderAdded(final ParameterProviderNode parameterProviderNode) {
|
||||
allParameterProviders.put(parameterProviderNode.getIdentifier(), parameterProviderNode);
|
||||
}
|
||||
|
||||
protected abstract ExtensionManager getExtensionManager();
|
||||
|
||||
protected abstract ProcessScheduler getProcessScheduler();
|
||||
|
@ -430,7 +489,8 @@ public abstract class AbstractFlowManager implements FlowManager {
|
|||
|
||||
@Override
|
||||
public ParameterContext createParameterContext(final String id, final String name, final Map<String, Parameter> parameters,
|
||||
final List<String> inheritedContextIds) {
|
||||
final List<String> inheritedContextIds,
|
||||
final ParameterProviderConfiguration parameterProviderConfiguration) {
|
||||
final boolean namingConflict = parameterContextManager.getParameterContexts().stream()
|
||||
.anyMatch(paramContext -> paramContext.getName().equals(name));
|
||||
|
||||
|
@ -439,7 +499,14 @@ public abstract class AbstractFlowManager implements FlowManager {
|
|||
}
|
||||
|
||||
final ParameterReferenceManager referenceManager = new StandardParameterReferenceManager(this);
|
||||
final ParameterContext parameterContext = new StandardParameterContext(id, name, referenceManager, getParameterContextParent());
|
||||
final ParameterContext parameterContext = new StandardParameterContext.Builder()
|
||||
.id(id)
|
||||
.name(name)
|
||||
.parameterReferenceManager(referenceManager)
|
||||
.parentAuthorizable(getParameterContextParent())
|
||||
.parameterProviderLookup(this)
|
||||
.parameterProviderConfiguration(parameterProviderConfiguration)
|
||||
.build();
|
||||
parameterContext.setParameters(parameters);
|
||||
|
||||
if (inheritedContextIds != null && !inheritedContextIds.isEmpty()) {
|
||||
|
|
|
@ -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.controller.parameter;
|
||||
|
||||
import org.apache.nifi.bundle.BundleCoordinate;
|
||||
import org.apache.nifi.controller.LoggableComponent;
|
||||
import org.apache.nifi.controller.TerminationAwareLogger;
|
||||
import org.apache.nifi.parameter.ParameterProvider;
|
||||
|
||||
/**
|
||||
* Holder for StandardParameterProviderNode to atomically swap out the component.
|
||||
*/
|
||||
class ParameterProviderDetails {
|
||||
|
||||
private final ParameterProvider parameterProvider;
|
||||
private final TerminationAwareLogger componentLog;
|
||||
private final BundleCoordinate bundleCoordinate;
|
||||
|
||||
public ParameterProviderDetails(final LoggableComponent<ParameterProvider> parameterProvider) {
|
||||
this.parameterProvider = parameterProvider.getComponent();
|
||||
this.componentLog = parameterProvider.getLogger();
|
||||
this.bundleCoordinate = parameterProvider.getBundleCoordinate();
|
||||
}
|
||||
|
||||
public ParameterProvider getParameterProvider() {
|
||||
return parameterProvider;
|
||||
}
|
||||
|
||||
public TerminationAwareLogger getComponentLog() {
|
||||
return componentLog;
|
||||
}
|
||||
|
||||
public BundleCoordinate getBundleCoordinate() {
|
||||
return bundleCoordinate;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,608 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.parameter;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.Restricted;
|
||||
import org.apache.nifi.annotation.documentation.DeprecationNotice;
|
||||
import org.apache.nifi.authorization.Resource;
|
||||
import org.apache.nifi.authorization.resource.Authorizable;
|
||||
import org.apache.nifi.authorization.resource.ResourceFactory;
|
||||
import org.apache.nifi.authorization.resource.ResourceType;
|
||||
import org.apache.nifi.bundle.Bundle;
|
||||
import org.apache.nifi.bundle.BundleCoordinate;
|
||||
import org.apache.nifi.components.ClassloaderIsolationKeyProvider;
|
||||
import org.apache.nifi.components.ConfigVerificationResult;
|
||||
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
|
||||
import org.apache.nifi.components.ConfigurableComponent;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.components.validation.ValidationStatus;
|
||||
import org.apache.nifi.components.validation.ValidationTrigger;
|
||||
import org.apache.nifi.controller.AbstractComponentNode;
|
||||
import org.apache.nifi.controller.ConfigurationContext;
|
||||
import org.apache.nifi.controller.ControllerServiceLookup;
|
||||
import org.apache.nifi.controller.LoggableComponent;
|
||||
import org.apache.nifi.controller.ParameterProviderNode;
|
||||
import org.apache.nifi.controller.ParametersApplication;
|
||||
import org.apache.nifi.controller.ReloadComponent;
|
||||
import org.apache.nifi.controller.TerminationAwareLogger;
|
||||
import org.apache.nifi.controller.ValidationContextFactory;
|
||||
import org.apache.nifi.controller.service.ControllerServiceProvider;
|
||||
import org.apache.nifi.controller.service.StandardConfigurationContext;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.nar.ExtensionManager;
|
||||
import org.apache.nifi.nar.InstanceClassLoader;
|
||||
import org.apache.nifi.nar.NarCloseable;
|
||||
import org.apache.nifi.parameter.Parameter;
|
||||
import org.apache.nifi.parameter.ParameterContext;
|
||||
import org.apache.nifi.parameter.ParameterDescriptor;
|
||||
import org.apache.nifi.parameter.ParameterGroup;
|
||||
import org.apache.nifi.parameter.ParameterGroupConfiguration;
|
||||
import org.apache.nifi.parameter.ParameterLookup;
|
||||
import org.apache.nifi.parameter.ParameterProvider;
|
||||
import org.apache.nifi.parameter.ParameterSensitivity;
|
||||
import org.apache.nifi.parameter.VerifiableParameterProvider;
|
||||
import org.apache.nifi.registry.ComponentVariableRegistry;
|
||||
import org.apache.nifi.util.CharacterFilterUtils;
|
||||
import org.apache.nifi.util.FormatUtils;
|
||||
import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URL;
|
||||
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.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReadWriteLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
import java.util.function.Function;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class StandardParameterProviderNode extends AbstractComponentNode implements ParameterProviderNode {
|
||||
|
||||
private static final Pattern PARAMETER_NAME_PATTERN = Pattern.compile("^[a-zA-Z0-9-_. ]+$");
|
||||
|
||||
private final AtomicReference<ParameterProviderDetails> parameterProviderRef;
|
||||
private final ControllerServiceLookup serviceLookup;
|
||||
|
||||
private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
|
||||
private final Lock readLock = rwLock.readLock();
|
||||
private final Lock writeLock = rwLock.writeLock();
|
||||
|
||||
private final Set<ParameterContext> referencingParameterContexts;
|
||||
|
||||
private final List<ParameterGroup> fetchedParameterGroups = new ArrayList<>();
|
||||
|
||||
private volatile String comment;
|
||||
|
||||
private final Authorizable parentAuthorizable;
|
||||
|
||||
public StandardParameterProviderNode(final LoggableComponent<ParameterProvider> parameterProvider, final String id, final Authorizable parentAuthorizable,
|
||||
final ControllerServiceProvider controllerServiceProvider, final ValidationContextFactory validationContextFactory,
|
||||
final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent, final ExtensionManager extensionManager,
|
||||
final ValidationTrigger validationTrigger) {
|
||||
|
||||
this(parameterProvider, id, parentAuthorizable, controllerServiceProvider, validationContextFactory,
|
||||
parameterProvider.getComponent().getClass().getSimpleName(), parameterProvider.getComponent().getClass().getCanonicalName(),
|
||||
variableRegistry, reloadComponent, extensionManager, validationTrigger, false);
|
||||
}
|
||||
|
||||
public StandardParameterProviderNode(final LoggableComponent<ParameterProvider> parameterProvider, final String id, final Authorizable parentAuthorizable,
|
||||
final ControllerServiceProvider controllerServiceProvider, final ValidationContextFactory validationContextFactory,
|
||||
final String componentType, final String canonicalClassName, final ComponentVariableRegistry variableRegistry,
|
||||
final ReloadComponent reloadComponent, final ExtensionManager extensionManager, final ValidationTrigger validationTrigger, final boolean isExtensionMissing) {
|
||||
super(id, validationContextFactory, controllerServiceProvider, componentType, canonicalClassName, variableRegistry, reloadComponent,
|
||||
extensionManager, validationTrigger, isExtensionMissing);
|
||||
this.parameterProviderRef = new AtomicReference<>(new ParameterProviderDetails(parameterProvider));
|
||||
this.serviceLookup = controllerServiceProvider;
|
||||
this.referencingParameterContexts = new HashSet<>();
|
||||
this.parentAuthorizable = parentAuthorizable;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Authorizable getParentAuthorizable() {
|
||||
return parentAuthorizable;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Resource getResource() {
|
||||
return ResourceFactory.getComponentResource(ResourceType.ParameterProvider, getIdentifier(), getName());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isRestricted() {
|
||||
return getParameterProvider().getClass().isAnnotationPresent(Restricted.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<?> getComponentClass() {
|
||||
return getParameterProvider().getClass();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isDeprecated() {
|
||||
return getParameterProvider().getClass().isAnnotationPresent(DeprecationNotice.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ParameterContext getParameterContext() {
|
||||
return null;
|
||||
}
|
||||
@Override
|
||||
public ConfigurableComponent getComponent() {
|
||||
return parameterProviderRef.get().getParameterProvider();
|
||||
}
|
||||
|
||||
@Override
|
||||
public BundleCoordinate getBundleCoordinate() {
|
||||
return parameterProviderRef.get().getBundleCoordinate();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TerminationAwareLogger getLogger() {
|
||||
return parameterProviderRef.get().getComponentLog();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ParameterProvider getParameterProvider() {
|
||||
return parameterProviderRef.get().getParameterProvider();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setParameterProvider(final LoggableComponent<ParameterProvider> parameterProvider) {
|
||||
this.parameterProviderRef.set(new ParameterProviderDetails(parameterProvider));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reload(final Set<URL> additionalUrls) throws ParameterProviderInstantiationException {
|
||||
final String additionalResourcesFingerprint = ClassLoaderUtils.generateAdditionalUrlsFingerprint(additionalUrls, determineClasloaderIsolationKey());
|
||||
setAdditionalResourcesFingerprint(additionalResourcesFingerprint);
|
||||
getReloadComponent().reload(this, getCanonicalClassName(), getBundleCoordinate(), additionalUrls);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isValidationNecessary() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ConfigurationContext getConfigurationContext() {
|
||||
return new StandardConfigurationContext(this, serviceLookup, null, getVariableRegistry());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void verifyModifiable() throws IllegalStateException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getComments() {
|
||||
return comment;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setComments(final String comment) {
|
||||
this.comment = CharacterFilterUtils.filterInvalidXmlCharacters(comment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void verifyCanClearState() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "ParameterProvider[id=" + getIdentifier() + "]";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getProcessGroupIdentifier() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ParameterLookup getParameterLookup() {
|
||||
return ParameterLookup.EMPTY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<ParameterContext> getReferences() {
|
||||
readLock.lock();
|
||||
try {
|
||||
return Collections.unmodifiableSet(referencingParameterContexts);
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addReference(final ParameterContext reference) {
|
||||
writeLock.lock();
|
||||
try {
|
||||
referencingParameterContexts.add(reference);
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeReference(final ParameterContext reference) {
|
||||
writeLock.lock();
|
||||
try {
|
||||
referencingParameterContexts.remove(reference);
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<ValidationResult> validateConfig() {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void verifyCanFetchParameters() {
|
||||
final ValidationStatus validationStatus = performValidation();
|
||||
if (validationStatus != ValidationStatus.VALID) {
|
||||
throw new IllegalStateException(String.format("Cannot fetch parameters for %s while validation state is %s", this, validationStatus));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void fetchParameters() {
|
||||
final ParameterProvider parameterProvider = parameterProviderRef.get().getParameterProvider();
|
||||
final ConfigurationContext configurationContext = getConfigurationContext();
|
||||
List<ParameterGroup> fetchedParameterGroups;
|
||||
try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getExtensionManager(), parameterProvider.getClass(), parameterProvider.getIdentifier())) {
|
||||
fetchedParameterGroups = parameterProvider.fetchParameters(configurationContext);
|
||||
} catch (final IOException e) {
|
||||
throw new RuntimeException(String.format("Error fetching parameters for %s", this), e);
|
||||
}
|
||||
|
||||
if (fetchedParameterGroups == null || fetchedParameterGroups.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
|
||||
final Set<String> parameterGroupNames = new HashSet<>();
|
||||
|
||||
writeLock.lock();
|
||||
try {
|
||||
this.fetchedParameterGroups.clear();
|
||||
for (final ParameterGroup group : fetchedParameterGroups) {
|
||||
final String groupName = group.getGroupName();
|
||||
if (parameterGroupNames.contains(groupName)) {
|
||||
throw new IllegalStateException(String.format("Cannot fetch parameters for %s: Parameter group [%s] is provided twice, which is not allowed", this, groupName));
|
||||
}
|
||||
final Collection<Parameter> parameters = group.getParameters();
|
||||
|
||||
if (parameters == null) {
|
||||
continue;
|
||||
}
|
||||
final List<Parameter> validParameters = new ArrayList<>();
|
||||
final Set<String> parameterNames = new HashSet<>();
|
||||
for (final Parameter parameter : parameters) {
|
||||
final ParameterDescriptor descriptor = parameter.getDescriptor();
|
||||
if (descriptor == null) {
|
||||
throw new IllegalStateException("Cannot fetch parameters for " + this + ": a Parameter is missing a ParameterDescriptor in the fetch response");
|
||||
}
|
||||
final String parameterName = descriptor.getName();
|
||||
if (parameterNames.contains(parameterName)) {
|
||||
throw new IllegalStateException(String.format("Cannot fetch parameters for %s: Parameter [%s] is provided in group [%s] twice, which is not allowed",
|
||||
this, parameterName, groupName));
|
||||
}
|
||||
|
||||
if (parameter.getValue() == null) {
|
||||
getLogger().warn("Skipping parameter [{}], which is missing a value", new Object[] {parameterName});
|
||||
continue;
|
||||
}
|
||||
|
||||
if (PARAMETER_NAME_PATTERN.matcher(parameter.getDescriptor().getName()).matches()) {
|
||||
validParameters.add(parameter);
|
||||
parameterNames.add(parameter.getDescriptor().getName());
|
||||
} else {
|
||||
getLogger().warn("Skipping parameter [{}}], whose name has invalid characters. Only alpha-numeric characters (a-z, A-Z, 0-9), hyphens (-), underscores (_), " +
|
||||
"periods (.), and spaces ( ) are accepted.", new Object[] {parameterName});
|
||||
}
|
||||
}
|
||||
this.fetchedParameterGroups.add(new ParameterGroup(groupName, toProvidedParameters(validParameters)));
|
||||
parameterGroupNames.add(groupName);
|
||||
}
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void verifyCanApplyParameters(final Collection<ParameterGroupConfiguration> parameterGroupConfigurations) {
|
||||
if (fetchedParameterGroups.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
readLock.lock();
|
||||
try {
|
||||
final Map<String, ParameterGroupConfiguration> parameterGroupConfigurationMap = parameterGroupConfigurations.stream()
|
||||
.collect(Collectors.toMap(ParameterGroupConfiguration::getParameterContextName, Function.identity()));
|
||||
for (final ParameterContext reference : getReferences()) {
|
||||
final ParameterGroupConfiguration groupConfiguration = parameterGroupConfigurationMap.get(reference.getName());
|
||||
if (groupConfiguration == null) {
|
||||
continue;
|
||||
}
|
||||
reference.verifyCanSetParameters(getFetchedParameterUpdateMap(reference, groupConfiguration));
|
||||
}
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void verifyCanDelete() {
|
||||
if (!getReferences().isEmpty()) {
|
||||
final String contextNameList = getReferences().stream()
|
||||
.map(ParameterContext::getName)
|
||||
.collect(Collectors.joining(", "));
|
||||
throw new IllegalStateException(String.format("Cannot delete %s while it is referenced by Contexts: [%s]", this, contextNameList));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ConfigVerificationResult> verifyConfiguration(final ConfigurationContext context, final ComponentLog logger, final ExtensionManager extensionManager) {
|
||||
final List<ConfigVerificationResult> results = new ArrayList<>();
|
||||
|
||||
try {
|
||||
final long startNanos = System.nanoTime();
|
||||
// Call super's verifyConfig, which will perform component validation
|
||||
results.addAll(super.verifyConfig(context.getProperties(), context.getAnnotationData(), null));
|
||||
final long validationComplete = System.nanoTime();
|
||||
|
||||
// If any invalid outcomes from validation, we do not want to perform additional verification, because we only run additional verification when the component is valid.
|
||||
// This is done in order to make it much simpler to develop these verifications, since the developer doesn't have to worry about whether or not the given values are valid.
|
||||
if (!results.isEmpty() && results.stream().anyMatch(result -> result.getOutcome() == Outcome.FAILED)) {
|
||||
return results;
|
||||
}
|
||||
|
||||
final ParameterProvider parameterProvider = getParameterProvider();
|
||||
if (parameterProvider instanceof VerifiableParameterProvider) {
|
||||
logger.debug("{} is a VerifiableParameterProvider. Will perform full verification of configuration.", this);
|
||||
final VerifiableParameterProvider verifiable = (VerifiableParameterProvider) parameterProvider;
|
||||
|
||||
// Check if the given configuration requires a different classloader than the current configuration
|
||||
final boolean classpathDifferent = isClasspathDifferent(context.getProperties());
|
||||
|
||||
if (classpathDifferent) {
|
||||
// Create a classloader for the given configuration and use that to verify the component's configuration
|
||||
final Bundle bundle = extensionManager.getBundle(getBundleCoordinate());
|
||||
final Set<URL> classpathUrls = getAdditionalClasspathResources(context.getProperties().keySet(), descriptor -> context.getProperty(descriptor).getValue());
|
||||
|
||||
final String classloaderIsolationKey = getClassLoaderIsolationKey(context);
|
||||
|
||||
final ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader();
|
||||
try (final InstanceClassLoader detectedClassLoader = extensionManager.createInstanceClassLoader(getComponentType(), getIdentifier(), bundle, classpathUrls, false,
|
||||
classloaderIsolationKey)) {
|
||||
Thread.currentThread().setContextClassLoader(detectedClassLoader);
|
||||
results.addAll(verifiable.verify(context, logger));
|
||||
} finally {
|
||||
Thread.currentThread().setContextClassLoader(currentClassLoader);
|
||||
}
|
||||
} else {
|
||||
// Verify the configuration, using the component's classloader
|
||||
try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, parameterProvider.getClass(), getIdentifier())) {
|
||||
results.addAll(verifiable.verify(context, logger));
|
||||
}
|
||||
}
|
||||
|
||||
final long validationNanos = validationComplete - startNanos;
|
||||
final long verificationNanos = System.nanoTime() - validationComplete;
|
||||
logger.debug("{} completed full configuration validation in {} plus {} for validation",
|
||||
this, FormatUtils.formatNanos(verificationNanos, false), FormatUtils.formatNanos(validationNanos, false));
|
||||
} else {
|
||||
logger.debug("{} is not a VerifiableParameterProvider, so will not perform full verification of configuration. Validation took {}", this,
|
||||
FormatUtils.formatNanos(validationComplete - startNanos, false));
|
||||
}
|
||||
} catch (final Throwable t) {
|
||||
logger.error("Failed to perform verification of Parameter Provider's configuration for {}", this, t);
|
||||
|
||||
results.add(new ConfigVerificationResult.Builder()
|
||||
.outcome(Outcome.FAILED)
|
||||
.verificationStepName("Perform Verification")
|
||||
.explanation("Encountered unexpected failure when attempting to perform verification: " + t)
|
||||
.build());
|
||||
}
|
||||
|
||||
return results;
|
||||
}
|
||||
|
||||
/**
|
||||
* Using the existing parameters from the ParameterContext and the fetched parameters in the provider, constructs
|
||||
* a map from Parameter name to updated Parameter (or null if the parameter has been removed in the fetch).
|
||||
* @param parameterContext A ParameterContext
|
||||
* @param parameterGroupConfiguration The configuration for the fetched parameter group
|
||||
* @return A map from name to Parameter (or null if parameter should be removed)
|
||||
*/
|
||||
private Map<String, Parameter> getFetchedParameterUpdateMap(final ParameterContext parameterContext, ParameterGroupConfiguration parameterGroupConfiguration) {
|
||||
final Map<String, Parameter> parameterUpdateMap = new HashMap<>();
|
||||
|
||||
final ParameterGroup parameterGroup = fetchedParameterGroups.stream()
|
||||
.filter(group -> parameterContext.getParameterProviderConfiguration().getParameterGroupName().equals(group.getGroupName()))
|
||||
.findFirst()
|
||||
.orElse(null);
|
||||
|
||||
if (parameterGroup == null) {
|
||||
return parameterUpdateMap;
|
||||
}
|
||||
|
||||
// Get a filtered list of the parameters with their sensitivity set based on group configuration
|
||||
final List<Parameter> configuredParameters = configureParameters(parameterGroup.getParameters(), parameterGroupConfiguration);
|
||||
|
||||
final Map<ParameterDescriptor, Parameter> fetchedParameterMap = configuredParameters.stream()
|
||||
.collect(Collectors.toMap(Parameter::getDescriptor, Function.identity()));
|
||||
|
||||
final Map<ParameterDescriptor, Parameter> currentParameters = parameterContext.getParameters();
|
||||
// Find parameters that were removed
|
||||
currentParameters.keySet().forEach(descriptor -> {
|
||||
if (!fetchedParameterMap.containsKey(descriptor)) {
|
||||
parameterUpdateMap.put(descriptor.getName(), null);
|
||||
}
|
||||
});
|
||||
// Add all changed and new parameters
|
||||
for (final Map.Entry<ParameterDescriptor, Parameter> entry : fetchedParameterMap.entrySet()) {
|
||||
final ParameterDescriptor descriptor = entry.getKey();
|
||||
final Parameter fetchedParameter = entry.getValue();
|
||||
final Parameter currentParameter = currentParameters.get(descriptor);
|
||||
if (currentParameter == null) {
|
||||
// Add if it's a new parameter
|
||||
parameterUpdateMap.put(descriptor.getName(), fetchedParameter);
|
||||
} else {
|
||||
final boolean isSensitivityChanged = currentParameter.getDescriptor().isSensitive() != fetchedParameter.getDescriptor().isSensitive();
|
||||
if (!Objects.equals(currentParameter.getValue(), fetchedParameter.getValue()) || isSensitivityChanged) {
|
||||
// Also add if it's an existing parameter that has a changed value
|
||||
parameterUpdateMap.put(descriptor.getName(), fetchedParameter);
|
||||
|
||||
if (isSensitivityChanged) {
|
||||
final ParameterSensitivity currentSensitivity = currentParameter.getDescriptor().isSensitive() ? ParameterSensitivity.SENSITIVE : ParameterSensitivity.NON_SENSITIVE;
|
||||
final ParameterSensitivity fetchedSensitivity = fetchedParameter.getDescriptor().isSensitive() ? ParameterSensitivity.SENSITIVE : ParameterSensitivity.NON_SENSITIVE;
|
||||
getLogger().info("Parameter [{}] sensitivity is being changed from {} to {}", new Object[] { descriptor.getName(),
|
||||
currentSensitivity.getName(), fetchedSensitivity.getName()});
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return parameterUpdateMap;
|
||||
}
|
||||
|
||||
/**
|
||||
* Filters the list of parameters based on the parameter names and sets the sensitivity accordingly
|
||||
* @param parameters A list of Parameters
|
||||
* @param groupConfiguration The user's configuration of the fetched parameters
|
||||
* @return A list of parameters with the configured sensitivities (only those found in the configuration will be included)
|
||||
*/
|
||||
private List<Parameter> configureParameters(final Collection<Parameter> parameters, final ParameterGroupConfiguration groupConfiguration) {
|
||||
// The requested parameter names will be used as a filter for the fetched parameters
|
||||
final Set<String> parameterNameFilter = new HashSet<>(groupConfiguration.getParameterSensitivities().keySet());
|
||||
|
||||
return parameters == null ? Collections.emptyList() : parameters.stream()
|
||||
.filter(parameter -> parameterNameFilter.contains(parameter.getDescriptor().getName()))
|
||||
.map(parameter -> {
|
||||
final String parameterName = parameter.getDescriptor().getName();
|
||||
final ParameterSensitivity sensitivity = groupConfiguration.getParameterSensitivities().get(parameterName);
|
||||
if (sensitivity == null) {
|
||||
throw new IllegalArgumentException(String.format("Parameter sensitivity must be specified for parameter [%s] in group [%s]",
|
||||
parameterName, groupConfiguration.getGroupName()));
|
||||
}
|
||||
final ParameterDescriptor parameterDescriptor = new ParameterDescriptor.Builder()
|
||||
.from(parameter.getDescriptor())
|
||||
.name(parameterName)
|
||||
.sensitive(sensitivity == ParameterSensitivity.SENSITIVE)
|
||||
.build();
|
||||
return new Parameter(parameterDescriptor, parameter.getValue(), parameter.getParameterContextId(), true);
|
||||
})
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets provided = true on all parameters in the list
|
||||
* @param parameters A list of Parameters
|
||||
* @return An equivalent list, but with provided = true
|
||||
*/
|
||||
private static List<Parameter> toProvidedParameters(final Collection<Parameter> parameters) {
|
||||
return parameters == null ? Collections.emptyList() : parameters.stream()
|
||||
.map(parameter -> new Parameter(parameter.getDescriptor(), parameter.getValue(), null, true))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<ParameterGroupConfiguration> getParameterGroupConfigurations() {
|
||||
final Map<String, ParameterContext> parameterContextMap = getReferences().stream()
|
||||
.collect(Collectors.toMap(context -> context.getParameterProviderConfiguration().getParameterGroupName(), Function.identity()));
|
||||
final Collection<ParameterGroupConfiguration> parameterGroupConfigurations = new ArrayList<>();
|
||||
fetchedParameterGroups.forEach(parameterGroup -> {
|
||||
final ParameterContext parameterContext = parameterContextMap.get(parameterGroup.getGroupName());
|
||||
final Set<String> fetchedParameterNames = parameterGroup.getParameters().stream()
|
||||
.map(parameter -> parameter.getDescriptor().getName())
|
||||
.collect(Collectors.toSet());
|
||||
final Map<String, ParameterSensitivity> parameterSensitivities = new HashMap<>();
|
||||
final ParameterGroupConfiguration groupConfiguration;
|
||||
final String parameterContextName;
|
||||
final Boolean isSynchronized;
|
||||
if (parameterContext != null) {
|
||||
isSynchronized = parameterContext.getParameterProviderConfiguration().isSynchronized();
|
||||
parameterContextName = parameterContext.getName();
|
||||
parameterContext.getParameters().forEach((descriptor, parameter) -> {
|
||||
// Don't add it at all if it was not fetched
|
||||
if (fetchedParameterNames.contains(descriptor.getName())) {
|
||||
final ParameterSensitivity sensitivity = descriptor.isSensitive() ? ParameterSensitivity.SENSITIVE : ParameterSensitivity.NON_SENSITIVE;
|
||||
parameterSensitivities.put(descriptor.getName(), sensitivity);
|
||||
}
|
||||
});
|
||||
} else {
|
||||
parameterContextName = parameterGroup.getGroupName();
|
||||
isSynchronized = null;
|
||||
}
|
||||
parameterGroup.getParameters().forEach(parameter -> {
|
||||
final String parameterName = parameter.getDescriptor().getName();
|
||||
if (!parameterSensitivities.containsKey(parameterName)) {
|
||||
// Null means not configured yet.
|
||||
parameterSensitivities.put(parameterName, null);
|
||||
}
|
||||
});
|
||||
|
||||
groupConfiguration = new ParameterGroupConfiguration(parameterGroup.getGroupName(), parameterContextName, parameterSensitivities, isSynchronized);
|
||||
parameterGroupConfigurations.add(groupConfiguration);
|
||||
});
|
||||
return parameterGroupConfigurations;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ParametersApplication> getFetchedParametersToApply(final Collection<ParameterGroupConfiguration> parameterGroupConfigurations) {
|
||||
readLock.lock();
|
||||
try {
|
||||
final Map<String, ParameterGroupConfiguration> parameterGroupConfigurationMap = parameterGroupConfigurations.stream()
|
||||
.collect(Collectors.toMap(ParameterGroupConfiguration::getParameterContextName, Function.identity()));
|
||||
final List<ParametersApplication> parametersApplications = new ArrayList<>();
|
||||
for (final ParameterContext parameterContext : getReferences()) {
|
||||
final ParameterGroupConfiguration groupConfiguration = parameterGroupConfigurationMap.get(parameterContext.getName());
|
||||
if (groupConfiguration == null || groupConfiguration.isSynchronized() == null || !groupConfiguration.isSynchronized()) {
|
||||
continue;
|
||||
}
|
||||
|
||||
final Map<String, Parameter> parameterUpdateMap = getFetchedParameterUpdateMap(parameterContext, groupConfiguration);
|
||||
parametersApplications.add(new ParametersApplication(parameterContext, parameterUpdateMap));
|
||||
}
|
||||
return parametersApplications;
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
protected String determineClasloaderIsolationKey() {
|
||||
final ConfigurableComponent component = getComponent();
|
||||
if (!(component instanceof ClassloaderIsolationKeyProvider)) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final ValidationContext validationContext = getValidationContextFactory().newValidationContext(getProperties(), getAnnotationData(), getProcessGroupIdentifier(), getIdentifier(),
|
||||
getParameterContext(), true);
|
||||
|
||||
return getClassLoaderIsolationKey(validationContext);
|
||||
}
|
||||
}
|
|
@ -19,6 +19,7 @@ package org.apache.nifi.controller.service;
|
|||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.controller.ComponentNode;
|
||||
import org.apache.nifi.controller.ControllerService;
|
||||
import org.apache.nifi.controller.ParameterProviderNode;
|
||||
import org.apache.nifi.controller.ProcessScheduler;
|
||||
import org.apache.nifi.controller.ProcessorNode;
|
||||
import org.apache.nifi.controller.ReportingTaskNode;
|
||||
|
@ -495,10 +496,13 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
|
|||
if (serviceNode == null) {
|
||||
final ReportingTaskNode taskNode = flowManager.getReportingTaskNode(componentId);
|
||||
if (taskNode == null) {
|
||||
throw new IllegalStateException("Could not find any Processor, Reporting Task, or Controller Service with identifier " + componentId);
|
||||
final ParameterProviderNode parameterProviderNode = flowManager.getParameterProvider(componentId);
|
||||
if (parameterProviderNode == null) {
|
||||
throw new IllegalStateException("Could not find any Processor, Reporting Task, Parameter Provider, or Controller Service with identifier " + componentId);
|
||||
}
|
||||
}
|
||||
|
||||
// we have confirmed that the component is a reporting task. We can only reference Controller Services
|
||||
// We have confirmed that the component is a reporting task or parameter provider. We can only reference Controller Services
|
||||
// that are scoped at the FlowController level in this case.
|
||||
final ControllerServiceNode rootServiceNode = flowManager.getRootControllerService(serviceIdentifier);
|
||||
return (rootServiceNode == null) ? null : rootServiceNode.getProxiedControllerService();
|
||||
|
|
|
@ -42,6 +42,7 @@ public class VolatileBulletinRepository implements BulletinRepository {
|
|||
private static final String CONTROLLER_BULLETIN_STORE_KEY = "CONTROLLER";
|
||||
private static final String SERVICE_BULLETIN_STORE_KEY = "SERVICE";
|
||||
private static final String REPORTING_TASK_BULLETIN_STORE_KEY = "REPORTING_TASK";
|
||||
private static final String PARAMETER_PROVIDER_BULLETIN_STORE_KEY = "PARAMETER_PROVIDER";
|
||||
|
||||
private final ConcurrentMap<String, ConcurrentMap<String, RingBuffer<Bulletin>>> bulletinStoreMap = new ConcurrentHashMap<>();
|
||||
private volatile BulletinProcessingStrategy processingStrategy = new DefaultBulletinProcessingStrategy();
|
||||
|
@ -299,6 +300,8 @@ public class VolatileBulletinRepository implements BulletinRepository {
|
|||
return SERVICE_BULLETIN_STORE_KEY;
|
||||
case REPORTING_TASK:
|
||||
return REPORTING_TASK_BULLETIN_STORE_KEY;
|
||||
case PARAMETER_PROVIDER:
|
||||
return PARAMETER_PROVIDER_BULLETIN_STORE_KEY;
|
||||
default:
|
||||
return bulletin.getGroupId();
|
||||
}
|
||||
|
@ -309,6 +312,7 @@ public class VolatileBulletinRepository implements BulletinRepository {
|
|||
case FLOW_CONTROLLER:
|
||||
case CONTROLLER_SERVICE:
|
||||
case REPORTING_TASK:
|
||||
case PARAMETER_PROVIDER:
|
||||
return true;
|
||||
default:
|
||||
return false;
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.nifi.connectable.Position;
|
|||
import org.apache.nifi.connectable.Size;
|
||||
import org.apache.nifi.controller.BackoffMechanism;
|
||||
import org.apache.nifi.controller.ComponentNode;
|
||||
import org.apache.nifi.controller.ParameterProviderNode;
|
||||
import org.apache.nifi.controller.ProcessorNode;
|
||||
import org.apache.nifi.controller.PropertyConfiguration;
|
||||
import org.apache.nifi.controller.ReportingTaskNode;
|
||||
|
@ -47,6 +48,7 @@ import org.apache.nifi.flow.BatchSize;
|
|||
import org.apache.nifi.flow.Bundle;
|
||||
import org.apache.nifi.flow.ComponentType;
|
||||
import org.apache.nifi.flow.ConnectableComponent;
|
||||
import org.apache.nifi.flow.ParameterProviderReference;
|
||||
import org.apache.nifi.flow.VersionedComponent;
|
||||
import org.apache.nifi.flow.VersionedConnection;
|
||||
import org.apache.nifi.flow.VersionedControllerService;
|
||||
|
@ -80,6 +82,9 @@ import org.apache.nifi.parameter.ParameterContextManager;
|
|||
import org.apache.nifi.parameter.ParameterDescriptor;
|
||||
import org.apache.nifi.parameter.ParameterReferenceManager;
|
||||
import org.apache.nifi.parameter.ParameterReferencedControllerServiceData;
|
||||
import org.apache.nifi.parameter.ParameterProviderConfiguration;
|
||||
import org.apache.nifi.parameter.ParameterReferenceManager;
|
||||
import org.apache.nifi.parameter.StandardParameterProviderConfiguration;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.registry.ComponentVariableRegistry;
|
||||
|
@ -245,7 +250,7 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
|||
|
||||
context.getFlowManager().withParameterContextResolution(() -> {
|
||||
try {
|
||||
synchronize(group, versionedExternalFlow.getFlowContents(), versionedExternalFlow.getParameterContexts());
|
||||
synchronize(group, versionedExternalFlow.getFlowContents(), versionedExternalFlow.getParameterContexts(), versionedExternalFlow.getParameterProviders());
|
||||
} catch (final ProcessorInstantiationException pie) {
|
||||
throw new RuntimeException(pie);
|
||||
}
|
||||
|
@ -254,7 +259,8 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
|||
group.onComponentModified();
|
||||
}
|
||||
|
||||
private void synchronize(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, VersionedParameterContext> versionedParameterContexts)
|
||||
private void synchronize(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, VersionedParameterContext> versionedParameterContexts,
|
||||
final Map<String, ParameterProviderReference> parameterProviderReferences)
|
||||
throws ProcessorInstantiationException {
|
||||
|
||||
// Some components, such as Processors, may have a Scheduled State of RUNNING in the proposed flow. However, if we
|
||||
|
@ -283,7 +289,7 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
|||
versionedParameterContexts.values().forEach(this::createParameterContextWithoutReferences);
|
||||
}
|
||||
|
||||
updateParameterContext(group, proposed, versionedParameterContexts, context.getComponentIdGenerator());
|
||||
updateParameterContext(group, proposed, versionedParameterContexts, parameterProviderReferences, context.getComponentIdGenerator());
|
||||
updateVariableRegistry(group, proposed);
|
||||
|
||||
final FlowFileConcurrency flowFileConcurrency = proposed.getFlowFileConcurrency() == null ? FlowFileConcurrency.UNBOUNDED :
|
||||
|
@ -398,7 +404,7 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
|||
removeMissingChildGroups(group, proposed, childGroupsByVersionedId);
|
||||
|
||||
// Synchronize Child Process Groups
|
||||
synchronizeChildGroups(group, proposed, versionedParameterContexts, childGroupsByVersionedId);
|
||||
synchronizeChildGroups(group, proposed, versionedParameterContexts, childGroupsByVersionedId, parameterProviderReferences);
|
||||
|
||||
synchronizeFunnels(group, proposed, funnelsByVersionedId);
|
||||
synchronizeInputPorts(group, proposed, proposedPortFinalNames, inputPortsByVersionedId);
|
||||
|
@ -453,7 +459,8 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
|||
}
|
||||
|
||||
private void synchronizeChildGroups(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, VersionedParameterContext> versionedParameterContexts,
|
||||
final Map<String, ProcessGroup> childGroupsByVersionedId) throws ProcessorInstantiationException {
|
||||
final Map<String, ProcessGroup> childGroupsByVersionedId,
|
||||
final Map<String, ParameterProviderReference> parameterProviderReferences) throws ProcessorInstantiationException {
|
||||
|
||||
for (final VersionedProcessGroup proposedChildGroup : proposed.getProcessGroups()) {
|
||||
final ProcessGroup childGroup = childGroupsByVersionedId.get(proposedChildGroup.getIdentifier());
|
||||
|
@ -472,7 +479,8 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
|||
}
|
||||
|
||||
if (childGroup == null) {
|
||||
final ProcessGroup added = addProcessGroup(group, proposedChildGroup, context.getComponentIdGenerator(), preExistingVariables, childParameterContexts);
|
||||
final ProcessGroup added = addProcessGroup(group, proposedChildGroup, context.getComponentIdGenerator(), preExistingVariables,
|
||||
childParameterContexts, parameterProviderReferences);
|
||||
context.getFlowManager().onProcessGroupAdded(added);
|
||||
added.findAllRemoteProcessGroups().forEach(RemoteProcessGroup::initialize);
|
||||
LOG.info("Added {} to {}", added, group);
|
||||
|
@ -486,7 +494,7 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
|||
.build();
|
||||
|
||||
sync.setSynchronizationOptions(options);
|
||||
sync.synchronize(childGroup, proposedChildGroup, childParameterContexts);
|
||||
sync.synchronize(childGroup, proposedChildGroup, childParameterContexts, parameterProviderReferences);
|
||||
|
||||
LOG.info("Updated {}", childGroup);
|
||||
}
|
||||
|
@ -1046,7 +1054,8 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
|||
}
|
||||
|
||||
private ProcessGroup addProcessGroup(final ProcessGroup destination, final VersionedProcessGroup proposed, final ComponentIdGenerator componentIdGenerator, final Set<String> variablesToSkip,
|
||||
final Map<String, VersionedParameterContext> versionedParameterContexts) throws ProcessorInstantiationException {
|
||||
final Map<String, VersionedParameterContext> versionedParameterContexts,
|
||||
final Map<String, ParameterProviderReference> parameterProviderReferences) throws ProcessorInstantiationException {
|
||||
final String id = componentIdGenerator.generateUuid(proposed.getIdentifier(), proposed.getInstanceIdentifier(), destination.getIdentifier());
|
||||
final ProcessGroup group = context.getFlowManager().createProcessGroup(id);
|
||||
group.setVersionedComponentId(proposed.getIdentifier());
|
||||
|
@ -1063,7 +1072,7 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
|||
.updateGroupSettings(true)
|
||||
.build();
|
||||
sync.setSynchronizationOptions(options);
|
||||
sync.synchronize(group, proposed, versionedParameterContexts);
|
||||
sync.synchronize(group, proposed, versionedParameterContexts, parameterProviderReferences);
|
||||
|
||||
return group;
|
||||
}
|
||||
|
@ -1791,7 +1800,7 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
|||
}
|
||||
|
||||
private void updateParameterContext(final ProcessGroup group, final VersionedProcessGroup proposed, final Map<String, VersionedParameterContext> versionedParameterContexts,
|
||||
final ComponentIdGenerator componentIdGenerator) {
|
||||
final Map<String, ParameterProviderReference> parameterProviderReferences, final ComponentIdGenerator componentIdGenerator) {
|
||||
// Update the Parameter Context
|
||||
final ParameterContext currentParamContext = group.getParameterContext();
|
||||
final String proposedParameterContextName = proposed.getParameterContextName();
|
||||
|
@ -1799,7 +1808,7 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
|||
group.setParameterContext(null);
|
||||
} else if (proposedParameterContextName != null) {
|
||||
final VersionedParameterContext versionedParameterContext = versionedParameterContexts.get(proposedParameterContextName);
|
||||
|
||||
createMissingParameterProvider(versionedParameterContext, versionedParameterContext.getParameterProvider(), parameterProviderReferences, componentIdGenerator);
|
||||
if (currentParamContext == null) {
|
||||
// Create a new Parameter Context based on the parameters provided
|
||||
|
||||
|
@ -1828,6 +1837,32 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
|||
}
|
||||
}
|
||||
|
||||
private void createMissingParameterProvider(final VersionedParameterContext versionedParameterContext, final String parameterProviderId,
|
||||
final Map<String, ParameterProviderReference> parameterProviderReferences, final ComponentIdGenerator componentIdGenerator) {
|
||||
String parameterProviderIdToSet = parameterProviderId;
|
||||
if (parameterProviderId != null) {
|
||||
ParameterProviderNode parameterProviderNode = context.getFlowManager().getParameterProvider(parameterProviderId);
|
||||
if (parameterProviderNode == null) {
|
||||
final ParameterProviderReference reference = parameterProviderReferences.get(parameterProviderId);
|
||||
if (reference == null) {
|
||||
parameterProviderIdToSet = null;
|
||||
} else {
|
||||
final String newParameterProviderId = componentIdGenerator.generateUuid(parameterProviderId, parameterProviderId, null);
|
||||
|
||||
final Bundle bundle = reference.getBundle();
|
||||
parameterProviderNode = context.getFlowManager().createParameterProvider(reference.getType(), newParameterProviderId,
|
||||
new BundleCoordinate(bundle.getGroup(), bundle.getArtifact(), bundle.getVersion()), true);
|
||||
|
||||
parameterProviderNode.pauseValidationTrigger(); // avoid triggering validation multiple times
|
||||
parameterProviderNode.setName(reference.getName());
|
||||
parameterProviderNode.resumeValidationTrigger();
|
||||
parameterProviderIdToSet = parameterProviderNode.getIdentifier();
|
||||
}
|
||||
}
|
||||
}
|
||||
versionedParameterContext.setParameterProvider(parameterProviderIdToSet);
|
||||
}
|
||||
|
||||
private void updateVariableRegistry(final ProcessGroup group, final VersionedProcessGroup proposed) {
|
||||
// Determine which variables have been added/removed and add/remove them from this group's variable registry.
|
||||
// We don't worry about if a variable value has changed, because variables are designed to be 'environment specific.'
|
||||
|
@ -1884,17 +1919,25 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
|||
|
||||
final Map<String, Parameter> parameters = new HashMap<>();
|
||||
for (final VersionedParameter versionedParameter : versionedParameterContext.getParameters()) {
|
||||
if (versionedParameter == null) {
|
||||
continue;
|
||||
}
|
||||
final ParameterDescriptor descriptor = new ParameterDescriptor.Builder()
|
||||
.name(versionedParameter.getName())
|
||||
.description(versionedParameter.getDescription())
|
||||
.sensitive(versionedParameter.isSensitive())
|
||||
.build();
|
||||
|
||||
final Parameter parameter = new Parameter(descriptor, versionedParameter.getValue());
|
||||
final Parameter parameter = new Parameter(descriptor, versionedParameter.getValue(), null, versionedParameter.isProvided());
|
||||
parameters.put(versionedParameter.getName(), parameter);
|
||||
}
|
||||
|
||||
return context.getFlowManager().createParameterContext(parameterContextId, versionedParameterContext.getName(), parameters, Collections.emptyList());
|
||||
return context.getFlowManager().createParameterContext(parameterContextId, versionedParameterContext.getName(), parameters, Collections.emptyList(), null);
|
||||
}
|
||||
|
||||
private ParameterProviderConfiguration getParameterProviderConfiguration(final VersionedParameterContext context) {
|
||||
return context.getParameterProvider() == null ? null
|
||||
: new StandardParameterProviderConfiguration(context.getParameterProvider(), context.getParameterGroupName(), context.isSynchronized());
|
||||
}
|
||||
|
||||
private ParameterContext createParameterContext(final VersionedParameterContext versionedParameterContext, final String parameterContextId,
|
||||
|
@ -1911,7 +1954,8 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
|||
|
||||
final AtomicReference<ParameterContext> contextReference = new AtomicReference<>();
|
||||
context.getFlowManager().withParameterContextResolution(() -> {
|
||||
final ParameterContext created = context.getFlowManager().createParameterContext(parameterContextId, versionedParameterContext.getName(), parameters, parameterContextRefs);
|
||||
final ParameterContext created = context.getFlowManager().createParameterContext(parameterContextId, versionedParameterContext.getName(), parameters, parameterContextRefs,
|
||||
getParameterProviderConfiguration(versionedParameterContext));
|
||||
contextReference.set(created);
|
||||
});
|
||||
|
||||
|
@ -1927,7 +1971,7 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
|||
.sensitive(versionedParameter.isSensitive())
|
||||
.build();
|
||||
|
||||
final Parameter parameter = new Parameter(descriptor, versionedParameter.getValue());
|
||||
final Parameter parameter = new Parameter(descriptor, versionedParameter.getValue(), null, versionedParameter.isProvided());
|
||||
parameters.put(versionedParameter.getName(), parameter);
|
||||
}
|
||||
|
||||
|
@ -1971,7 +2015,7 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
|||
.sensitive(versionedParameter.isSensitive())
|
||||
.build();
|
||||
|
||||
final Parameter parameter = new Parameter(descriptor, versionedParameter.getValue());
|
||||
final Parameter parameter = new Parameter(descriptor, versionedParameter.getValue(), null, versionedParameter.isProvided());
|
||||
parameters.put(versionedParameter.getName(), parameter);
|
||||
}
|
||||
|
||||
|
@ -1985,6 +2029,9 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
|||
.map(name -> selectParameterContext(versionedParameterContexts.get(name), versionedParameterContexts))
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
if (versionedParameterContext.getParameterProvider() != null && currentParameterContext.getParameterProvider() == null) {
|
||||
currentParameterContext.configureParameterProvider(getParameterProviderConfiguration(versionedParameterContext));
|
||||
}
|
||||
}
|
||||
|
||||
private boolean isEqual(final BundleCoordinate coordinate, final Bundle bundle) {
|
||||
|
|
|
@ -16,12 +16,38 @@
|
|||
*/
|
||||
package org.apache.nifi.parameter;
|
||||
|
||||
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
||||
|
||||
/**
|
||||
* Represents only an ID reference for a ParameterContext.
|
||||
*/
|
||||
public class ReferenceOnlyParameterContext extends StandardParameterContext {
|
||||
|
||||
public ReferenceOnlyParameterContext(String id) {
|
||||
super(id, String.format("Reference-Only Parameter Context [%s]", id), ParameterReferenceManager.EMPTY, null);
|
||||
public ReferenceOnlyParameterContext(final String id) {
|
||||
super(id, String.format("Reference-Only Parameter Context [%s]", id), ParameterReferenceManager.EMPTY, null, null, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* A ParameterContext's identity is its identifier.
|
||||
* @param obj Another object
|
||||
* @return Whether this is equal to the object
|
||||
*/
|
||||
@Override
|
||||
public boolean equals(final Object obj) {
|
||||
if (obj instanceof ReferenceOnlyParameterContext) {
|
||||
final ReferenceOnlyParameterContext other = (ReferenceOnlyParameterContext) obj;
|
||||
return (getIdentifier().equals(other.getIdentifier()));
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A ParameterContext's identity is its identifier.
|
||||
* @return The hash code
|
||||
*/
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return new HashCodeBuilder().append(getClass().getName()).append(getIdentifier()).toHashCode();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.nifi.parameter;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
||||
import org.apache.nifi.authorization.AccessDeniedException;
|
||||
import org.apache.nifi.authorization.Authorizer;
|
||||
import org.apache.nifi.authorization.RequestAction;
|
||||
|
@ -27,8 +28,10 @@ import org.apache.nifi.authorization.resource.ResourceType;
|
|||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.controller.ComponentNode;
|
||||
import org.apache.nifi.controller.ParameterProviderNode;
|
||||
import org.apache.nifi.controller.ProcessorNode;
|
||||
import org.apache.nifi.controller.PropertyConfiguration;
|
||||
import org.apache.nifi.controller.parameter.ParameterProviderLookup;
|
||||
import org.apache.nifi.controller.service.ControllerServiceNode;
|
||||
import org.apache.nifi.controller.service.ControllerServiceState;
|
||||
import org.apache.nifi.groups.ProcessGroup;
|
||||
|
@ -54,24 +57,30 @@ public class StandardParameterContext implements ParameterContext {
|
|||
|
||||
private final String id;
|
||||
private final ParameterReferenceManager parameterReferenceManager;
|
||||
private final ParameterProviderLookup parameterProviderLookup;
|
||||
private final Authorizable parentAuthorizable;
|
||||
|
||||
private String name;
|
||||
private long version = 0L;
|
||||
private final Map<ParameterDescriptor, Parameter> parameters = new LinkedHashMap<>();
|
||||
private final List<ParameterContext> inheritedParameterContexts = new ArrayList<>();
|
||||
private ParameterProvider parameterProvider;
|
||||
private ParameterProviderConfiguration parameterProviderConfiguration;
|
||||
private volatile String description;
|
||||
|
||||
private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
|
||||
private final Lock readLock = rwLock.readLock();
|
||||
private final Lock writeLock = rwLock.writeLock();
|
||||
|
||||
public StandardParameterContext(final String id, final String name, final ParameterReferenceManager parameterReferenceManager,
|
||||
final Authorizable parentAuthorizable) {
|
||||
protected StandardParameterContext(final String id, final String name, final ParameterReferenceManager parameterReferenceManager,
|
||||
final Authorizable parentAuthorizable, final ParameterProviderLookup parameterProviderLookup,
|
||||
final ParameterProviderConfiguration parameterProviderConfiguration) {
|
||||
this.id = Objects.requireNonNull(id);
|
||||
this.name = Objects.requireNonNull(name);
|
||||
this.parameterReferenceManager = parameterReferenceManager;
|
||||
this.parentAuthorizable = parentAuthorizable;
|
||||
this.parameterProviderLookup = parameterProviderLookup;
|
||||
this.configureParameterProvider(parameterProviderConfiguration);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -115,11 +124,10 @@ public class StandardParameterContext implements ParameterContext {
|
|||
|
||||
@Override
|
||||
public void setParameters(final Map<String, Parameter> updatedParameters) {
|
||||
final Map<String, ParameterUpdate> parameterUpdates = new HashMap<>();
|
||||
|
||||
writeLock.lock();
|
||||
try {
|
||||
this.version++;
|
||||
|
||||
final Map<ParameterDescriptor, Parameter> currentEffectiveParameters = getEffectiveParameters();
|
||||
final Map<ParameterDescriptor, Parameter> effectiveProposedParameters = getEffectiveParameters(getProposedParameters(updatedParameters));
|
||||
|
||||
|
@ -131,12 +139,12 @@ public class StandardParameterContext implements ParameterContext {
|
|||
updateParameters(parameters, updatedParameters, true);
|
||||
|
||||
// Get a list of all effective updates in order to alert referencing components
|
||||
parameterUpdates.putAll(updateParameters(currentEffectiveParameters, effectiveParameterUpdates, false));
|
||||
final Map<String, ParameterUpdate> parameterUpdates = new HashMap<>(updateParameters(currentEffectiveParameters, effectiveParameterUpdates, false));
|
||||
|
||||
alertReferencingComponents(parameterUpdates);
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
|
||||
alertReferencingComponents(parameterUpdates);
|
||||
}
|
||||
|
||||
private Map<ParameterDescriptor, Parameter> getProposedParameters(final Map<String, Parameter> proposedParameterUpdates) {
|
||||
|
@ -146,9 +154,7 @@ public class StandardParameterContext implements ParameterContext {
|
|||
final Parameter parameter = entry.getValue();
|
||||
if (parameter == null) {
|
||||
final Optional<Parameter> existingParameter = getParameter(parameterName);
|
||||
if (existingParameter.isPresent()) {
|
||||
proposedParameters.remove(existingParameter.get().getDescriptor());
|
||||
}
|
||||
existingParameter.ifPresent(value -> proposedParameters.remove(value.getDescriptor()));
|
||||
} else {
|
||||
// Remove is necessary first in case sensitivity changes
|
||||
proposedParameters.remove(parameter.getDescriptor());
|
||||
|
@ -202,9 +208,14 @@ public class StandardParameterContext implements ParameterContext {
|
|||
parameterUpdates.put(parameterName, new StandardParameterUpdate(parameterName, oldParameter.getValue(), null, parameterDescriptor.isSensitive()));
|
||||
} else {
|
||||
final Parameter updatedParameter = createFullyPopulatedParameter(parameter);
|
||||
|
||||
final Parameter oldParameter = performUpdate ? currentParameters.put(updatedParameter.getDescriptor(), updatedParameter)
|
||||
: currentParameters.get(updatedParameter.getDescriptor());
|
||||
final Parameter oldParameter;
|
||||
if (performUpdate) {
|
||||
// Necessary to remove first, because the sensitivity may change, and ParameterDescriptor#hashCode only relies on `name`
|
||||
oldParameter = currentParameters.remove(updatedParameter.getDescriptor());
|
||||
currentParameters.put(updatedParameter.getDescriptor(), updatedParameter);
|
||||
} else {
|
||||
oldParameter = currentParameters.get(updatedParameter.getDescriptor());
|
||||
}
|
||||
if (oldParameter == null || !Objects.equals(oldParameter.getValue(), updatedParameter.getValue())) {
|
||||
final String previousValue = oldParameter == null ? null : oldParameter.getValue();
|
||||
parameterUpdates.put(parameterName, new StandardParameterUpdate(parameterName, previousValue, updatedParameter.getValue(), updatedParameter.getDescriptor().isSensitive()));
|
||||
|
@ -229,7 +240,7 @@ public class StandardParameterContext implements ParameterContext {
|
|||
private Parameter createFullyPopulatedParameter(final Parameter proposedParameter) {
|
||||
final ParameterDescriptor descriptor = getFullyPopulatedDescriptor(proposedParameter);
|
||||
final String value = getFullyPopulatedValue(proposedParameter);
|
||||
return new Parameter(descriptor, value);
|
||||
return new Parameter(descriptor, value, proposedParameter.getParameterContextId(), proposedParameter.isProvided());
|
||||
}
|
||||
|
||||
private String getFullyPopulatedValue(final Parameter proposedParameter) {
|
||||
|
@ -244,6 +255,14 @@ public class StandardParameterContext implements ParameterContext {
|
|||
|
||||
final Parameter oldParameter = parameters.get(proposedParameter.getDescriptor());
|
||||
|
||||
if (proposedParameter.isProvided()) {
|
||||
final String description = oldParameter == null ? null : oldParameter.getDescriptor().getDescription();
|
||||
return new ParameterDescriptor.Builder()
|
||||
.from(descriptor)
|
||||
.description(description)
|
||||
.build();
|
||||
}
|
||||
|
||||
// We know that the Parameters have the same name, since this is what the Descriptor's hashCode & equality are based on. The only thing that may be different
|
||||
// is the description. And since the proposed Parameter does not have a Description, we want to use whatever is currently set.
|
||||
return oldParameter == null ? descriptor : oldParameter.getDescriptor();
|
||||
|
@ -297,17 +316,6 @@ public class StandardParameterContext implements ParameterContext {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasEffectiveValueIfRemoved(final ParameterDescriptor parameterDescriptor) {
|
||||
final Map<ParameterDescriptor, List<Parameter>> allOverrides = getAllParametersIncludingOverrides();
|
||||
final List<Parameter> parameters = allOverrides.get(parameterDescriptor);
|
||||
if (parameters == null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return parameters.size() > 1;
|
||||
}
|
||||
|
||||
private ParameterDescriptor unescape(final ParameterDescriptor descriptor) {
|
||||
final String parameterName = descriptor.getName().trim();
|
||||
if ((parameterName.startsWith("'") && parameterName.endsWith("'")) || (parameterName.startsWith("\"") && parameterName.endsWith("\""))) {
|
||||
|
@ -347,7 +355,7 @@ public class StandardParameterContext implements ParameterContext {
|
|||
Objects.requireNonNull(inheritedParameterContexts, "Inherited parameter contexts must be specified");
|
||||
|
||||
final Map<ParameterDescriptor, Parameter> currentEffectiveParameters = getEffectiveParameters();
|
||||
final Map<ParameterDescriptor, Parameter> effectiveProposedParameters = getEffectiveParameters(inheritedParameterContexts, getProposedParameters(parameterUpdates), new HashMap<>());
|
||||
final Map<ParameterDescriptor, Parameter> effectiveProposedParameters = getEffectiveParameters(inheritedParameterContexts, getProposedParameters(parameterUpdates));
|
||||
|
||||
return getEffectiveParameterUpdates(currentEffectiveParameters, effectiveProposedParameters);
|
||||
}
|
||||
|
@ -359,7 +367,7 @@ public class StandardParameterContext implements ParameterContext {
|
|||
* @return The view of the parameters with all overriding applied
|
||||
*/
|
||||
private Map<ParameterDescriptor, Parameter> getEffectiveParameters(final Map<ParameterDescriptor, Parameter> proposedParameters) {
|
||||
return getEffectiveParameters(this.inheritedParameterContexts, proposedParameters, new HashMap<>());
|
||||
return getEffectiveParameters(this.inheritedParameterContexts, proposedParameters);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -369,13 +377,12 @@ public class StandardParameterContext implements ParameterContext {
|
|||
* @return The view of the parameters with all overriding applied
|
||||
*/
|
||||
private Map<ParameterDescriptor, Parameter> getEffectiveParameters(final List<ParameterContext> parameterContexts) {
|
||||
return getEffectiveParameters(parameterContexts, this.parameters, new HashMap<>());
|
||||
return getEffectiveParameters(parameterContexts, this.parameters);
|
||||
}
|
||||
|
||||
private Map<ParameterDescriptor, List<Parameter>> getAllParametersIncludingOverrides() {
|
||||
final Map<ParameterDescriptor, List<Parameter>> allOverrides = new HashMap<>();
|
||||
getEffectiveParameters(this.inheritedParameterContexts, this.parameters, allOverrides);
|
||||
return allOverrides;
|
||||
private Map<ParameterDescriptor, Parameter> getEffectiveParameters(final List<ParameterContext> parameterContexts,
|
||||
final Map<ParameterDescriptor, Parameter> proposedParameters) {
|
||||
return getEffectiveParameters(parameterContexts, proposedParameters, new HashMap<>());
|
||||
}
|
||||
|
||||
private Map<ParameterDescriptor, Parameter> getEffectiveParameters(final List<ParameterContext> parameterContexts,
|
||||
|
@ -446,6 +453,72 @@ public class StandardParameterContext implements ParameterContext {
|
|||
return parameterReferenceManager;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ParameterProviderLookup getParameterProviderLookup() {
|
||||
return parameterProviderLookup;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ParameterProvider getParameterProvider() {
|
||||
readLock.lock();
|
||||
try {
|
||||
return parameterProvider;
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void configureParameterProvider(final ParameterProviderConfiguration parameterProviderConfiguration) {
|
||||
if (parameterProviderConfiguration == null) {
|
||||
return;
|
||||
}
|
||||
this.parameterProviderConfiguration = parameterProviderConfiguration;
|
||||
|
||||
writeLock.lock();
|
||||
try {
|
||||
final ParameterProviderNode parameterProviderNode = parameterProviderLookup.getParameterProvider(parameterProviderConfiguration.getParameterProviderId());
|
||||
if (parameterProviderNode == null) {
|
||||
throw new IllegalArgumentException(String.format("Could not configure Parameter Provider %s, which could not be found",
|
||||
parameterProviderConfiguration.getParameterProviderId()));
|
||||
}
|
||||
final boolean hasUserEnteredParameters = parameters.values().stream().anyMatch(parameter -> !parameter.isProvided());
|
||||
|
||||
if (hasUserEnteredParameters) {
|
||||
throw new IllegalArgumentException(String.format("A Parameter Provider [%s] cannot be set since there are already user-entered parameters " +
|
||||
"in Context [%s]", parameterProvider.getIdentifier(), name));
|
||||
}
|
||||
|
||||
this.parameterProvider = parameterProviderNode.getParameterProvider();
|
||||
registerParameterProvider(parameterProvider);
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ParameterProviderConfiguration getParameterProviderConfiguration() {
|
||||
readLock.lock();
|
||||
try {
|
||||
return parameterProviderConfiguration;
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
private void registerParameterProvider(final ParameterProvider parameterProvider) {
|
||||
final ParameterProviderNode parameterProviderNode = getParameterProviderNode(parameterProvider);
|
||||
parameterProviderNode.addReference(this);
|
||||
}
|
||||
|
||||
private ParameterProviderNode getParameterProviderNode(final ParameterProvider parameterProvider) {
|
||||
final ParameterProviderNode parameterProviderNode = parameterProviderLookup.getParameterProvider(parameterProvider.getIdentifier());
|
||||
if (parameterProviderNode == null) {
|
||||
throw new IllegalStateException(String.format("Parameter Provider Node is missing for Parameter Provider [%s]", parameterProvider.getIdentifier()));
|
||||
}
|
||||
return parameterProviderNode;
|
||||
}
|
||||
|
||||
/**
|
||||
* Verifies that no cycles would exist in the ParameterContext reference graph, if this ParameterContext were
|
||||
* to inherit from the given list of ParameterContexts.
|
||||
|
@ -482,15 +555,16 @@ public class StandardParameterContext implements ParameterContext {
|
|||
verifyCanUpdateParameterContext(parameterUpdates, inheritedParameterContexts, false);
|
||||
}
|
||||
|
||||
private void verifyCanUpdateParameterContext(final Map<String, Parameter> parameterUpdates, final List<ParameterContext> inheritedParameterContexts, final boolean duringUpdate) {
|
||||
private void verifyCanUpdateParameterContext(final Map<String, Parameter> parameterUpdates, final List<ParameterContext> inheritedParameterContexts,
|
||||
final boolean duringUpdate) {
|
||||
verifyProvidedParameters(parameterUpdates);
|
||||
if (inheritedParameterContexts == null) {
|
||||
return;
|
||||
}
|
||||
verifyNoCycles(inheritedParameterContexts);
|
||||
|
||||
final Map<ParameterDescriptor, Parameter> currentEffectiveParameters = getEffectiveParameters();
|
||||
final Map<ParameterDescriptor, Parameter> effectiveProposedParameters = getEffectiveParameters(inheritedParameterContexts, getProposedParameters(parameterUpdates), new HashMap<>());
|
||||
final Map<String, Parameter> effectiveParameterUpdates = getEffectiveParameterUpdates(currentEffectiveParameters, effectiveProposedParameters);
|
||||
final Map<String, Parameter> effectiveParameterUpdates = getEffectiveParameterUpdates(parameterUpdates, inheritedParameterContexts);
|
||||
|
||||
try {
|
||||
verifyCanSetParameters(currentEffectiveParameters, effectiveParameterUpdates, duringUpdate);
|
||||
|
@ -501,6 +575,15 @@ public class StandardParameterContext implements ParameterContext {
|
|||
}
|
||||
}
|
||||
|
||||
private void verifyProvidedParameters(final Map<String, Parameter> parameterUpdates) {
|
||||
parameterUpdates.forEach((parameterName, parameter) -> {
|
||||
if (parameterProvider != null && parameter != null && !parameter.isProvided()) {
|
||||
throw new IllegalArgumentException(String.format("Cannot make user-entered updates to Parameter Context [%s] parameter [%s] because its parameters " +
|
||||
"are provided by Parameter Provider [%s]", name, parameterName, parameterProvider.getIdentifier()));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setInheritedParameterContexts(final List<ParameterContext> inheritedParameterContexts) {
|
||||
if (inheritedParameterContexts == null || inheritedParameterContexts.equals(this.inheritedParameterContexts)) {
|
||||
|
@ -538,7 +621,7 @@ public class StandardParameterContext implements ParameterContext {
|
|||
* @param effectiveProposedParameters A map of effective parameters that would result if a proposed update were applied
|
||||
* @return a map that can be used to indicate all effective parameters updates, including removed parameters
|
||||
*/
|
||||
private static Map<String, Parameter> getEffectiveParameterUpdates(final Map<ParameterDescriptor, Parameter> currentEffectiveParameters,
|
||||
private Map<String, Parameter> getEffectiveParameterUpdates(final Map<ParameterDescriptor, Parameter> currentEffectiveParameters,
|
||||
final Map<ParameterDescriptor, Parameter> effectiveProposedParameters) {
|
||||
final Map<String, Parameter> effectiveParameterUpdates = new HashMap<>();
|
||||
for (final Map.Entry<ParameterDescriptor, Parameter> entry : effectiveProposedParameters.entrySet()) {
|
||||
|
@ -558,14 +641,30 @@ public class StandardParameterContext implements ParameterContext {
|
|||
}
|
||||
for (final Map.Entry<ParameterDescriptor, Parameter> entry : currentEffectiveParameters.entrySet()) {
|
||||
final ParameterDescriptor currentParameterDescriptor = entry.getKey();
|
||||
if (!effectiveProposedParameters.containsKey(currentParameterDescriptor)) {
|
||||
// If a current parameter is not in the proposed parameters, it was effectively removed
|
||||
if (!effectiveProposedParameters.containsKey(currentParameterDescriptor)) {
|
||||
final Parameter parameter = entry.getValue();
|
||||
if (parameter.isProvided() && hasReferencingComponents(parameter)) {
|
||||
logger.info("Provided parameter [{}] was removed from the source, but it is referenced by a component, so the parameter will be preserved",
|
||||
currentParameterDescriptor.getName());
|
||||
} else {
|
||||
effectiveParameterUpdates.put(currentParameterDescriptor.getName(), null);
|
||||
}
|
||||
}
|
||||
}
|
||||
return effectiveParameterUpdates;
|
||||
}
|
||||
|
||||
private boolean hasReferencingComponents(final Parameter parameter) {
|
||||
if (parameter == null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final String parameterName = parameter.getDescriptor().getName();
|
||||
return parameterReferenceManager.getProcessorsReferencing(this, parameterName).size() > 0
|
||||
|| parameterReferenceManager.getControllerServicesReferencing(this, parameterName).size() > 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ParameterContext> getInheritedParameterContexts() {
|
||||
readLock.lock();
|
||||
|
@ -580,8 +679,7 @@ public class StandardParameterContext implements ParameterContext {
|
|||
public List<String> getInheritedParameterContextNames() {
|
||||
readLock.lock();
|
||||
try {
|
||||
return inheritedParameterContexts.stream().map(ParameterContext::getName)
|
||||
.collect(Collectors.toList());
|
||||
return inheritedParameterContexts.stream().map(ParameterContext::getName).collect(Collectors.toList());
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
|
@ -621,11 +719,18 @@ public class StandardParameterContext implements ParameterContext {
|
|||
* assumed that these will be stopped prior to the actual update.
|
||||
* @throws IllegalStateException if setting the given set of Parameters is not legal
|
||||
*/
|
||||
public void verifyCanSetParameters(final Map<String, Parameter> updatedParameters, final boolean duringUpdate) {
|
||||
private void verifyCanSetParameters(final Map<String, Parameter> updatedParameters, final boolean duringUpdate) {
|
||||
verifyCanSetParameters(parameters, updatedParameters, duringUpdate);
|
||||
}
|
||||
|
||||
public void verifyCanSetParameters(final Map<ParameterDescriptor, Parameter> currentParameters, final Map<String, Parameter> updatedParameters, final boolean duringUpdate) {
|
||||
final boolean updatingUserEnteredParameters = updatedParameters.values().stream()
|
||||
.anyMatch(parameter -> parameter != null && !parameter.isProvided());
|
||||
if (parameterProvider != null && updatingUserEnteredParameters) {
|
||||
throw new IllegalArgumentException(String.format("Parameters for Context [%s] cannot be manually updated because they are " +
|
||||
"provided by Parameter Provider [%s]", name, parameterProvider.getIdentifier()));
|
||||
}
|
||||
|
||||
// Ensure that the updated parameters will not result in changing the sensitivity flag of any parameter.
|
||||
for (final Map.Entry<String, Parameter> entry : updatedParameters.entrySet()) {
|
||||
final String parameterName = entry.getKey();
|
||||
|
@ -657,6 +762,9 @@ public class StandardParameterContext implements ParameterContext {
|
|||
if (existingDescriptor.isSensitive() != updatedDescriptor.isSensitive() && updatedParameter.getValue() != null) {
|
||||
final String existingSensitiveDescription = existingDescriptor.isSensitive() ? "sensitive" : "not sensitive";
|
||||
final String updatedSensitiveDescription = updatedDescriptor.isSensitive() ? "sensitive" : "not sensitive";
|
||||
if (existingParameter.isProvided() && updatedParameter.isProvided()) {
|
||||
return;
|
||||
}
|
||||
|
||||
throw new IllegalStateException("Cannot update Parameters because doing so would change Parameter '" + existingDescriptor.getName() + "' from " + existingSensitiveDescription
|
||||
+ " to " + updatedSensitiveDescription);
|
||||
|
@ -728,23 +836,27 @@ public class StandardParameterContext implements ParameterContext {
|
|||
for (final ParameterContext parameterContext : inheritedParameterContexts) {
|
||||
parameterContext.authorize(authorizer, action, user);
|
||||
}
|
||||
if (parameterProvider != null) {
|
||||
authorizeParameterProviderRead(authorizer, parameterProvider, user);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAuthorized(final Authorizer authorizer, final RequestAction action, final NiFiUser user) {
|
||||
boolean isAuthorized = ParameterContext.super.isAuthorized(authorizer, action, user);
|
||||
|
||||
if (RequestAction.READ == action) {
|
||||
for (final ParameterContext parameterContext : inheritedParameterContexts) {
|
||||
isAuthorized &= parameterContext.isAuthorized(authorizer, action, user);
|
||||
if (!isAuthorized) {
|
||||
break;
|
||||
}
|
||||
try {
|
||||
authorize(authorizer, action, user);
|
||||
return true;
|
||||
} catch (final AccessDeniedException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return isAuthorized;
|
||||
private void authorizeParameterProviderRead(final Authorizer authorizer, final ParameterProvider parameterProvider, final NiFiUser user) {
|
||||
final ParameterProviderNode parameterProviderNode = parameterProviderLookup.getParameterProvider(parameterProvider.getIdentifier());
|
||||
if (parameterProviderNode != null) {
|
||||
parameterProviderNode.authorize(authorizer, RequestAction.READ, user);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -766,4 +878,101 @@ public class StandardParameterContext implements ParameterContext {
|
|||
public Resource getResource() {
|
||||
return ResourceFactory.getComponentResource(ResourceType.ParameterContext, getIdentifier(), getName());
|
||||
}
|
||||
|
||||
/**
|
||||
* A ParameterContext's identity is its identifier.
|
||||
* @param obj Another object
|
||||
* @return Whether this is equal to the object
|
||||
*/
|
||||
@Override
|
||||
public boolean equals(final Object obj) {
|
||||
if (obj != null && obj.getClass().equals(StandardParameterContext.class)) {
|
||||
final StandardParameterContext other = (StandardParameterContext) obj;
|
||||
return (getIdentifier().equals(other.getIdentifier()));
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A ParameterContext's identity is its identifier.
|
||||
* @return The hash code
|
||||
*/
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return new HashCodeBuilder().append(getClass().getName()).append(getIdentifier()).toHashCode();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
private String id;
|
||||
private String name;
|
||||
private ParameterReferenceManager parameterReferenceManager;
|
||||
private Authorizable parentAuthorizable;
|
||||
private ParameterProviderLookup parameterProviderLookup;
|
||||
private ParameterProviderConfiguration parameterProviderConfiguration;
|
||||
|
||||
/**
|
||||
* Sets the ParameterContext id -- required
|
||||
* @param id The ParameterContext id
|
||||
* @return The builder
|
||||
*/
|
||||
public Builder id(final String id) {
|
||||
this.id = id;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the ParameterContext name -- required
|
||||
* @param name The ParameterContext name
|
||||
* @return The builder
|
||||
*/
|
||||
public Builder name(final String name) {
|
||||
this.name = name;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the ParameterReferenceManager
|
||||
* @param parameterReferenceManager The ParameterReferenceManager
|
||||
* @return The builder
|
||||
*/
|
||||
public Builder parameterReferenceManager(final ParameterReferenceManager parameterReferenceManager) {
|
||||
this.parameterReferenceManager = parameterReferenceManager;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the parent Authorizable
|
||||
* @param parentAuthorizable The parent Authorizable
|
||||
* @return The builder
|
||||
*/
|
||||
public Builder parentAuthorizable(final Authorizable parentAuthorizable) {
|
||||
this.parentAuthorizable = parentAuthorizable;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the ParameterProviderLookup
|
||||
* @param parameterProviderLookup The ParameterProviderLookup
|
||||
* @return The builder
|
||||
*/
|
||||
public Builder parameterProviderLookup(final ParameterProviderLookup parameterProviderLookup) {
|
||||
this.parameterProviderLookup = parameterProviderLookup;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the ParameterProvider configuration
|
||||
* @param parameterProviderConfiguration The ParameterProvider configuration
|
||||
* @return The builder
|
||||
*/
|
||||
public Builder parameterProviderConfiguration(final ParameterProviderConfiguration parameterProviderConfiguration) {
|
||||
this.parameterProviderConfiguration = parameterProviderConfiguration;
|
||||
return this;
|
||||
}
|
||||
|
||||
public StandardParameterContext build() {
|
||||
return new StandardParameterContext(id, name, parameterReferenceManager, parentAuthorizable, parameterProviderLookup, parameterProviderConfiguration);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,47 @@
|
|||
/*
|
||||
* 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.parameter;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
public class StandardParameterProviderConfiguration implements ParameterProviderConfiguration {
|
||||
|
||||
private final String parameterProviderId;
|
||||
private final String parameterGroupName;
|
||||
private final boolean isSynchronized;
|
||||
|
||||
public StandardParameterProviderConfiguration(final String parameterProviderId, final String parameterGroupName, final Boolean isSynchronized) {
|
||||
this.parameterProviderId = Objects.requireNonNull(parameterProviderId, "Parameter Provider ID is required");
|
||||
this.parameterGroupName = Objects.requireNonNull(parameterGroupName, "Parameter Group Name is required");
|
||||
this.isSynchronized = isSynchronized == null ? false : isSynchronized;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getParameterProviderId() {
|
||||
return parameterProviderId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getParameterGroupName() {
|
||||
return parameterGroupName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isSynchronized() {
|
||||
return isSynchronized;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* 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.parameter;
|
||||
|
||||
import org.apache.nifi.controller.NodeTypeProvider;
|
||||
import org.apache.nifi.controller.kerberos.KerberosConfig;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
|
||||
import java.io.File;
|
||||
|
||||
public class StandardParameterProviderInitializationContext implements ParameterProviderInitializationContext {
|
||||
|
||||
private final String id;
|
||||
private final String name;
|
||||
private final ComponentLog logger;
|
||||
private final KerberosConfig kerberosConfig;
|
||||
private final NodeTypeProvider nodeTypeProvider;
|
||||
|
||||
public StandardParameterProviderInitializationContext(final String id, final String name, final ComponentLog logger, final KerberosConfig kerberosConfig,
|
||||
final NodeTypeProvider nodeTypeProvider) {
|
||||
this.id = id;
|
||||
this.name = name;
|
||||
this.logger = logger;
|
||||
this.kerberosConfig = kerberosConfig;
|
||||
this.nodeTypeProvider = nodeTypeProvider;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getIdentifier() {
|
||||
return id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ComponentLog getLogger() {
|
||||
return logger;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getKerberosServicePrincipal() {
|
||||
return kerberosConfig == null ? null : kerberosConfig.getPrincipal();
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getKerberosServiceKeytab() {
|
||||
return kerberosConfig == null ? null : kerberosConfig.getKeytabLocation();
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getKerberosConfigurationFile() {
|
||||
return kerberosConfig == null ? null : kerberosConfig.getConfigFile();
|
||||
}
|
||||
|
||||
@Override
|
||||
public NodeTypeProvider getNodeTypeProvider() {
|
||||
return nodeTypeProvider;
|
||||
}
|
||||
}
|
|
@ -19,6 +19,7 @@ package org.apache.nifi.registry.flow;
|
|||
|
||||
import org.apache.nifi.authorization.user.NiFiUser;
|
||||
import org.apache.nifi.flow.ExternalControllerServiceReference;
|
||||
import org.apache.nifi.flow.ParameterProviderReference;
|
||||
import org.apache.nifi.flow.VersionedFlowCoordinates;
|
||||
import org.apache.nifi.flow.VersionedParameterContext;
|
||||
import org.apache.nifi.flow.VersionedProcessGroup;
|
||||
|
@ -181,13 +182,15 @@ public class RestBasedFlowRegistry implements FlowRegistry {
|
|||
@Override
|
||||
public VersionedFlowSnapshot registerVersionedFlowSnapshot(final VersionedFlow flow, final VersionedProcessGroup snapshot,
|
||||
final Map<String, ExternalControllerServiceReference> externalControllerServices,
|
||||
final Map<String, VersionedParameterContext> parameterContexts, final String comments,
|
||||
final Map<String, VersionedParameterContext> parameterContexts,
|
||||
final Map<String, ParameterProviderReference> parameterProviderReferences, final String comments,
|
||||
final int expectedVersion, final NiFiUser user) throws IOException, NiFiRegistryException {
|
||||
final FlowSnapshotClient snapshotClient = getFlowSnapshotClient(user);
|
||||
final VersionedFlowSnapshot versionedFlowSnapshot = new VersionedFlowSnapshot();
|
||||
versionedFlowSnapshot.setFlowContents(snapshot);
|
||||
versionedFlowSnapshot.setExternalControllerServices(externalControllerServices);
|
||||
versionedFlowSnapshot.setParameterContexts(parameterContexts);
|
||||
versionedFlowSnapshot.setParameterProviders(parameterProviderReferences);
|
||||
versionedFlowSnapshot.setFlowEncodingVersion(FLOW_ENCODING_VERSION);
|
||||
|
||||
final VersionedFlowSnapshotMetadata metadata = new VersionedFlowSnapshotMetadata();
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.nifi.connectable.Funnel;
|
|||
import org.apache.nifi.connectable.Port;
|
||||
import org.apache.nifi.controller.ComponentNode;
|
||||
import org.apache.nifi.controller.ControllerService;
|
||||
import org.apache.nifi.controller.ParameterProviderNode;
|
||||
import org.apache.nifi.controller.ProcessorNode;
|
||||
import org.apache.nifi.controller.PropertyConfiguration;
|
||||
import org.apache.nifi.controller.ReportingTaskNode;
|
||||
|
@ -42,6 +43,8 @@ import org.apache.nifi.flow.ComponentType;
|
|||
import org.apache.nifi.flow.ConnectableComponent;
|
||||
import org.apache.nifi.flow.ConnectableComponentType;
|
||||
import org.apache.nifi.flow.ControllerServiceAPI;
|
||||
import org.apache.nifi.flow.ExternalControllerServiceReference;
|
||||
import org.apache.nifi.flow.ParameterProviderReference;
|
||||
import org.apache.nifi.flow.PortType;
|
||||
import org.apache.nifi.flow.Position;
|
||||
import org.apache.nifi.flow.VersionedConnection;
|
||||
|
@ -49,6 +52,9 @@ import org.apache.nifi.flow.VersionedControllerService;
|
|||
import org.apache.nifi.flow.VersionedFlowCoordinates;
|
||||
import org.apache.nifi.flow.VersionedFunnel;
|
||||
import org.apache.nifi.flow.VersionedLabel;
|
||||
import org.apache.nifi.flow.VersionedParameter;
|
||||
import org.apache.nifi.flow.VersionedParameterContext;
|
||||
import org.apache.nifi.flow.VersionedParameterProvider;
|
||||
import org.apache.nifi.flow.VersionedPort;
|
||||
import org.apache.nifi.flow.VersionedProcessGroup;
|
||||
import org.apache.nifi.flow.VersionedProcessor;
|
||||
|
@ -65,15 +71,14 @@ import org.apache.nifi.nar.ExtensionManager;
|
|||
import org.apache.nifi.parameter.Parameter;
|
||||
import org.apache.nifi.parameter.ParameterContext;
|
||||
import org.apache.nifi.parameter.ParameterDescriptor;
|
||||
import org.apache.nifi.parameter.ParameterProvider;
|
||||
import org.apache.nifi.parameter.ParameterProviderConfiguration;
|
||||
import org.apache.nifi.parameter.ParameterReferencedControllerServiceData;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.registry.VariableDescriptor;
|
||||
import org.apache.nifi.flow.ExternalControllerServiceReference;
|
||||
import org.apache.nifi.registry.flow.FlowRegistry;
|
||||
import org.apache.nifi.registry.flow.FlowRegistryClient;
|
||||
import org.apache.nifi.registry.flow.VersionControlInformation;
|
||||
import org.apache.nifi.flow.VersionedParameter;
|
||||
import org.apache.nifi.flow.VersionedParameterContext;
|
||||
import org.apache.nifi.remote.PublicPort;
|
||||
import org.apache.nifi.remote.RemoteGroupPort;
|
||||
|
||||
|
@ -430,6 +435,25 @@ public class NiFiRegistryFlowMapper {
|
|||
return versionedTask;
|
||||
}
|
||||
|
||||
public VersionedParameterProvider mapParameterProvider(final ParameterProviderNode parameterProviderNode, final ControllerServiceProvider serviceProvider) {
|
||||
final VersionedParameterProvider versionedParameterProvider = new VersionedParameterProvider();
|
||||
versionedParameterProvider.setIdentifier(parameterProviderNode.getIdentifier());
|
||||
if (flowMappingOptions.isMapInstanceIdentifiers()) {
|
||||
versionedParameterProvider.setInstanceIdentifier(parameterProviderNode.getIdentifier());
|
||||
}
|
||||
versionedParameterProvider.setAnnotationData(parameterProviderNode.getAnnotationData());
|
||||
versionedParameterProvider.setBundle(mapBundle(parameterProviderNode.getBundleCoordinate()));
|
||||
versionedParameterProvider.setComments(parameterProviderNode.getComments());
|
||||
versionedParameterProvider.setComponentType(ComponentType.PARAMETER_PROVIDER);
|
||||
versionedParameterProvider.setName(parameterProviderNode.getName());
|
||||
|
||||
versionedParameterProvider.setProperties(mapProperties(parameterProviderNode, serviceProvider));
|
||||
versionedParameterProvider.setPropertyDescriptors(mapPropertyDescriptors(parameterProviderNode, serviceProvider, Collections.emptySet(), Collections.emptyMap()));
|
||||
versionedParameterProvider.setType(parameterProviderNode.getCanonicalClassName());
|
||||
|
||||
return versionedParameterProvider;
|
||||
}
|
||||
|
||||
public VersionedControllerService mapControllerService(final ControllerServiceNode controllerService, final ControllerServiceProvider serviceProvider, final Set<String> includedGroupIds,
|
||||
final Map<String, ExternalControllerServiceReference> externalControllerServiceReferences) {
|
||||
final VersionedControllerService versionedService = new InstantiatedVersionedControllerService(controllerService.getIdentifier(), controllerService.getProcessGroupIdentifier());
|
||||
|
@ -771,6 +795,7 @@ public class NiFiRegistryFlowMapper {
|
|||
final String versionedContextId = flowMappingOptions.getComponentIdLookup().getComponentId(Optional.empty(), parameterContext.getIdentifier());
|
||||
versionedParameterContext.setIdentifier(versionedContextId);
|
||||
versionedParameterContext.setInheritedParameterContexts(parameterContext.getInheritedParameterContextNames());
|
||||
configureParameterProvider(parameterContext, versionedParameterContext);
|
||||
|
||||
if (flowMappingOptions.isMapInstanceIdentifiers()) {
|
||||
versionedParameterContext.setInstanceIdentifier(parameterContext.getIdentifier());
|
||||
|
@ -779,31 +804,43 @@ public class NiFiRegistryFlowMapper {
|
|||
return versionedParameterContext;
|
||||
}
|
||||
|
||||
private void configureParameterProvider(final ParameterContext parameterContext, final VersionedParameterContext versionedParameterContext) {
|
||||
final ParameterProviderConfiguration parameterProviderConfiguration = parameterContext.getParameterProviderConfiguration();
|
||||
if (parameterProviderConfiguration != null) {
|
||||
versionedParameterContext.setParameterGroupName(parameterProviderConfiguration.getParameterGroupName());
|
||||
versionedParameterContext.setParameterProvider(parameterProviderConfiguration.getParameterProviderId());
|
||||
versionedParameterContext.setSynchronized(parameterProviderConfiguration.isSynchronized());
|
||||
}
|
||||
}
|
||||
|
||||
public Map<String, VersionedParameterContext> mapParameterContexts(final ProcessGroup processGroup,
|
||||
final boolean mapDescendantVersionedFlows) {
|
||||
final boolean mapDescendantVersionedFlows,
|
||||
final Map<String, ParameterProviderReference> parameterProviderReferences) {
|
||||
// cannot use a set to enforce uniqueness of parameter contexts because VersionedParameterContext in the
|
||||
// registry data model doesn't currently implement hashcode/equals based on context name
|
||||
final Map<String, VersionedParameterContext> parameterContexts = new HashMap<>();
|
||||
mapParameterContexts(processGroup, mapDescendantVersionedFlows, parameterContexts);
|
||||
mapParameterContexts(processGroup, mapDescendantVersionedFlows, parameterContexts, parameterProviderReferences);
|
||||
return parameterContexts;
|
||||
}
|
||||
|
||||
private void mapParameterContexts(final ProcessGroup processGroup, final boolean mapDescendantVersionedFlows,
|
||||
final Map<String, VersionedParameterContext> parameterContexts) {
|
||||
final Map<String, VersionedParameterContext> parameterContexts,
|
||||
final Map<String, ParameterProviderReference> parameterProviderReferences) {
|
||||
final ParameterContext parameterContext = processGroup.getParameterContext();
|
||||
if (parameterContext != null) {
|
||||
mapParameterContext(parameterContext, parameterContexts);
|
||||
mapParameterContext(parameterContext, parameterContexts, parameterProviderReferences);
|
||||
}
|
||||
|
||||
for (final ProcessGroup child : processGroup.getProcessGroups()) {
|
||||
// only include child process group parameter contexts if boolean indicator is true or process group is unversioned
|
||||
if (mapDescendantVersionedFlows || child.getVersionControlInformation() == null) {
|
||||
mapParameterContexts(child, mapDescendantVersionedFlows, parameterContexts);
|
||||
mapParameterContexts(child, mapDescendantVersionedFlows, parameterContexts, parameterProviderReferences);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void mapParameterContext(final ParameterContext parameterContext, final Map<String, VersionedParameterContext> parameterContexts) {
|
||||
private void mapParameterContext(final ParameterContext parameterContext, final Map<String, VersionedParameterContext> parameterContexts,
|
||||
final Map<String, ParameterProviderReference> parameterProviderReferences) {
|
||||
// map this process group's parameter context and add to the collection
|
||||
final Set<VersionedParameter> parameters = mapParameters(parameterContext);
|
||||
|
||||
|
@ -811,8 +848,13 @@ public class NiFiRegistryFlowMapper {
|
|||
versionedContext.setName(parameterContext.getName());
|
||||
versionedContext.setParameters(parameters);
|
||||
versionedContext.setInheritedParameterContexts(parameterContext.getInheritedParameterContextNames());
|
||||
|
||||
configureParameterProvider(parameterContext, versionedContext);
|
||||
if (versionedContext.getParameterProvider() != null) {
|
||||
parameterProviderReferences.put(versionedContext.getParameterProvider(), createParameterProviderReference(parameterContext));
|
||||
}
|
||||
for (final ParameterContext inheritedParameterContext : parameterContext.getInheritedParameterContexts()) {
|
||||
mapParameterContext(inheritedParameterContext, parameterContexts);
|
||||
mapParameterContext(inheritedParameterContext, parameterContexts, parameterProviderReferences);
|
||||
}
|
||||
|
||||
parameterContexts.put(versionedContext.getName(), versionedContext);
|
||||
|
@ -852,21 +894,36 @@ public class NiFiRegistryFlowMapper {
|
|||
return versionedParameter;
|
||||
}
|
||||
|
||||
private ParameterProviderReference createParameterProviderReference(final ParameterContext parameterContext) {
|
||||
if (parameterContext.getParameterProvider() == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final ParameterProviderReference reference = new ParameterProviderReference();
|
||||
final ParameterProvider parameterProvider = parameterContext.getParameterProvider();
|
||||
final ParameterProviderNode parameterProviderNode = parameterContext.getParameterProviderLookup().getParameterProvider(parameterProvider.getIdentifier());
|
||||
final BundleCoordinate bundleCoordinate = parameterProviderNode.getBundleCoordinate();
|
||||
|
||||
reference.setIdentifier(parameterProvider.getIdentifier());
|
||||
reference.setName(parameterProviderNode.getName());
|
||||
reference.setType(parameterProvider.getClass().getName());
|
||||
reference.setBundle(new Bundle(bundleCoordinate.getGroup(), bundleCoordinate.getId(), bundleCoordinate.getVersion()));
|
||||
|
||||
return reference;
|
||||
}
|
||||
|
||||
private VersionedParameter mapParameter(final Parameter parameter) {
|
||||
return mapParameter(parameter, parameter.getValue());
|
||||
}
|
||||
|
||||
private VersionedParameter mapParameter(final Parameter parameter, final String value) {
|
||||
if (parameter == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final ParameterDescriptor descriptor = parameter.getDescriptor();
|
||||
|
||||
final VersionedParameter versionedParameter = new VersionedParameter();
|
||||
versionedParameter.setDescription(descriptor.getDescription());
|
||||
versionedParameter.setName(descriptor.getName());
|
||||
versionedParameter.setSensitive(descriptor.isSensitive());
|
||||
versionedParameter.setProvided(parameter.isProvided());
|
||||
|
||||
final boolean mapParameterValue = flowMappingOptions.isMapSensitiveConfiguration() || !descriptor.isSensitive();
|
||||
final String parameterValue;
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
package org.apache.nifi.registry.flow.mapping;
|
||||
|
||||
import org.apache.nifi.flow.VersionedControllerService;
|
||||
import org.apache.nifi.flow.VersionedParameterProvider;
|
||||
import org.apache.nifi.flow.VersionedProcessGroup;
|
||||
import org.apache.nifi.flow.VersionedReportingTask;
|
||||
import org.apache.nifi.flow.VersionedParameterContext;
|
||||
|
@ -33,18 +34,21 @@ public class StandardComparableDataFlow implements ComparableDataFlow {
|
|||
private final Set<VersionedControllerService> controllerLevelServices;
|
||||
private final Set<VersionedReportingTask> reportingTasks;
|
||||
private final Set<VersionedParameterContext> parameterContexts;
|
||||
private final Set<VersionedParameterProvider> parameterProviders;
|
||||
|
||||
public StandardComparableDataFlow(final String name, final VersionedProcessGroup contents) {
|
||||
this(name, contents, Collections.emptySet(), Collections.emptySet(), Collections.emptySet());
|
||||
this(name, contents, Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), Collections.emptySet());
|
||||
}
|
||||
|
||||
public StandardComparableDataFlow(final String name, final VersionedProcessGroup contents, final Set<VersionedControllerService> controllerLevelServices,
|
||||
final Set<VersionedReportingTask> reportingTasks, final Set<VersionedParameterContext> parameterContexts) {
|
||||
final Set<VersionedReportingTask> reportingTasks, final Set<VersionedParameterContext> parameterContexts,
|
||||
final Set<VersionedParameterProvider> parameterProviders) {
|
||||
this.name = name;
|
||||
this.contents = contents;
|
||||
this.controllerLevelServices = controllerLevelServices == null ? Collections.emptySet() : new HashSet<>(controllerLevelServices);
|
||||
this.reportingTasks = reportingTasks == null ? Collections.emptySet() : new HashSet<>(reportingTasks);
|
||||
this.parameterContexts = parameterContexts == null ? Collections.emptySet() : new HashSet<>(parameterContexts);
|
||||
this.parameterProviders = parameterProviders == null ? Collections.emptySet() : new HashSet<>(parameterProviders);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -71,4 +75,9 @@ public class StandardComparableDataFlow implements ComparableDataFlow {
|
|||
public Set<VersionedParameterContext> getParameterContexts() {
|
||||
return parameterContexts;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<VersionedParameterProvider> getParameterProviders() {
|
||||
return parameterProviders;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -56,6 +56,7 @@ import org.apache.nifi.parameter.Parameter;
|
|||
import org.apache.nifi.parameter.ParameterContext;
|
||||
import org.apache.nifi.parameter.ParameterContextManager;
|
||||
import org.apache.nifi.parameter.ParameterDescriptor;
|
||||
import org.apache.nifi.parameter.ParameterProviderConfiguration;
|
||||
import org.apache.nifi.parameter.ParameterReferenceManager;
|
||||
import org.apache.nifi.parameter.StandardParameterContext;
|
||||
import org.apache.nifi.parameter.StandardParameterContextManager;
|
||||
|
@ -93,6 +94,7 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertThrows;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.mockito.AdditionalMatchers.or;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.anyBoolean;
|
||||
import static org.mockito.ArgumentMatchers.anyCollection;
|
||||
|
@ -101,6 +103,7 @@ import static org.mockito.ArgumentMatchers.anyMap;
|
|||
import static org.mockito.ArgumentMatchers.anySet;
|
||||
import static org.mockito.ArgumentMatchers.anyString;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.ArgumentMatchers.isNull;
|
||||
import static org.mockito.ArgumentMatchers.nullable;
|
||||
import static org.mockito.Mockito.atLeast;
|
||||
import static org.mockito.Mockito.doAnswer;
|
||||
|
@ -150,7 +153,11 @@ public class StandardVersionedComponentSynchronizerTest {
|
|||
doAnswer(invocation -> {
|
||||
final String id = invocation.getArgument(0, String.class);
|
||||
final String name = invocation.getArgument(1, String.class);
|
||||
final ParameterContext parameterContext = new StandardParameterContext(id, name, parameterReferenceManager, null);
|
||||
final ParameterContext parameterContext = new StandardParameterContext.Builder()
|
||||
.id(id)
|
||||
.name(name)
|
||||
.parameterReferenceManager(parameterReferenceManager)
|
||||
.build();
|
||||
|
||||
final Map<String, Parameter> parameterMap = invocation.getArgument(2, Map.class);
|
||||
parameterContext.setParameters(parameterMap);
|
||||
|
@ -164,7 +171,7 @@ public class StandardVersionedComponentSynchronizerTest {
|
|||
parameterContextManager.addParameterContext(parameterContext);
|
||||
|
||||
return parameterContext;
|
||||
}).when(flowManager).createParameterContext(anyString(), anyString(), anyMap(), anyList());
|
||||
}).when(flowManager).createParameterContext(anyString(), anyString(), anyMap(), anyList(), or(any(ParameterProviderConfiguration.class), isNull()));
|
||||
|
||||
final VersionedFlowSynchronizationContext context = new VersionedFlowSynchronizationContext.Builder()
|
||||
.componentIdGenerator(componentIdGenerator)
|
||||
|
|
|
@ -38,13 +38,19 @@ import static junit.framework.TestCase.assertTrue;
|
|||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertThrows;
|
||||
|
||||
public class TestStandardParameterContext {
|
||||
|
||||
@Test
|
||||
public void testUpdatesApply() {
|
||||
final ParameterReferenceManager referenceManager = new HashMapParameterReferenceManager();
|
||||
final StandardParameterContext context = new StandardParameterContext("unit-test-context", "unit-test-context", referenceManager, null);
|
||||
|
||||
final StandardParameterContext context = new StandardParameterContext.Builder()
|
||||
.id("unit-test-context")
|
||||
.name("unit-test-context")
|
||||
.parameterReferenceManager(referenceManager)
|
||||
.build();
|
||||
|
||||
final ParameterDescriptor abcDescriptor = new ParameterDescriptor.Builder().name("abc").build();
|
||||
final ParameterDescriptor xyzDescriptor = new ParameterDescriptor.Builder().name("xyz").build();
|
||||
|
@ -99,8 +105,11 @@ public class TestStandardParameterContext {
|
|||
@Test
|
||||
public void testUpdateDescription() {
|
||||
final ParameterReferenceManager referenceManager = new HashMapParameterReferenceManager();
|
||||
final StandardParameterContext context = new StandardParameterContext("unit-test-context", "unit-test-context", referenceManager, null);
|
||||
|
||||
final StandardParameterContext context = new StandardParameterContext.Builder()
|
||||
.id("unit-test-context")
|
||||
.name("unit-test-context")
|
||||
.parameterReferenceManager(referenceManager)
|
||||
.build();
|
||||
final ParameterDescriptor abcDescriptor = new ParameterDescriptor.Builder().name("abc").description("abc").build();
|
||||
|
||||
final Map<String, Parameter> parameters = new HashMap<>();
|
||||
|
@ -132,10 +141,50 @@ public class TestStandardParameterContext {
|
|||
assertNull(abcParam.getValue());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateSensitivity() {
|
||||
final ParameterReferenceManager referenceManager = new HashMapParameterReferenceManager();
|
||||
final StandardParameterContext context = new StandardParameterContext.Builder()
|
||||
.id("unit-test-context")
|
||||
.name("unit-test-context")
|
||||
.parameterReferenceManager(referenceManager)
|
||||
.build();
|
||||
final ParameterDescriptor abcDescriptor = new ParameterDescriptor.Builder().name("abc").description("abc").build();
|
||||
|
||||
final Map<String, Parameter> parameters = new HashMap<>();
|
||||
parameters.put("abc", new Parameter(abcDescriptor, "123", null, true));
|
||||
|
||||
context.setParameters(parameters);
|
||||
|
||||
Parameter abcParam = context.getParameter("abc").get();
|
||||
assertEquals(abcDescriptor, abcParam.getDescriptor());
|
||||
assertEquals("abc", abcParam.getDescriptor().getDescription());
|
||||
assertEquals("123", abcParam.getValue());
|
||||
|
||||
ParameterDescriptor updatedDescriptor = new ParameterDescriptor.Builder().name("abc").description("abc").sensitive(true).build();
|
||||
final Parameter unprovidedParam = new Parameter(updatedDescriptor, "321", null, false);
|
||||
assertThrows(IllegalStateException.class, () -> context.setParameters(Collections.singletonMap("abc", unprovidedParam)));
|
||||
|
||||
final Parameter newSensitivityParam = new Parameter(updatedDescriptor, "321", null, true);
|
||||
context.setParameters(Collections.singletonMap("abc", newSensitivityParam));
|
||||
|
||||
abcParam = context.getParameter("abc").get();
|
||||
assertEquals(abcDescriptor, abcParam.getDescriptor());
|
||||
assertTrue(abcParam.getDescriptor().isSensitive());
|
||||
|
||||
context.getParameters().keySet().forEach(pd -> {
|
||||
assertTrue(pd.isSensitive());
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testChangeDescription() {
|
||||
final ParameterReferenceManager referenceManager = new HashMapParameterReferenceManager();
|
||||
final StandardParameterContext context = new StandardParameterContext("unit-test-context", "unit-test-context", referenceManager, null);
|
||||
final StandardParameterContext context = new StandardParameterContext.Builder()
|
||||
.id("unit-test-context")
|
||||
.name("unit-test-context")
|
||||
.parameterReferenceManager(referenceManager)
|
||||
.build();
|
||||
final ParameterDescriptor xyzDescriptor = new ParameterDescriptor.Builder().name("xyz").build();
|
||||
final Map<String, Parameter> parameters = new HashMap<>();
|
||||
parameters.put("xyz", new Parameter(xyzDescriptor, "123"));
|
||||
|
@ -159,8 +208,11 @@ public class TestStandardParameterContext {
|
|||
public void testChangingSensitivity() {
|
||||
// Ensure no changes applied
|
||||
final ParameterReferenceManager referenceManager = new HashMapParameterReferenceManager();
|
||||
final StandardParameterContext context = new StandardParameterContext("unit-test-context", "unit-test-context", referenceManager, null);
|
||||
|
||||
final StandardParameterContext context = new StandardParameterContext.Builder()
|
||||
.id("unit-test-context")
|
||||
.name("unit-test-context")
|
||||
.parameterReferenceManager(referenceManager)
|
||||
.build();
|
||||
final ParameterDescriptor abcDescriptor = new ParameterDescriptor.Builder().name("abc").sensitive(true).build();
|
||||
final ParameterDescriptor xyzDescriptor = new ParameterDescriptor.Builder().name("xyz").build();
|
||||
final ParameterDescriptor fooDescriptor = new ParameterDescriptor.Builder().name("foo").description("bar").sensitive(true).build();
|
||||
|
@ -197,8 +249,7 @@ public class TestStandardParameterContext {
|
|||
@Test
|
||||
public void testChangingParameterForRunningProcessor() {
|
||||
final HashMapParameterReferenceManager referenceManager = new HashMapParameterReferenceManager();
|
||||
final StandardParameterContext context = new StandardParameterContext("unit-test-context", "unit-test-context", referenceManager, null);
|
||||
|
||||
final ParameterContext context = createStandardParameterContext(referenceManager);
|
||||
final ProcessorNode procNode = getProcessorNode("abc", referenceManager);
|
||||
|
||||
final ParameterDescriptor abcDescriptor = new ParameterDescriptor.Builder().name("abc").sensitive(true).build();
|
||||
|
@ -431,8 +482,7 @@ public class TestStandardParameterContext {
|
|||
@Test
|
||||
public void testChangingParameterForEnabledControllerService() {
|
||||
final HashMapParameterReferenceManager referenceManager = new HashMapParameterReferenceManager();
|
||||
final StandardParameterContext context = new StandardParameterContext("unit-test-context", "unit-test-context", referenceManager, null);
|
||||
|
||||
final ParameterContext context = createStandardParameterContext(referenceManager);
|
||||
final ControllerServiceNode serviceNode = Mockito.mock(ControllerServiceNode.class);
|
||||
enableControllerService(serviceNode);
|
||||
|
||||
|
@ -470,6 +520,14 @@ public class TestStandardParameterContext {
|
|||
}
|
||||
}
|
||||
|
||||
private ParameterContext createStandardParameterContext(final ParameterReferenceManager referenceManager) {
|
||||
return new StandardParameterContext.Builder()
|
||||
.id("unit-test-context")
|
||||
.name("unit-test-context")
|
||||
.parameterReferenceManager(referenceManager)
|
||||
.build();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSetParameterContexts_foundCycle() {
|
||||
final StandardParameterContextManager parameterContextLookup = new StandardParameterContextManager();
|
||||
|
@ -586,38 +644,6 @@ public class TestStandardParameterContext {
|
|||
Assert.assertEquals("d", effectiveParameters.get(grandchild).getParameterContextId());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHasEffectiveValueIfRemoved() {
|
||||
final StandardParameterContextManager parameterContextLookup = new StandardParameterContextManager();
|
||||
// Set up a hierarchy as follows:
|
||||
// a (foo, bar, baz)
|
||||
// |
|
||||
// b (foo, child)
|
||||
// |
|
||||
// c (bar, grandchild)
|
||||
//
|
||||
// foo is in a/b; bar is in a/c; baz is only in a
|
||||
final ParameterContext a = createParameterContext("a", parameterContextLookup);
|
||||
final ParameterDescriptor foo = addParameter(a, "foo", "a.foo");
|
||||
final ParameterDescriptor bar = addParameter(a, "bar", "a.bar");
|
||||
final ParameterDescriptor baz = addParameter(a, "baz", "a.baz");
|
||||
|
||||
final ParameterContext b = createParameterContext("b", parameterContextLookup);
|
||||
addParameter(b,"foo", "b.foo");
|
||||
final ParameterDescriptor child = addParameter(b, "child", "b.child");
|
||||
|
||||
final ParameterContext c = createParameterContext("c", parameterContextLookup);
|
||||
addParameter(c, "bar", "c.foo");
|
||||
addParameter(c, "grandchild", "c.child");
|
||||
|
||||
a.setInheritedParameterContexts(Arrays.asList(b));
|
||||
b.setInheritedParameterContexts(Arrays.asList(c));
|
||||
|
||||
assertTrue(a.hasEffectiveValueIfRemoved(foo));
|
||||
assertTrue(a.hasEffectiveValueIfRemoved(bar));
|
||||
assertFalse(a.hasEffectiveValueIfRemoved(baz));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetEffectiveParameters_duplicateOverride() {
|
||||
final StandardParameterContextManager parameterContextLookup = new StandardParameterContextManager();
|
||||
|
@ -765,7 +791,7 @@ public class TestStandardParameterContext {
|
|||
|
||||
private static ParameterContext createParameterContext(final String id, final ParameterContextManager parameterContextLookup,
|
||||
final ParameterReferenceManager referenceManager, final ParameterContext... children) {
|
||||
final ParameterContext parameterContext = new StandardParameterContext(id, id.toUpperCase(), referenceManager, null );
|
||||
final ParameterContext parameterContext = new StandardParameterContext.Builder().id(id).name(id.toUpperCase()).parameterReferenceManager(referenceManager).build();
|
||||
parameterContext.setInheritedParameterContexts(Arrays.asList(children));
|
||||
|
||||
parameterContextLookup.addParameterContext(parameterContext);
|
||||
|
|
|
@ -1131,17 +1131,21 @@ public abstract class AbstractComponentNode implements ComponentNode {
|
|||
|
||||
// There is an update to the parameter. We want to return the previous value of the Parameter.
|
||||
final ParameterDescriptor parameterDescriptor;
|
||||
final boolean isProvided;
|
||||
if (optionalParameter.isPresent()) {
|
||||
parameterDescriptor = optionalParameter.get().getDescriptor();
|
||||
final Parameter previousParameter = optionalParameter.get();
|
||||
parameterDescriptor = previousParameter.getDescriptor();
|
||||
isProvided = previousParameter.isProvided();
|
||||
} else {
|
||||
parameterDescriptor = new ParameterDescriptor.Builder()
|
||||
.name(parameterName)
|
||||
.description("")
|
||||
.sensitive(true)
|
||||
.build();
|
||||
isProvided = false;
|
||||
}
|
||||
|
||||
final Parameter updatedParameter = new Parameter(parameterDescriptor, parameterUpdate.getPreviousValue());
|
||||
final Parameter updatedParameter = new Parameter(parameterDescriptor, parameterUpdate.getPreviousValue(), null, isProvided);
|
||||
return Optional.of(updatedParameter);
|
||||
}
|
||||
|
||||
|
|
|
@ -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.controller;
|
||||
|
||||
import org.apache.nifi.components.ConfigVerificationResult;
|
||||
import org.apache.nifi.logging.ComponentLog;
|
||||
import org.apache.nifi.nar.ExtensionManager;
|
||||
import org.apache.nifi.parameter.ParameterContext;
|
||||
import org.apache.nifi.parameter.ParameterProvider;
|
||||
import org.apache.nifi.parameter.ParameterGroupConfiguration;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
public interface ParameterProviderNode extends ComponentNode {
|
||||
|
||||
ParameterProvider getParameterProvider();
|
||||
|
||||
void setParameterProvider(LoggableComponent<ParameterProvider> parameterProvider);
|
||||
|
||||
ConfigurationContext getConfigurationContext();
|
||||
|
||||
String getComments();
|
||||
|
||||
void setComments(String comment);
|
||||
|
||||
void verifyCanFetchParameters();
|
||||
|
||||
void fetchParameters();
|
||||
|
||||
void verifyCanApplyParameters(Collection<ParameterGroupConfiguration> parameterNames);
|
||||
|
||||
Collection<ParameterGroupConfiguration> getParameterGroupConfigurations();
|
||||
|
||||
List<ParametersApplication> getFetchedParametersToApply(Collection<ParameterGroupConfiguration> parameterGroupConfigurations);
|
||||
|
||||
void verifyCanClearState();
|
||||
|
||||
void verifyCanDelete();
|
||||
|
||||
/**
|
||||
* @return all ParameterContexts that reference this ParameterProvider
|
||||
*/
|
||||
Set<ParameterContext> getReferences();
|
||||
|
||||
/**
|
||||
* Indicates that a parameter context is now referencing this Parameter Provider
|
||||
* @param parameterContext the parameter context that references this provider
|
||||
*/
|
||||
void addReference(ParameterContext parameterContext);
|
||||
|
||||
/**
|
||||
* Indicates that a parameter context is no longer referencing this Parameter Provider
|
||||
* @param parameterContext the parameter context that no longer references this provider
|
||||
*/
|
||||
void removeReference(ParameterContext parameterContext);
|
||||
|
||||
/**
|
||||
* Verifies that the given configuration is valid for the Parameter Provider
|
||||
*
|
||||
* @param context the configuration to verify
|
||||
* @param logger a logger that can be used when performing verification
|
||||
* @param extensionManager extension manager that is used for obtaining appropriate NAR ClassLoaders
|
||||
* @return a list of results indicating whether or not the given configuration is valid
|
||||
*/
|
||||
List<ConfigVerificationResult> verifyConfiguration(ConfigurationContext context, ComponentLog logger, ExtensionManager extensionManager);
|
||||
}
|
|
@ -0,0 +1,45 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller;
|
||||
|
||||
import org.apache.nifi.parameter.Parameter;
|
||||
import org.apache.nifi.parameter.ParameterContext;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Encapsulates parameter updates that could be applied to a given ParameterContext.
|
||||
*/
|
||||
public class ParametersApplication {
|
||||
|
||||
private final ParameterContext parameterContext;
|
||||
|
||||
private final Map<String, Parameter> parameterUpdates;
|
||||
|
||||
public ParametersApplication(final ParameterContext parameterContext, final Map<String, Parameter> parameterUpdates) {
|
||||
this.parameterContext = parameterContext;
|
||||
this.parameterUpdates = parameterUpdates;
|
||||
}
|
||||
|
||||
public ParameterContext getParameterContext() {
|
||||
return parameterContext;
|
||||
}
|
||||
|
||||
public Map<String, Parameter> getParameterUpdates() {
|
||||
return parameterUpdates;
|
||||
}
|
||||
}
|
|
@ -19,6 +19,7 @@ package org.apache.nifi.controller;
|
|||
import org.apache.nifi.bundle.BundleCoordinate;
|
||||
import org.apache.nifi.controller.exception.ControllerServiceInstantiationException;
|
||||
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
|
||||
import org.apache.nifi.controller.parameter.ParameterProviderInstantiationException;
|
||||
import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
|
||||
import org.apache.nifi.controller.service.ControllerServiceNode;
|
||||
|
||||
|
@ -66,4 +67,16 @@ public interface ReloadComponent {
|
|||
void reload(ReportingTaskNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls)
|
||||
throws ReportingTaskInstantiationException;
|
||||
|
||||
/**
|
||||
* Changes the underlying ParameterProvider held by the node to an instance of the new type.
|
||||
*
|
||||
* @param existingNode the ParameterProvider being updated
|
||||
* @param newType the fully qualified class name of the new type
|
||||
* @param bundleCoordinate the bundle coordinate of the new type
|
||||
* @param additionalUrls additional URLs to be added to the instance class loader of the new component
|
||||
* @throws ParameterProviderInstantiationException if unable to create an instance of the new type
|
||||
*/
|
||||
void reload(ParameterProviderNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls)
|
||||
throws ParameterProviderInstantiationException;
|
||||
|
||||
}
|
||||
|
|
|
@ -21,10 +21,12 @@ import org.apache.nifi.connectable.Connectable;
|
|||
import org.apache.nifi.connectable.Connection;
|
||||
import org.apache.nifi.connectable.Funnel;
|
||||
import org.apache.nifi.connectable.Port;
|
||||
import org.apache.nifi.controller.ParameterProviderNode;
|
||||
import org.apache.nifi.controller.ProcessorNode;
|
||||
import org.apache.nifi.controller.ReportingTaskNode;
|
||||
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
|
||||
import org.apache.nifi.controller.label.Label;
|
||||
import org.apache.nifi.controller.parameter.ParameterProviderLookup;
|
||||
import org.apache.nifi.controller.service.ControllerServiceNode;
|
||||
import org.apache.nifi.flowfile.FlowFilePrioritizer;
|
||||
import org.apache.nifi.groups.ProcessGroup;
|
||||
|
@ -32,6 +34,7 @@ import org.apache.nifi.groups.RemoteProcessGroup;
|
|||
import org.apache.nifi.parameter.Parameter;
|
||||
import org.apache.nifi.parameter.ParameterContext;
|
||||
import org.apache.nifi.parameter.ParameterContextManager;
|
||||
import org.apache.nifi.parameter.ParameterProviderConfiguration;
|
||||
import org.apache.nifi.web.api.dto.FlowSnippetDTO;
|
||||
|
||||
import java.net.URL;
|
||||
|
@ -42,7 +45,7 @@ import java.util.Optional;
|
|||
import java.util.Set;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
public interface FlowManager {
|
||||
public interface FlowManager extends ParameterProviderLookup {
|
||||
String ROOT_GROUP_ID_ALIAS = "root";
|
||||
String DEFAULT_ROOT_GROUP_NAME = "NiFi Flow";
|
||||
|
||||
|
@ -312,7 +315,14 @@ public interface FlowManager {
|
|||
|
||||
Set<ReportingTaskNode> getAllReportingTasks();
|
||||
|
||||
ParameterProviderNode createParameterProvider(String type, String id, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls, boolean firstTimeAdded,
|
||||
boolean registerLogObserver);
|
||||
|
||||
ParameterProviderNode createParameterProvider(String type, String id, BundleCoordinate bundleCoordinate, boolean firstTimeAdded);
|
||||
|
||||
void removeParameterProvider(ParameterProviderNode parameterProvider);
|
||||
|
||||
Set<ParameterProviderNode> getAllParameterProviders();
|
||||
|
||||
Set<ControllerServiceNode> getAllControllerServices();
|
||||
|
||||
|
@ -321,7 +331,6 @@ public interface FlowManager {
|
|||
ControllerServiceNode createControllerService(String type, String id, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls, boolean firstTimeAdded,
|
||||
boolean registerLogObserver, String classloaderIsolationKey);
|
||||
|
||||
|
||||
Set<ControllerServiceNode> getRootControllerServices();
|
||||
|
||||
void addRootControllerService(ControllerServiceNode serviceNode);
|
||||
|
@ -343,11 +352,13 @@ public interface FlowManager {
|
|||
* @param parameters The Parameters
|
||||
* @param inheritedContextIds The identifiers of any Parameter Contexts that the newly created Parameter Context should inherit from. The order of the identifiers in the List determines the
|
||||
* order in which parameters with conflicting names are resolved. I.e., the Parameter Context whose ID comes first in the List is preferred.
|
||||
* @param parameterProviderConfiguration Optional configuration for a ParameterProvider
|
||||
* @return The created ParameterContext
|
||||
* @throws IllegalStateException If <code>parameterContexts</code> is not empty and this method is called without being wrapped
|
||||
* by {@link FlowManager#withParameterContextResolution(Runnable)}
|
||||
*/
|
||||
ParameterContext createParameterContext(String id, String name, Map<String, Parameter> parameters, List<String> inheritedContextIds);
|
||||
ParameterContext createParameterContext(String id, String name, Map<String, Parameter> parameters, List<String> inheritedContextIds,
|
||||
ParameterProviderConfiguration parameterProviderConfiguration);
|
||||
|
||||
/**
|
||||
* Performs the given ParameterContext-related action, and then resolves all inherited ParameterContext references.
|
||||
|
@ -370,7 +381,8 @@ public interface FlowManager {
|
|||
ParameterContextManager getParameterContextManager();
|
||||
|
||||
/**
|
||||
* @return the number of each type of component (Processor, Controller Service, Process Group, Funnel, Input Port, Output Port, Reporting Task, Remote Process Group)
|
||||
* @return the number of each type of component (Processor, Controller Service, Process Group, Funnel, Input Port, Output Port,
|
||||
* Parameter Provider, Reporting Task, Remote Process Group)
|
||||
*/
|
||||
Map<String, Integer> getComponentCounts();
|
||||
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
package org.apache.nifi.controller.flow;
|
||||
|
||||
import org.apache.nifi.flow.VersionedControllerService;
|
||||
import org.apache.nifi.flow.VersionedParameterProvider;
|
||||
import org.apache.nifi.flow.VersionedProcessGroup;
|
||||
import org.apache.nifi.flow.VersionedReportingTask;
|
||||
import org.apache.nifi.flow.VersionedParameterContext;
|
||||
|
@ -30,6 +31,7 @@ public class VersionedDataflow {
|
|||
private int maxTimerDrivenThreadCount;
|
||||
private List<VersionedRegistry> registries;
|
||||
private List<VersionedParameterContext> parameterContexts;
|
||||
private List<VersionedParameterProvider> parameterProviders;
|
||||
private List<VersionedControllerService> controllerServices;
|
||||
private List<VersionedReportingTask> reportingTasks;
|
||||
private Set<VersionedTemplate> templates;
|
||||
|
@ -83,6 +85,14 @@ public class VersionedDataflow {
|
|||
this.reportingTasks = reportingTasks;
|
||||
}
|
||||
|
||||
public List<VersionedParameterProvider> getParameterProviders() {
|
||||
return parameterProviders;
|
||||
}
|
||||
|
||||
public void setParameterProviders(final List<VersionedParameterProvider> parameterProviders) {
|
||||
this.parameterProviders = parameterProviders;
|
||||
}
|
||||
|
||||
public VersionedProcessGroup getRootGroup() {
|
||||
return rootGroup;
|
||||
}
|
||||
|
|
|
@ -0,0 +1,29 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.parameter;
|
||||
|
||||
public class ParameterProviderInstantiationException extends Exception {
|
||||
|
||||
public ParameterProviderInstantiationException(final String className, final Throwable t) {
|
||||
super(className, t);
|
||||
}
|
||||
|
||||
public ParameterProviderInstantiationException(final String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,33 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.controller.parameter;
|
||||
|
||||
import org.apache.nifi.controller.ParameterProviderNode;
|
||||
|
||||
/**
|
||||
* A ParameterProviderLookup is able to look up <code>ParameterProviderNode</code>s by id.
|
||||
*/
|
||||
public interface ParameterProviderLookup {
|
||||
|
||||
/**
|
||||
* @param identifier of node
|
||||
* @return the parameter provider that has the given identifier, or
|
||||
* <code>null</code> if no parameter provider exists with that ID
|
||||
*/
|
||||
ParameterProviderNode getParameterProvider(String identifier);
|
||||
|
||||
}
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.nifi.parameter;
|
||||
|
||||
import org.apache.nifi.authorization.resource.ComponentAuthorizable;
|
||||
import org.apache.nifi.controller.parameter.ParameterProviderLookup;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -78,18 +79,6 @@ public interface ParameterContext extends ParameterLookup, ComponentAuthorizable
|
|||
*/
|
||||
Optional<Parameter> getParameter(ParameterDescriptor parameterDescriptor);
|
||||
|
||||
/**
|
||||
* Checks whether this ParameterContext would still have an effective value for the given parameter if the
|
||||
* parameter was removed from this or any inherited parameter context, no matter how indirect. This allows
|
||||
* the ParameterContext to be checked for validity: if it will still have an effective value, the parameter
|
||||
* can be safely removed.
|
||||
*
|
||||
* @param parameterDescriptor parameter descriptor to check
|
||||
* @return True if, when the parameter is removed, this ParameterContext would still have an effective value
|
||||
* for the parameter.
|
||||
*/
|
||||
boolean hasEffectiveValueIfRemoved(ParameterDescriptor parameterDescriptor);
|
||||
|
||||
/**
|
||||
* Returns the Map of all Parameters in this context (not in any inherited ParameterContexts). Note that the Map that
|
||||
* is returned may either be immutable or may be a defensive copy but modifying the Map that is returned will have
|
||||
|
@ -110,14 +99,14 @@ public interface ParameterContext extends ParameterLookup, ComponentAuthorizable
|
|||
Map<ParameterDescriptor, Parameter> getEffectiveParameters();
|
||||
|
||||
/**
|
||||
* Returns the resulting map of effective parameter updates if the given parameter updates and inherited parameter contexts were to be applied.
|
||||
* This allows potential changes to be detected before actually applying the parameter updates.
|
||||
* @param parameterUpdates A map from parameter name to updated parameter (null if removal is desired)
|
||||
* @param inheritedParameterContexts An ordered list of parameter contexts to inherit from
|
||||
* @return The effective map of parameter updates that would result if these changes were applied. This includes only parameters that would
|
||||
* be effectively updated or removed, and is mapped by parameter name
|
||||
* Returns a map from parameter name to Parameter, representing all parameters that would be effectively
|
||||
* updated if the provided configuration was applied. Only parameters that would be effectively updated or added are
|
||||
* included in the map. A null value for a Parameter represents an effective deletion of that parameter name.
|
||||
* @param parameters A proposed map from parameter name to Parameter (if Parameter is null, this represents a deletion)
|
||||
* @param inheritedParameterContexts A proposed list of inherited parameter contexts
|
||||
* @return A map of effective parameter updates
|
||||
*/
|
||||
Map<String, Parameter> getEffectiveParameterUpdates(Map<String, Parameter> parameterUpdates, List<ParameterContext> inheritedParameterContexts);
|
||||
Map<String, Parameter> getEffectiveParameterUpdates(final Map<String, Parameter> parameters, final List<ParameterContext> inheritedParameterContexts);
|
||||
|
||||
/**
|
||||
* Returns the ParameterReferenceManager that is associated with this ParameterContext
|
||||
|
@ -125,6 +114,29 @@ public interface ParameterContext extends ParameterLookup, ComponentAuthorizable
|
|||
*/
|
||||
ParameterReferenceManager getParameterReferenceManager();
|
||||
|
||||
/**
|
||||
* Returns the ParameterProviderLookup that is associated with this ParameterContext
|
||||
* @return the ParameterProviderLookup that is associated with this ParameterContext
|
||||
*/
|
||||
ParameterProviderLookup getParameterProviderLookup();
|
||||
|
||||
/**
|
||||
*
|
||||
* @return The {@link ParameterProvider}, or null if none is set. In the latter case, Parameters are set manually.
|
||||
*/
|
||||
ParameterProvider getParameterProvider();
|
||||
|
||||
/**
|
||||
* @return The configuration for the ParameterProvider, or null if none is configured
|
||||
*/
|
||||
ParameterProviderConfiguration getParameterProviderConfiguration();
|
||||
|
||||
/**
|
||||
* Configures a {@link ParameterProvider} that will be used to provide Parameters.
|
||||
* @param parameterProviderConfiguration The configuration for the ParameterProvider
|
||||
*/
|
||||
void configureParameterProvider(ParameterProviderConfiguration parameterProviderConfiguration);
|
||||
|
||||
/**
|
||||
* Verifies whether the parameter context can be updated with the provided parameters and inherited parameter contexts.
|
||||
* @param parameterUpdates A map from parameter name to updated parameter (null if removal is desired)
|
||||
|
|
|
@ -0,0 +1,39 @@
|
|||
/*
|
||||
* 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.parameter;
|
||||
|
||||
public interface ParameterProviderConfiguration {
|
||||
|
||||
/**
|
||||
*
|
||||
* @return The identifier of the {@link ParameterProvider} that provides parameters to the Parameter Context.
|
||||
*/
|
||||
String getParameterProviderId();
|
||||
|
||||
/**
|
||||
*
|
||||
* @return The name of the {@link ParameterGroup} fetched by the ParameterProvider that maps to the Parameter Context.
|
||||
*/
|
||||
String getParameterGroupName();
|
||||
|
||||
/**
|
||||
*
|
||||
* @return If true, indicates that the ParameterContext should receive updates from the ParameterProvider when its
|
||||
* parameters are fetched.
|
||||
*/
|
||||
boolean isSynchronized();
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue