NIFI-8650 Added Flow Analysis Rules
- Added new extensible Component Type: FlowAnalysisRule - Added DisallowComponentType Rule implementation - Flow Analysis Rules can be managed from the UI under Controller Settings -> Flow Analysis Rules - Flow Analysis Rules can be set up with an enforcement policy of WARN or ENFORCE - Flow Analysis Rules can evaluate an individual Component or a Process Group This closes #7191 Signed-off-by: David Handermann <exceptionfactory@apache.org>
|
@ -31,6 +31,7 @@ public enum Component {
|
||||||
Connection,
|
Connection,
|
||||||
ControllerService,
|
ControllerService,
|
||||||
ReportingTask,
|
ReportingTask,
|
||||||
|
FlowAnalysisRule,
|
||||||
FlowRegistryClient,
|
FlowRegistryClient,
|
||||||
ParameterContext,
|
ParameterContext,
|
||||||
ParameterProvider,
|
ParameterProvider,
|
||||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.nifi.components.state.StateManager;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <p>
|
* <p>
|
||||||
* Annotation that a Processor, ReportingTask, ParameterProvider, or Controller Service can use to indicate
|
* Annotation that a Processor, ReportingTask, FlowAnalysisRule, ParameterProvider, or Controller Service can use to indicate
|
||||||
* that the component makes use of the {@link StateManager}. This annotation provides the
|
* 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
|
* 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
|
* understand what is shown to them and know what they are clearing should they choose to
|
||||||
|
|
|
@ -26,7 +26,8 @@ import java.lang.annotation.Target;
|
||||||
/**
|
/**
|
||||||
* Annotation that may be placed on a {@link org.apache.nifi.processor.Processor Processor},
|
* Annotation that may be placed on a {@link org.apache.nifi.processor.Processor Processor},
|
||||||
* {@link org.apache.nifi.controller.ControllerService ControllerService},
|
* {@link org.apache.nifi.controller.ControllerService ControllerService},
|
||||||
* {@link org.apache.nifi.parameter.ParameterProvider ParameterProvider}, or
|
* {@link org.apache.nifi.parameter.ParameterProvider ParameterProvider},
|
||||||
|
* {@link org.apache.nifi.flowanalysis.FlowAnalysisRule FlowAnalysisRule}, or
|
||||||
* {@link org.apache.nifi.reporting.ReportingTask ReportingTask} allowing for a
|
* {@link org.apache.nifi.reporting.ReportingTask ReportingTask} allowing for a
|
||||||
* description to be provided. This description can be provided to a user in
|
* description to be provided. This description can be provided to a user in
|
||||||
* logs, UI, etc.
|
* logs, UI, etc.
|
||||||
|
|
|
@ -27,7 +27,8 @@ import org.apache.nifi.components.ConfigurableComponent;
|
||||||
/**
|
/**
|
||||||
* Annotation that may be placed on a null {@link org.apache.nifi.processor.Processor Processor},
|
* Annotation that may be placed on a null {@link org.apache.nifi.processor.Processor Processor},
|
||||||
* {@link org.apache.nifi.controller.ControllerService ControllerService},
|
* {@link org.apache.nifi.controller.ControllerService ControllerService},
|
||||||
* {@link org.apache.nifi.parameter.ParameterProvider ParameterProvider}, or
|
* {@link org.apache.nifi.parameter.ParameterProvider ParameterProvider},
|
||||||
|
* {@link org.apache.nifi.flowanalysis.FlowAnalysisRule FlowAnalysisRule}, or
|
||||||
* {@link org.apache.nifi.reporting.ReportingTask ReportingTask} that indicates
|
* {@link org.apache.nifi.reporting.ReportingTask ReportingTask} that indicates
|
||||||
* this component is related to the components listed.
|
* this component is related to the components listed.
|
||||||
*
|
*
|
||||||
|
|
|
@ -26,7 +26,8 @@ import java.lang.annotation.Target;
|
||||||
/**
|
/**
|
||||||
* Annotation that can be applied to a {@link org.apache.nifi.processor.Processor Processor},
|
* Annotation that can be applied to a {@link org.apache.nifi.processor.Processor Processor},
|
||||||
* {@link org.apache.nifi.controller.ControllerService ControllerService},
|
* {@link org.apache.nifi.controller.ControllerService ControllerService},
|
||||||
* {@link org.apache.nifi.parameter.ParameterProvider ParameterProvider}, or
|
* {@link org.apache.nifi.parameter.ParameterProvider ParameterProvider},
|
||||||
|
* {@link org.apache.nifi.flowanalysis.FlowAnalysisRule FlowAnalysisRule}, or
|
||||||
* {@link org.apache.nifi.reporting.ReportingTask ReportingTask} in order to
|
* {@link org.apache.nifi.reporting.ReportingTask ReportingTask} in order to
|
||||||
* associate tags (keywords) with the component. These tags do not affect the
|
* 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
|
* component in any way but serve as additional documentation and can be used to
|
||||||
|
|
|
@ -27,8 +27,9 @@ import java.lang.annotation.Target;
|
||||||
* <p>
|
* <p>
|
||||||
* Marker annotation a {@link org.apache.nifi.processor.Processor Processor},
|
* Marker annotation a {@link org.apache.nifi.processor.Processor Processor},
|
||||||
* {@link org.apache.nifi.controller.ControllerService ControllerService},
|
* {@link org.apache.nifi.controller.ControllerService ControllerService},
|
||||||
* {@link org.apache.nifi.registry.flow.FlowRegistryClient FlowRegistryClient}
|
* {@link org.apache.nifi.registry.flow.FlowRegistryClient FlowRegistryClient},
|
||||||
* {@link org.apache.nifi.parameter.ParameterProvider ParameterProvider}, or
|
* {@link org.apache.nifi.parameter.ParameterProvider ParameterProvider},
|
||||||
|
* {@link org.apache.nifi.flowanalysis.FlowAnalysisRule FlowAnalysisRule}, or
|
||||||
* {@link org.apache.nifi.reporting.ReportingTask ReportingTask} implementation
|
* {@link org.apache.nifi.reporting.ReportingTask ReportingTask} implementation
|
||||||
* can use to indicate a method should be called whenever the component is added
|
* 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
|
* to the flow. This method will be called once for the entire life of a
|
||||||
|
|
|
@ -29,8 +29,9 @@ import org.apache.nifi.processor.ProcessContext;
|
||||||
* <p>
|
* <p>
|
||||||
* Marker annotation a {@link org.apache.nifi.processor.Processor Processor},
|
* Marker annotation a {@link org.apache.nifi.processor.Processor Processor},
|
||||||
* {@link org.apache.nifi.controller.ControllerService ControllerService},
|
* {@link org.apache.nifi.controller.ControllerService ControllerService},
|
||||||
* {@link org.apache.nifi.registry.flow.FlowRegistryClient FlowRegistryClient}
|
* {@link org.apache.nifi.registry.flow.FlowRegistryClient FlowRegistryClient},
|
||||||
* {@link org.apache.nifi.parameter.ParameterProvider ParameterProvider}, or
|
* {@link org.apache.nifi.parameter.ParameterProvider ParameterProvider},
|
||||||
|
* {@link org.apache.nifi.flowanalysis.FlowAnalysisRule FlowAnalysisRule}, or
|
||||||
* {@link org.apache.nifi.reporting.ReportingTask ReportingTask} implementation
|
* {@link org.apache.nifi.reporting.ReportingTask ReportingTask} implementation
|
||||||
* can use to indicate a method should be called whenever the component is
|
* 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
|
* removed from the flow. This method will be called once for the entire life of
|
||||||
|
|
|
@ -26,11 +26,12 @@ import org.apache.nifi.processor.ProcessSessionFactory;
|
||||||
import org.apache.nifi.processor.Processor;
|
import org.apache.nifi.processor.Processor;
|
||||||
import org.apache.nifi.reporting.InitializationException;
|
import org.apache.nifi.reporting.InitializationException;
|
||||||
import org.apache.nifi.reporting.ReportingTask;
|
import org.apache.nifi.reporting.ReportingTask;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRule;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <p>
|
* <p>
|
||||||
* This interface provides a mechanism for creating services that are shared
|
* This interface provides a mechanism for creating services that are shared
|
||||||
* among all {@link Processor}s, {@link ReportingTask}s, {@link ParameterProvider}s and other
|
* among all {@link Processor}s, {@link ReportingTask}s, {@link FlowAnalysisRule}s, {@link ParameterProvider}s and other
|
||||||
* {@code ControllerService}s.
|
* {@code ControllerService}s.
|
||||||
* </p>
|
* </p>
|
||||||
*
|
*
|
||||||
|
|
|
@ -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;
|
||||||
|
|
||||||
|
import org.apache.nifi.flow.VersionedControllerService;
|
||||||
|
|
||||||
|
public interface VersionedControllerServiceLookup {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param serviceIdentifier of controller service
|
||||||
|
* @return the VersionedControllerService with the given
|
||||||
|
* identifier
|
||||||
|
*/
|
||||||
|
VersionedControllerService getVersionedControllerService(String serviceIdentifier);
|
||||||
|
}
|
|
@ -44,9 +44,11 @@ import org.apache.nifi.components.ConfigurableComponent;
|
||||||
import org.apache.nifi.components.PropertyDescriptor;
|
import org.apache.nifi.components.PropertyDescriptor;
|
||||||
import org.apache.nifi.controller.ControllerService;
|
import org.apache.nifi.controller.ControllerService;
|
||||||
import org.apache.nifi.documentation.init.DocumentationControllerServiceInitializationContext;
|
import org.apache.nifi.documentation.init.DocumentationControllerServiceInitializationContext;
|
||||||
|
import org.apache.nifi.documentation.init.DocumentationFlowAnalysisRuleInitializationContext;
|
||||||
import org.apache.nifi.documentation.init.DocumentationParameterProviderInitializationContext;
|
import org.apache.nifi.documentation.init.DocumentationParameterProviderInitializationContext;
|
||||||
import org.apache.nifi.documentation.init.DocumentationProcessorInitializationContext;
|
import org.apache.nifi.documentation.init.DocumentationProcessorInitializationContext;
|
||||||
import org.apache.nifi.documentation.init.DocumentationReportingInitializationContext;
|
import org.apache.nifi.documentation.init.DocumentationReportingInitializationContext;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRule;
|
||||||
import org.apache.nifi.parameter.ParameterProvider;
|
import org.apache.nifi.parameter.ParameterProvider;
|
||||||
import org.apache.nifi.processor.Processor;
|
import org.apache.nifi.processor.Processor;
|
||||||
import org.apache.nifi.processor.Relationship;
|
import org.apache.nifi.processor.Relationship;
|
||||||
|
@ -87,6 +89,8 @@ public abstract class AbstractDocumentationWriter implements ExtensionDocumentat
|
||||||
initialize((ControllerService) component);
|
initialize((ControllerService) component);
|
||||||
} else if (component instanceof ReportingTask) {
|
} else if (component instanceof ReportingTask) {
|
||||||
initialize((ReportingTask) component);
|
initialize((ReportingTask) component);
|
||||||
|
} else if (component instanceof FlowAnalysisRule) {
|
||||||
|
initialize((FlowAnalysisRule) component);
|
||||||
} else if (component instanceof ParameterProvider) {
|
} else if (component instanceof ParameterProvider) {
|
||||||
initialize((ParameterProvider) component);
|
initialize((ParameterProvider) component);
|
||||||
}
|
}
|
||||||
|
@ -107,6 +111,10 @@ public abstract class AbstractDocumentationWriter implements ExtensionDocumentat
|
||||||
reportingTask.initialize(new DocumentationReportingInitializationContext());
|
reportingTask.initialize(new DocumentationReportingInitializationContext());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected void initialize(final FlowAnalysisRule flowAnalysisRule) throws InitializationException {
|
||||||
|
flowAnalysisRule.initialize(new DocumentationFlowAnalysisRuleInitializationContext());
|
||||||
|
}
|
||||||
|
|
||||||
protected void initialize(final ParameterProvider parameterProvider) throws InitializationException {
|
protected void initialize(final ParameterProvider parameterProvider) throws InitializationException {
|
||||||
parameterProvider.initialize(new DocumentationParameterProviderInitializationContext());
|
parameterProvider.initialize(new DocumentationParameterProviderInitializationContext());
|
||||||
}
|
}
|
||||||
|
@ -260,6 +268,9 @@ public abstract class AbstractDocumentationWriter implements ExtensionDocumentat
|
||||||
if (component instanceof ReportingTask) {
|
if (component instanceof ReportingTask) {
|
||||||
return ExtensionType.REPORTING_TASK;
|
return ExtensionType.REPORTING_TASK;
|
||||||
}
|
}
|
||||||
|
if (component instanceof ReportingTask) {
|
||||||
|
return ExtensionType.FLOW_ANALYSIS_RULE;
|
||||||
|
}
|
||||||
if (component instanceof ParameterProvider) {
|
if (component instanceof ParameterProvider) {
|
||||||
return ExtensionType.PARAMETER_PROVIDER;
|
return ExtensionType.PARAMETER_PROVIDER;
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,5 +23,7 @@ public enum ExtensionType {
|
||||||
|
|
||||||
REPORTING_TASK,
|
REPORTING_TASK,
|
||||||
|
|
||||||
|
FLOW_ANALYSIS_RULE,
|
||||||
|
|
||||||
PARAMETER_PROVIDER;
|
PARAMETER_PROVIDER;
|
||||||
}
|
}
|
||||||
|
|
|
@ -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.documentation.init;
|
||||||
|
|
||||||
|
import org.apache.nifi.controller.NodeTypeProvider;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRuleInitializationContext;
|
||||||
|
import org.apache.nifi.logging.ComponentLog;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
|
import java.util.UUID;
|
||||||
|
|
||||||
|
public class DocumentationFlowAnalysisRuleInitializationContext implements FlowAnalysisRuleInitializationContext {
|
||||||
|
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 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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -31,6 +31,7 @@ public enum ComponentType {
|
||||||
LABEL("Label"),
|
LABEL("Label"),
|
||||||
CONTROLLER_SERVICE("Controller Service"),
|
CONTROLLER_SERVICE("Controller Service"),
|
||||||
REPORTING_TASK("Reporting Task"),
|
REPORTING_TASK("Reporting Task"),
|
||||||
|
FLOW_ANALYSIS_RULE("Flow Analysis Rule"),
|
||||||
PARAMETER_CONTEXT("Parameter Context"),
|
PARAMETER_CONTEXT("Parameter Context"),
|
||||||
PARAMETER_PROVIDER("Parameter Provider"),
|
PARAMETER_PROVIDER("Parameter Provider"),
|
||||||
TEMPLATE("Template"),
|
TEMPLATE("Template"),
|
||||||
|
|
|
@ -25,7 +25,7 @@ public interface VersionedConfigurableComponent {
|
||||||
|
|
||||||
Map<String, VersionedPropertyDescriptor> getPropertyDescriptors();
|
Map<String, VersionedPropertyDescriptor> getPropertyDescriptors();
|
||||||
|
|
||||||
void setPropertyDescriptors(Map<String,VersionedPropertyDescriptor> propertyDescriptors);
|
void setPropertyDescriptors(Map<String, VersionedPropertyDescriptor> propertyDescriptors);
|
||||||
|
|
||||||
Map<String,String> getProperties();
|
Map<String,String> getProperties();
|
||||||
|
|
||||||
|
|
|
@ -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.flow;
|
||||||
|
|
||||||
|
import io.swagger.annotations.ApiModelProperty;
|
||||||
|
import org.apache.nifi.flowanalysis.EnforcementPolicy;
|
||||||
|
|
||||||
|
public class VersionedFlowAnalysisRule extends VersionedConfigurableExtension {
|
||||||
|
|
||||||
|
private ScheduledState scheduledState;
|
||||||
|
private EnforcementPolicy enforcementPolicy;
|
||||||
|
|
||||||
|
@ApiModelProperty("How to handle violations.")
|
||||||
|
public EnforcementPolicy getEnforcementPolicy() {
|
||||||
|
return enforcementPolicy;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setEnforcementPolicy(EnforcementPolicy enforcementPolicy) {
|
||||||
|
this.enforcementPolicy = enforcementPolicy;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ComponentType getComponentType() {
|
||||||
|
return ComponentType.FLOW_ANALYSIS_RULE;
|
||||||
|
}
|
||||||
|
|
||||||
|
@ApiModelProperty("Indicates the scheduled state for the flow analysis rule")
|
||||||
|
public ScheduledState getScheduledState() {
|
||||||
|
return scheduledState;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setScheduledState(final ScheduledState scheduledState) {
|
||||||
|
this.scheduledState = scheduledState;
|
||||||
|
}
|
||||||
|
}
|
|
@ -24,6 +24,7 @@ public class VersionedPropertyDescriptor {
|
||||||
private String displayName;
|
private String displayName;
|
||||||
private boolean identifiesControllerService;
|
private boolean identifiesControllerService;
|
||||||
private boolean sensitive;
|
private boolean sensitive;
|
||||||
|
private boolean dynamic;
|
||||||
private VersionedResourceDefinition resourceDefinition;
|
private VersionedResourceDefinition resourceDefinition;
|
||||||
|
|
||||||
@ApiModelProperty("The name of the property")
|
@ApiModelProperty("The name of the property")
|
||||||
|
@ -62,6 +63,15 @@ public class VersionedPropertyDescriptor {
|
||||||
this.sensitive = sensitive;
|
this.sensitive = sensitive;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ApiModelProperty("Whether or not the property is user-defined")
|
||||||
|
public boolean isDynamic() {
|
||||||
|
return dynamic;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setDynamic(boolean dynamic) {
|
||||||
|
this.dynamic = dynamic;
|
||||||
|
}
|
||||||
|
|
||||||
@ApiModelProperty("Returns the Resource Definition that defines which type(s) of resource(s) this property references, if any")
|
@ApiModelProperty("Returns the Resource Definition that defines which type(s) of resource(s) this property references, if any")
|
||||||
public VersionedResourceDefinition getResourceDefinition() {
|
public VersionedResourceDefinition getResourceDefinition() {
|
||||||
return resourceDefinition;
|
return resourceDefinition;
|
||||||
|
|
|
@ -0,0 +1,67 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.nifi.flowanalysis;
|
||||||
|
|
||||||
|
import java.util.StringJoiner;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Abstract class holding information about a {@link FlowAnalysisRule} violation.
|
||||||
|
*/
|
||||||
|
public abstract class AbstractAnalysisResult {
|
||||||
|
protected final String issueId;
|
||||||
|
protected final String message;
|
||||||
|
protected final String explanation;
|
||||||
|
|
||||||
|
protected AbstractAnalysisResult(final String issueId, final String message, final String explanation) {
|
||||||
|
this.issueId = issueId;
|
||||||
|
this.message = message;
|
||||||
|
this.explanation = explanation;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return A rule-defined id that corresponds to a unique type of issue recognized by the rule.
|
||||||
|
* Newer analysis runs may produce a result with the same issueId in which case the old one will
|
||||||
|
* be overwritten (or recreated if it is the same in other aspects as well).
|
||||||
|
* However, if the previous result was disabled the new one will be disabled as well.
|
||||||
|
*/
|
||||||
|
public String getIssueId() {
|
||||||
|
return issueId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the rule violation message
|
||||||
|
*/
|
||||||
|
public String getMessage() {
|
||||||
|
return message;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a detailed explanation of the nature of the violation
|
||||||
|
*/
|
||||||
|
public String getExplanation() {
|
||||||
|
return explanation;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return new StringJoiner(", ", this.getClass().getSimpleName() + "[", "]")
|
||||||
|
.add("issueId='" + issueId + "'")
|
||||||
|
.add("message='" + message + "'")
|
||||||
|
.add("explanation='" + explanation + "'")
|
||||||
|
.toString();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,53 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.nifi.flowanalysis;
|
||||||
|
|
||||||
|
import org.apache.nifi.components.AbstractConfigurableComponent;
|
||||||
|
import org.apache.nifi.logging.ComponentLog;
|
||||||
|
import org.apache.nifi.reporting.InitializationException;
|
||||||
|
|
||||||
|
public abstract class AbstractFlowAnalysisRule extends AbstractConfigurableComponent implements FlowAnalysisRule {
|
||||||
|
private String identifier;
|
||||||
|
private String description;
|
||||||
|
|
||||||
|
private ComponentLog logger;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void initialize(FlowAnalysisRuleInitializationContext context) throws InitializationException {
|
||||||
|
identifier = context.getIdentifier();
|
||||||
|
description = getClass().getSimpleName() + "[id=" + identifier + "]";
|
||||||
|
logger = context.getLogger();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getIdentifier() {
|
||||||
|
return identifier;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return description;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the logger that has been provided to the component by the
|
||||||
|
* framework in its initialize method
|
||||||
|
*/
|
||||||
|
protected ComponentLog getLogger() {
|
||||||
|
return logger;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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.flowanalysis;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Holds information about a component violating a {@link FlowAnalysisRule}
|
||||||
|
*/
|
||||||
|
public class ComponentAnalysisResult extends AbstractAnalysisResult {
|
||||||
|
/**
|
||||||
|
* @param issueId A rule-defined id that corresponds to a unique type of issue recognized by the rule.
|
||||||
|
* Newer analysis runs may produce a result with the same issueId in which case the old one will
|
||||||
|
* be overwritten (or recreated if it is the same in other aspects as well).
|
||||||
|
* However, if the previous result was disabled the new one will be disabled as well.
|
||||||
|
* @param message A violation message
|
||||||
|
*/
|
||||||
|
public ComponentAnalysisResult(final String issueId, final String message) {
|
||||||
|
this(issueId, message, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param issueId A rule-defined id that corresponds to a unique type of issue recognized by the rule.
|
||||||
|
* Newer analysis runs may produce a result with the same issueId in which case the old one will
|
||||||
|
* be overwritten (or recreated if it is the same in other aspects as well).
|
||||||
|
* However, if the previous result was disabled the new one will be disabled as well.
|
||||||
|
* @param message A violation message
|
||||||
|
* @param explanation A detailed explanation of the violation
|
||||||
|
*/
|
||||||
|
public ComponentAnalysisResult(final String issueId, final String message, final String explanation) {
|
||||||
|
super(issueId, message, explanation);
|
||||||
|
}
|
||||||
|
}
|
|
@ -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.flowanalysis;
|
||||||
|
|
||||||
|
public enum EnforcementPolicy {
|
||||||
|
/**
|
||||||
|
* Rules with this enforcement policy only warns about rule violations.
|
||||||
|
*/
|
||||||
|
WARN,
|
||||||
|
/**
|
||||||
|
* Rules with this enforcement policy also invalidate the corresponding components and fixing
|
||||||
|
* these problems are to be considered mandatory.
|
||||||
|
*/
|
||||||
|
ENFORCE;
|
||||||
|
}
|
|
@ -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.flowanalysis;
|
||||||
|
|
||||||
|
import org.apache.nifi.controller.VersionedControllerServiceLookup;
|
||||||
|
|
||||||
|
import java.util.Optional;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Used for accessing flow- or other analysis-related information
|
||||||
|
*/
|
||||||
|
public interface FlowAnalysisContext {
|
||||||
|
/**
|
||||||
|
* @return the {@link VersionedControllerServiceLookup} which can be used to obtain
|
||||||
|
* Versioned Controller Services during flow analysis
|
||||||
|
*/
|
||||||
|
VersionedControllerServiceLookup getVersionedControllerServiceLookup();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the currently configured maximum number of threads that can be
|
||||||
|
* used for executing processors at any given time.
|
||||||
|
*/
|
||||||
|
int getMaxTimerDrivenThreadCount();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return <code>true</code> if this instance of NiFi is configured to be part of a cluster, <code>false</code>
|
||||||
|
* if this instance of NiFi is a standalone instance
|
||||||
|
*/
|
||||||
|
boolean isClustered();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return an Optional with the ID of this node in the cluster, or empty if either this node is not clustered or the Node Identifier
|
||||||
|
* has not yet been established
|
||||||
|
*/
|
||||||
|
Optional<String> getClusterNodeIdentifier();
|
||||||
|
}
|
|
@ -0,0 +1,69 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.nifi.flowanalysis;
|
||||||
|
|
||||||
|
import org.apache.nifi.components.ConfigurableComponent;
|
||||||
|
import org.apache.nifi.flow.VersionedComponent;
|
||||||
|
import org.apache.nifi.flow.VersionedProcessGroup;
|
||||||
|
import org.apache.nifi.reporting.InitializationException;
|
||||||
|
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A single rule that can analyze components or a flow (represented by a process group)
|
||||||
|
*/
|
||||||
|
public interface FlowAnalysisRule extends ConfigurableComponent {
|
||||||
|
/**
|
||||||
|
* Provides the Flow Analysis Rule with access to objects that may be of use
|
||||||
|
* throughout its lifecycle
|
||||||
|
*
|
||||||
|
* @param context see {@link FlowAnalysisRuleInitializationContext}
|
||||||
|
* @throws org.apache.nifi.reporting.InitializationException if unable to initialize
|
||||||
|
*/
|
||||||
|
void initialize(FlowAnalysisRuleInitializationContext context) throws InitializationException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Analyze a component provided by the framework.
|
||||||
|
* This is a callback method invoked by the framework.
|
||||||
|
* It should be expected that this method will be called with any and all available components.
|
||||||
|
*
|
||||||
|
* @param component the component to be analyzed
|
||||||
|
* @param context see {@link FlowAnalysisRuleContext}
|
||||||
|
* @return a collection of {@link ComponentAnalysisResult} as the result of the analysis of the given component
|
||||||
|
*/
|
||||||
|
default Collection<ComponentAnalysisResult> analyzeComponent(VersionedComponent component, FlowAnalysisRuleContext context) {
|
||||||
|
return Collections.emptySet();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Analyze a flow or a part of it, represented by a process group.
|
||||||
|
* This is a callback method invoked by the framework.
|
||||||
|
* It should be expected that this method will be called by the root process group and all of its child process groups.
|
||||||
|
* In case a flow analysis is requested for a particular process group this method will be called for all it's child
|
||||||
|
* process groups as well.
|
||||||
|
*
|
||||||
|
* @param processGroup the process group to be analyzed
|
||||||
|
* @param context see {@link FlowAnalysisRuleContext}
|
||||||
|
* @return a collection of {@link GroupAnalysisResult} as the result of the analysis.
|
||||||
|
* One {@link GroupAnalysisResult} in the collection can either refer to a component within the analyzed process group,
|
||||||
|
* to a child process group or the entirety of the process group
|
||||||
|
*/
|
||||||
|
default Collection<GroupAnalysisResult> analyzeProcessGroup(VersionedProcessGroup processGroup, FlowAnalysisRuleContext context) {
|
||||||
|
return Collections.emptySet();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,53 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.nifi.flowanalysis;
|
||||||
|
|
||||||
|
import org.apache.nifi.components.PropertyDescriptor;
|
||||||
|
import org.apache.nifi.components.state.StateManager;
|
||||||
|
import org.apache.nifi.context.PropertyContext;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This interface provides a bridge between the NiFi Framework and a
|
||||||
|
* {@link FlowAnalysisRule}. This context allows a FlowAnalysisRule to access
|
||||||
|
* configuration supplied by the user.
|
||||||
|
*/
|
||||||
|
public interface FlowAnalysisRuleContext extends PropertyContext {
|
||||||
|
/**
|
||||||
|
* @return the name of the rule that is being triggered
|
||||||
|
*/
|
||||||
|
String getRuleName();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a Map of all known {@link PropertyDescriptor}s to their
|
||||||
|
* configured properties. This Map will contain a <code>null</code> for any
|
||||||
|
* Property that has not been configured by the user, even if the
|
||||||
|
* PropertyDescriptor has a default value
|
||||||
|
*/
|
||||||
|
Map<PropertyDescriptor, String> getProperties();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the StateManager that can be used to store and retrieve state for this component
|
||||||
|
*/
|
||||||
|
StateManager getStateManager();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a FlowAnalysisContext that can be used to access flow- or other analysis-related information
|
||||||
|
*/
|
||||||
|
FlowAnalysisContext getFlowAnalysisContext();
|
||||||
|
}
|
|
@ -0,0 +1,46 @@
|
||||||
|
/*
|
||||||
|
* 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.flowanalysis;
|
||||||
|
|
||||||
|
import org.apache.nifi.controller.NodeTypeProvider;
|
||||||
|
import org.apache.nifi.kerberos.KerberosContext;
|
||||||
|
import org.apache.nifi.logging.ComponentLog;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Provides configuration information to a
|
||||||
|
* FlowAnalysisRule at the time of initialization
|
||||||
|
*/
|
||||||
|
public interface FlowAnalysisRuleInitializationContext extends KerberosContext {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the identifier for the FlowAnalysisRule
|
||||||
|
*/
|
||||||
|
String getIdentifier();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @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.
|
||||||
|
* @since Apache NiFi 1.5.0
|
||||||
|
*/
|
||||||
|
NodeTypeProvider getNodeTypeProvider();
|
||||||
|
}
|
|
@ -0,0 +1,22 @@
|
||||||
|
/*
|
||||||
|
* 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.flowanalysis;
|
||||||
|
|
||||||
|
public enum FlowAnalysisRuleState {
|
||||||
|
ENABLED,
|
||||||
|
DISABLED;
|
||||||
|
}
|
|
@ -0,0 +1,112 @@
|
||||||
|
/*
|
||||||
|
* 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.flowanalysis;
|
||||||
|
|
||||||
|
import org.apache.nifi.flow.VersionedComponent;
|
||||||
|
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.StringJoiner;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Holds information about a {@link FlowAnalysisRule} violation after analyzing (a part of) the flow, represented by a process group.
|
||||||
|
* One such analysis can result in multiple instances of this class.
|
||||||
|
*/
|
||||||
|
public class GroupAnalysisResult extends AbstractAnalysisResult {
|
||||||
|
private final Optional<VersionedComponent> component;
|
||||||
|
|
||||||
|
private GroupAnalysisResult(final String issueId, final String message, final String explanation, final Optional<VersionedComponent> component) {
|
||||||
|
super(issueId, message, explanation);
|
||||||
|
this.component = component;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the component this result corresponds to or empty if this result corresponds to the entirety of the process group that was analyzed
|
||||||
|
*/
|
||||||
|
public Optional<VersionedComponent> getComponent() {
|
||||||
|
return component;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return new StringJoiner(", ", this.getClass().getSimpleName() + "[", "]")
|
||||||
|
.add("issueId='" + issueId + "'")
|
||||||
|
.add("message='" + message + "'")
|
||||||
|
.add("explanation='" + explanation + "'")
|
||||||
|
.add("component='" + component + "'")
|
||||||
|
.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Build a new analysis result tied to the currently analyzed process group
|
||||||
|
*
|
||||||
|
* @param issueId A rule-defined id that corresponds to a unique type of issue recognized by the rule.
|
||||||
|
* Newer analysis runs may produce a result with the same issueId in which case the old one will
|
||||||
|
* be overwritten (or recreated if it is the same in other aspects as well).
|
||||||
|
* However, if the previous result was disabled the new one will be disabled as well.
|
||||||
|
* @param message A violation message
|
||||||
|
* @return a Builder for a new analysis result instance tied to the currently analyzed process group
|
||||||
|
*/
|
||||||
|
public static Builder forGroup(final String issueId, final String message) {
|
||||||
|
return new Builder(null, issueId, message);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Build a new analysis result tied to a component.
|
||||||
|
* Note that the result will be scoped to the process group of the component and not the currently analyzed group.
|
||||||
|
* This means that even when a new analysis is run against that process group, this result will become obsolete.
|
||||||
|
*
|
||||||
|
* @param component The component that this result is tied to
|
||||||
|
* @param issueId A rule-defined id that corresponds to a unique type of issue recognized by the rule.
|
||||||
|
* Newer analysis runs may produce a result with the same issueId in which case the old one will
|
||||||
|
* be overwritten (or recreated if it is the same in other aspects as well).
|
||||||
|
* However, if the previous result was disabled the new one will be disabled as well.
|
||||||
|
* @param message A violation message
|
||||||
|
* @return a Builder for a new analysis result tied to a component
|
||||||
|
*/
|
||||||
|
public static Builder forComponent(final VersionedComponent component, final String issueId, final String message) {
|
||||||
|
return new Builder(component, issueId, message);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Builder {
|
||||||
|
private final VersionedComponent component;
|
||||||
|
private final String issueId;
|
||||||
|
private final String message;
|
||||||
|
private String explanation;
|
||||||
|
|
||||||
|
private Builder(final VersionedComponent component, final String issueId, final String message) {
|
||||||
|
this.component = component;
|
||||||
|
this.issueId = issueId;
|
||||||
|
this.message = message;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param explanation A detailed explanation of the violation
|
||||||
|
* @return this Builder
|
||||||
|
*/
|
||||||
|
public Builder explanation(final String explanation) {
|
||||||
|
this.explanation = explanation;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the flow analysis result
|
||||||
|
*/
|
||||||
|
public GroupAnalysisResult build() {
|
||||||
|
return new GroupAnalysisResult(issueId, message, explanation, Optional.ofNullable(component));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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.flowanalysis;
|
||||||
|
|
||||||
|
import org.apache.nifi.components.ConfigVerificationResult;
|
||||||
|
import org.apache.nifi.controller.ConfigurationContext;
|
||||||
|
import org.apache.nifi.logging.ComponentLog;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* <p>
|
||||||
|
* Any Flow Analysis Rule that implements this interface will be provided the opportunity to verify
|
||||||
|
* a given configuration of the Flow Analysis Rule. This allows the Flow Analysis Rule 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 Flow Analysis Rule is not valid, it cannot be started. However, verification may be
|
||||||
|
* more expensive or time-consuming to complete. For example, validation may ensure that a username is
|
||||||
|
* provided for connecting to an external service but should not perform any sort of network connection
|
||||||
|
* in order to verify that the username is accurate. Verification, on the other hand, may create resources
|
||||||
|
* such as network connections, may be more expensive to complete, and may be run only when a user invokes
|
||||||
|
* the action (though verification may later occur at other stages, such as when starting a component).
|
||||||
|
* </p>
|
||||||
|
*
|
||||||
|
* <p>
|
||||||
|
* Verification is allowed to be run only when a Flow Analysis Rule is fully stopped. I.e., it has no active threads
|
||||||
|
* and currently has a state of STOPPED. Therefore, any initialization logic that may need to be performed
|
||||||
|
* before the Flow Analysis Rule is triggered may also be required for verification. However, the framework is not responsible
|
||||||
|
* for triggering the Lifecycle management stages, such as @OnScheduled before triggering the verification. Such
|
||||||
|
* methods should be handled by the {@link #verify(ConfigurationContext, ComponentLog)} itself.
|
||||||
|
* The {@link #verify(ConfigurationContext, ComponentLog)} method will only be called if the configuration is valid according to the
|
||||||
|
* validation rules (i.e., all Property Descriptors' validators and customValidate methods have indicated that the configuration is valid).
|
||||||
|
* </p>
|
||||||
|
*/
|
||||||
|
public interface VerifiableFlowAnalysisRule {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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);
|
||||||
|
|
||||||
|
}
|
|
@ -46,6 +46,11 @@ public enum ComponentType {
|
||||||
*/
|
*/
|
||||||
REPORTING_TASK,
|
REPORTING_TASK,
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Bulletin is associated with a Flow Analysis Rule
|
||||||
|
*/
|
||||||
|
FLOW_ANALYSIS_RULE,
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Bulletin is associated with a Process Group
|
* Bulletin is associated with a Process Group
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -304,6 +304,9 @@ public class NiFiProperties extends ApplicationProperties {
|
||||||
public static final String ANALYTICS_CONNECTION_MODEL_SCORE_NAME = "nifi.analytics.connection.model.score.name";
|
public static final String ANALYTICS_CONNECTION_MODEL_SCORE_NAME = "nifi.analytics.connection.model.score.name";
|
||||||
public static final String ANALYTICS_CONNECTION_MODEL_SCORE_THRESHOLD = "nifi.analytics.connection.model.score.threshold";
|
public static final String ANALYTICS_CONNECTION_MODEL_SCORE_THRESHOLD = "nifi.analytics.connection.model.score.threshold";
|
||||||
|
|
||||||
|
// flow analysis properties
|
||||||
|
public static final String BACKGROUND_FLOW_ANALYSIS_SCHEDULE = "nifi.flow.analysis.background.task.schedule";
|
||||||
|
|
||||||
// runtime monitoring properties
|
// runtime monitoring properties
|
||||||
public static final String MONITOR_LONG_RUNNING_TASK_SCHEDULE = "nifi.monitor.long.running.task.schedule";
|
public static final String MONITOR_LONG_RUNNING_TASK_SCHEDULE = "nifi.monitor.long.running.task.schedule";
|
||||||
public static final String MONITOR_LONG_RUNNING_TASK_THRESHOLD = "nifi.monitor.long.running.task.threshold";
|
public static final String MONITOR_LONG_RUNNING_TASK_THRESHOLD = "nifi.monitor.long.running.task.threshold";
|
||||||
|
|
After Width: | Height: | Size: 81 KiB |
After Width: | Height: | Size: 79 KiB |
After Width: | Height: | Size: 77 KiB |
After Width: | Height: | Size: 17 KiB |
After Width: | Height: | Size: 13 KiB |
After Width: | Height: | Size: 168 KiB |
Before Width: | Height: | Size: 23 KiB After Width: | Height: | Size: 72 KiB |
|
@ -95,6 +95,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.
|
*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.
|
||||||
|
|
||||||
|
*Flow Analysis Rules*: Flow Analysis Rules can analyze components or (parts of) the flow. They may produce rule violations which can help adjust or maintain optimal flow design. The DFM adds and configures Flow Analysis Rules in the User Interface as desired.
|
||||||
|
|
||||||
*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.
|
*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.
|
*Funnel*: A funnel is a NiFi component that is used to combine the data from several Connections into a single Connection.
|
||||||
|
@ -193,7 +195,7 @@ The available global access policies are:
|
||||||
|======================
|
|======================
|
||||||
|Policy |Privilege
|
|Policy |Privilege
|
||||||
|view the UI |Allows users to view the UI
|
|view the UI |Allows users to view the UI
|
||||||
|access the controller |Allows users to view and modify the controller including Management Controller Services, Reporting Tasks, Registry Clients, Parameter Providers and nodes in the cluster
|
|access the controller |Allows users to view and modify the controller including Management Controller Services, Reporting Tasks, Flow Analysis Rules, Registry Clients, Parameter Providers and nodes in the cluster
|
||||||
|query provenance |Allows users to submit a provenance search and request even lineage
|
|query provenance |Allows users to submit a provenance search and request even lineage
|
||||||
|access restricted components |Allows users to create/modify restricted components assuming other permissions are sufficient. The restricted
|
|access restricted components |Allows users to create/modify restricted components assuming other permissions are sufficient. The restricted
|
||||||
components may indicate which specific permissions are required. Permissions can be granted for specific restrictions or be granted regardless
|
components may indicate which specific permissions are required. Permissions can be granted for specific restrictions or be granted regardless
|
||||||
|
@ -453,10 +455,10 @@ choosing `Configure`.
|
||||||
|
|
||||||
[[component-versioning]]
|
[[component-versioning]]
|
||||||
=== Component Versions
|
=== Component Versions
|
||||||
You have access to information about the version of your Processors, Controller Services, and Reporting Tasks.
|
You have access to information about the version of your Processors, Controller Services, Reporting Tasks and Flow Analysis Rules.
|
||||||
This is especially useful when you are working within a clustered environment with multiple NiFi instances running
|
This is especially useful when you are working within a clustered environment with multiple NiFi instances running
|
||||||
different versions of a component or if you have upgraded to a newer version of a processor. The Add Processor,
|
different versions of a component or if you have upgraded to a newer version of a processor. The Add Processor,
|
||||||
Add Controller Service, and Add Reporting Task dialogs include a column identifying the component version, as well
|
Add Controller Service, Add Reporting Task and Add Flow Analysis Rule dialogs include a column identifying the component version, as well
|
||||||
as the name of the component, the organization or group that created the component, and the NAR bundle that contains
|
as the name of the component, the organization or group that created the component, and the NAR bundle that contains
|
||||||
the component.
|
the component.
|
||||||
|
|
||||||
|
@ -897,7 +899,7 @@ The values of properties in the flow, including sensitive properties, can be par
|
||||||
- A sensitive property can only reference a Sensitive Parameter
|
- A sensitive property can only reference a Sensitive Parameter
|
||||||
- A non-sensitive property can only reference a Non-Sensitive Parameter
|
- A non-sensitive property can only reference a Non-Sensitive Parameter
|
||||||
- Properties that reference Controller Services can not use Parameters
|
- Properties that reference Controller Services can not use Parameters
|
||||||
- Parameters cannot be referenced in Reporting Tasks or in Management Controller Services
|
- Parameters cannot be referenced in Reporting Tasks, Flow Analysis Rules or in Management Controller Services
|
||||||
|
|
||||||
The UI indicates whether a Parameter can be used for a property value.
|
The UI indicates whether a Parameter can be used for a property value.
|
||||||
|
|
||||||
|
@ -1286,9 +1288,9 @@ For more information, see the <<administration-guide.adoc#custom_properties,Cust
|
||||||
[[Controller_Services]]
|
[[Controller_Services]]
|
||||||
=== Controller Services
|
=== Controller Services
|
||||||
|
|
||||||
Controller Services are shared services that can be used by reporting tasks, processors, and other services to utilize for configuration or task execution.
|
Controller Services are shared services that can be used by reporting tasks, flow analysis rules, processors, and other services to utilize for configuration or task execution.
|
||||||
|
|
||||||
IMPORTANT: Controller Services defined on the controller level are limited to reporting tasks and other services defined there. Controller Services for use by processors in your dataflow must be defined in the configuration of the root process group or sub-process group(s) where they will be used.
|
IMPORTANT: Controller Services defined on the controller level are limited to reporting tasks, flow analysis rules and other services defined there. Controller Services for use by processors in your dataflow must be defined in the configuration of the root process group or sub-process group(s) where they will be used.
|
||||||
|
|
||||||
NOTE: If your NiFi instance is secured, your ability to view and add Controller Services is dependent on the privileges assigned to you. If you do not have access to one or more Controller Services, you are not able to see or access it in the UI. Access privileges can be assigned on a global or Controller Service-specific basis (see <<UI-with-multi-tenant-authorization>> for more information).
|
NOTE: If your NiFi instance is secured, your ability to view and add Controller Services is dependent on the privileges assigned to you. If you do not have access to one or more Controller Services, you are not able to see or access it in the UI. Access privileges can be assigned on a global or Controller Service-specific basis (see <<UI-with-multi-tenant-authorization>> for more information).
|
||||||
|
|
||||||
|
@ -1299,7 +1301,7 @@ To add a Management Controller Service, select Controller Settings from the Glob
|
||||||
|
|
||||||
image:controller-settings-selection.png["Global Menu - Controller Settings"]
|
image:controller-settings-selection.png["Global Menu - Controller Settings"]
|
||||||
|
|
||||||
This displays the NiFi Settings window. The window has five tabs: General, Management Controller Services, Reporting Tasks, Registry Clients and Parameter Providers. The General tab provides settings for the overall maximum thread counts of the instance.
|
This displays the NiFi Settings window. The window has six tabs: General, Management Controller Services, Reporting Tasks, Flow Analysis Rules, Registry Clients and Parameter Providers. The General tab provides settings for the overall maximum thread counts of the instance.
|
||||||
|
|
||||||
image:settings-general-tab.png["Controller Settings General Tab"]
|
image:settings-general-tab.png["Controller Settings General Tab"]
|
||||||
|
|
||||||
|
@ -1399,6 +1401,49 @@ The Comments tab is just an open-text field, where the DFM may include comments
|
||||||
|
|
||||||
When you want to run the Reporting Task, click the "Start" button (image:iconStart.png["Start Button"]).
|
When you want to run the Reporting Task, click the "Start" button (image:iconStart.png["Start Button"]).
|
||||||
|
|
||||||
|
[[Flow_Analysis_Rules]]
|
||||||
|
=== Flow Analysis Rules
|
||||||
|
|
||||||
|
Flow Analysis Rules can analyze components or (parts of) the flow. They may produce rule violations which can help adjust or maintain optimal flow design.
|
||||||
|
Each rule can either be a Recommendation or a Policy which can be set on the Configure Flow Analysis Rule window.
|
||||||
|
Rule violations of Recommendation type rules can be reported and viewed later but otherwise have no impact on functionality.
|
||||||
|
Rule violations of Policy type rules can also be reported and viewed later but also impacts functionality: components that violate a Policy become invalid and remain
|
||||||
|
so until the rule violation is resolved.
|
||||||
|
The DFM adds and configures Flow Analysis Rules similar to the process for Controller Services. To add a Flow Analysis Rule, select Controller Settings from the Global Menu.
|
||||||
|
|
||||||
|
image:controller-settings-selection.png["Global Menu - Controller Settings"]
|
||||||
|
|
||||||
|
This displays the NiFi Settings window. Select the Flow Analysis Rules tab and click the `+` button in the upper-right corner to create a new Flow Analysis Rule.
|
||||||
|
|
||||||
|
image:flow-analysis-rules-tab.png["Flow Analysis Rules Tab"]
|
||||||
|
|
||||||
|
The Add Flow Analysis Rule window opens. This window is similar to the Add Processor window. It provides a list of the available Flow Analysis Rules on the right and a tag cloud, showing the most common category tags used for Flow Analysis Rules, on the left. The DFM may click any tag in the tag cloud in order to narrow down the list of Flow Analysis Rules to those that fit the categories desired. The DFM may also use the Filter field at the top-right of the window to search for the desired Flow Analysis Rule or use the Source drop-down at the top-left to filter the list by the group who created them. Upon selecting a Flow Analysis Rule from the list, the DFM can see a description of the rule below. Select the desired flow analysis rule and click Add, or simply double-click the name of the service to add it.
|
||||||
|
|
||||||
|
image:add-flow-analysis-rule-window.png["Add Flow Analysis Rule Window"]
|
||||||
|
|
||||||
|
Once a Flow Analysis Rule has been added, the DFM may configure it by clicking the "Configure" button in the far-right column (when the rule is disabled). Other buttons in this column include "Enable", "Disable", "View Configuration", "Remove", "State" and "Access Policies".
|
||||||
|
|
||||||
|
image:flow-analysis-rules-configure-buttons.png["Flow Analysis Rules Configure Buttons"]
|
||||||
|
|
||||||
|
You can obtain information about Flow Analysis Rules by clicking the "View Details", "Usage", and "Alerts" buttons in the left-hand column.
|
||||||
|
|
||||||
|
image:flow-analysis-rules-info-buttons.png["Flow Analysis Rules Information Buttons"]
|
||||||
|
|
||||||
|
When the DFM clicks the "Configure" button, a Configure Flow Analysis Rule window opens. It has three tabs: Settings, Properties, and Comments. This window is similar to the Configure Processor window. The Settings tab provides a place for the DFM to give the Flow Analysis Rule a unique name (if desired). It also lists the UUID, Type, and Bundle information for the rule and provides a setting for its type (Recommendation or Policy). The DFM may hover the mouse over the question mark icons to see more information about each setting.
|
||||||
|
|
||||||
|
image:configure-flow-analysis-rule-settings.png["Configure Flow Analysis Rule Settings"]
|
||||||
|
|
||||||
|
The Properties tab lists the various properties that may be configured for the rule. The DFM may hover the mouse over the question mark icons to see more information about each property.
|
||||||
|
|
||||||
|
image:configure-flow-analysis-rule-properties.png["Configure Flow Analysis Rule Properties"]
|
||||||
|
|
||||||
|
The Comments tab is just an open-text field, where the DFM may include comments about the rule. After configuring the Flow Analysis Rule, click "Apply" to save the configuration and close the window, or click "Cancel" to discard the changes and close the window.
|
||||||
|
|
||||||
|
When you want the Flow Analysis Rule to be active, click the "Enable" button (image:iconEnable.png["Enable Button"]).
|
||||||
|
|
||||||
|
When you want the Flow Analysis Rule to be inactive, click the "Disable" button (image:iconDisable.png["Disable Button"]).
|
||||||
|
Disabling a rule also renders all corresponding violations null and void.
|
||||||
|
|
||||||
|
|
||||||
[[Connecting_Components]]
|
[[Connecting_Components]]
|
||||||
=== Connecting Components
|
=== Connecting Components
|
||||||
|
|
|
@ -0,0 +1,46 @@
|
||||||
|
/*
|
||||||
|
* 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 request to analyze (a part) of the flow.
|
||||||
|
*/
|
||||||
|
@XmlType(name = "analyzeFlowRequest")
|
||||||
|
public class AnalyzeFlowRequestDTO extends AsynchronousRequestDTO<AnalyzeFlowRequestUpdateStepDTO> {
|
||||||
|
public AnalyzeFlowRequestDTO() {
|
||||||
|
}
|
||||||
|
|
||||||
|
private String processGroupId;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The id of the process group representing (a part of) the flow to be analyzed.
|
||||||
|
*
|
||||||
|
* @return The id
|
||||||
|
*/
|
||||||
|
@ApiModelProperty("The id of the process group representing (a part of) the flow to be analyzed.")
|
||||||
|
public String getProcessGroupId() {
|
||||||
|
return this.processGroupId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setProcessGroupId(final String processGroupId) {
|
||||||
|
this.processGroupId = processGroupId;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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 = "analyzeFlowRequestUpdateStep")
|
||||||
|
public class AnalyzeFlowRequestUpdateStepDTO extends UpdateStepDTO {
|
||||||
|
}
|
|
@ -0,0 +1,293 @@
|
||||||
|
/*
|
||||||
|
* 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;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
@XmlType(name = "flowAnalysisRule")
|
||||||
|
public class FlowAnalysisRuleDTO 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 state;
|
||||||
|
private String comments;
|
||||||
|
private Boolean persistsState;
|
||||||
|
private Boolean restricted;
|
||||||
|
private Boolean deprecated;
|
||||||
|
private Boolean isExtensionMissing;
|
||||||
|
private Boolean multipleVersionsAvailable;
|
||||||
|
private Boolean supportsSensitiveDynamicProperties;
|
||||||
|
|
||||||
|
private String enforcementPolicy;
|
||||||
|
|
||||||
|
private Map<String, String> properties;
|
||||||
|
private Map<String, PropertyDescriptorDTO> descriptors;
|
||||||
|
private Set<String> sensitiveDynamicPropertyNames;
|
||||||
|
|
||||||
|
private Collection<String> validationErrors;
|
||||||
|
private String validationStatus;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return user-defined name of the flow analysis rule
|
||||||
|
*/
|
||||||
|
@ApiModelProperty(
|
||||||
|
value = "The name of the flow analysis rule."
|
||||||
|
)
|
||||||
|
public String getName() {
|
||||||
|
return name;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setName(String name) {
|
||||||
|
this.name = name;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return user-defined comments for the flow analysis rule
|
||||||
|
*/
|
||||||
|
@ApiModelProperty(
|
||||||
|
value = "The comments of the flow analysis rule."
|
||||||
|
)
|
||||||
|
public String getComments() {
|
||||||
|
return comments;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setComments(String comments) {
|
||||||
|
this.comments = comments;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return type of flow analysis rule
|
||||||
|
*/
|
||||||
|
@ApiModelProperty(
|
||||||
|
value = "The fully qualified type of the flow analysis rule."
|
||||||
|
)
|
||||||
|
public String getType() {
|
||||||
|
return type;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setType(String type) {
|
||||||
|
this.type = type;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The details of the artifact that bundled this flow analysis rule type.
|
||||||
|
*
|
||||||
|
* @return The bundle details
|
||||||
|
*/
|
||||||
|
@ApiModelProperty(
|
||||||
|
value = "The details of the artifact that bundled this flow analysis rule type."
|
||||||
|
)
|
||||||
|
public BundleDTO getBundle() {
|
||||||
|
return bundle;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setBundle(BundleDTO bundle) {
|
||||||
|
this.bundle = bundle;
|
||||||
|
}
|
||||||
|
/**
|
||||||
|
* @return whether this flow analysis rule persists state
|
||||||
|
*/
|
||||||
|
@ApiModelProperty(
|
||||||
|
value = "Whether the flow analysis rule persists state."
|
||||||
|
)
|
||||||
|
public Boolean getPersistsState() {
|
||||||
|
return persistsState;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setPersistsState(Boolean persistsState) {
|
||||||
|
this.persistsState = persistsState;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return whether this flow analysis rule requires elevated privileges
|
||||||
|
*/
|
||||||
|
@ApiModelProperty(
|
||||||
|
value = "Whether the flow analysis rule requires elevated privileges."
|
||||||
|
)
|
||||||
|
public Boolean getRestricted() {
|
||||||
|
return restricted;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setRestricted(Boolean restricted) {
|
||||||
|
this.restricted = restricted;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Whether the flow analysis rule has been deprecated.
|
||||||
|
*/
|
||||||
|
@ApiModelProperty(
|
||||||
|
value = "Whether the flow analysis rule has been deprecated."
|
||||||
|
)
|
||||||
|
public Boolean getDeprecated() {
|
||||||
|
return deprecated;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setDeprecated(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(Boolean extensionMissing) {
|
||||||
|
isExtensionMissing = extensionMissing;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return whether this flow analysis rule has multiple versions available
|
||||||
|
*/
|
||||||
|
@ApiModelProperty(
|
||||||
|
value = "Whether the flow analysis rule has multiple versions available."
|
||||||
|
)
|
||||||
|
public Boolean getMultipleVersionsAvailable() {
|
||||||
|
return multipleVersionsAvailable;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setMultipleVersionsAvailable(Boolean multipleVersionsAvailable) {
|
||||||
|
this.multipleVersionsAvailable = multipleVersionsAvailable;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return whether this flow analysis rule supports sensitive dynamic properties
|
||||||
|
*/
|
||||||
|
@ApiModelProperty(
|
||||||
|
value = "Whether the flow analysis rule supports sensitive dynamic properties."
|
||||||
|
)
|
||||||
|
public Boolean getSupportsSensitiveDynamicProperties() {
|
||||||
|
return supportsSensitiveDynamicProperties;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setSupportsSensitiveDynamicProperties(final Boolean supportsSensitiveDynamicProperties) {
|
||||||
|
this.supportsSensitiveDynamicProperties = supportsSensitiveDynamicProperties;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return current scheduling state of the flow analysis rule
|
||||||
|
*/
|
||||||
|
@ApiModelProperty(
|
||||||
|
value = "The state of the flow analysis rule.",
|
||||||
|
allowableValues = "ENABLED, DISABLED"
|
||||||
|
)
|
||||||
|
public String getState() {
|
||||||
|
return state;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setState(String state) {
|
||||||
|
this.state = state;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Enforcement Policy
|
||||||
|
*/
|
||||||
|
@ApiModelProperty(
|
||||||
|
value = "Enforcement Policy."
|
||||||
|
)
|
||||||
|
public String getEnforcementPolicy() {
|
||||||
|
return enforcementPolicy;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setEnforcementPolicy(String enforcementPolicy) {
|
||||||
|
this.enforcementPolicy = enforcementPolicy;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return flow analysis rule's properties
|
||||||
|
*/
|
||||||
|
@ApiModelProperty(
|
||||||
|
value = "The properties of the flow analysis rule."
|
||||||
|
)
|
||||||
|
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 flow analysis rules properties."
|
||||||
|
)
|
||||||
|
public Map<String, PropertyDescriptorDTO> getDescriptors() {
|
||||||
|
return descriptors;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setDescriptors(Map<String, PropertyDescriptorDTO> descriptors) {
|
||||||
|
this.descriptors = descriptors;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Set of sensitive dynamic property names
|
||||||
|
*/
|
||||||
|
@ApiModelProperty(
|
||||||
|
value = "Set of sensitive dynamic property names"
|
||||||
|
)
|
||||||
|
public Set<String> getSensitiveDynamicPropertyNames() {
|
||||||
|
return sensitiveDynamicPropertyNames;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setSensitiveDynamicPropertyNames(final Set<String> sensitiveDynamicPropertyNames) {
|
||||||
|
this.sensitiveDynamicPropertyNames = sensitiveDynamicPropertyNames;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets the validation errors from this flow analysis rule. These validation errors represent the problems with the flow analysis rule that must be resolved before it can be scheduled to run.
|
||||||
|
*
|
||||||
|
* @return The validation errors
|
||||||
|
*/
|
||||||
|
@ApiModelProperty(
|
||||||
|
value = "Gets the validation errors from the flow analysis rule. These validation errors represent the problems with the flow analysis rule 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 Flow Analysis Rule is valid, invalid, or still in the process of validating (i.e., it is unknown whether or not the Flow Analysis Rule 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,153 @@
|
||||||
|
/*
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A result of a rule violation produced during a flow analysis
|
||||||
|
*/
|
||||||
|
@XmlType(name = "flowAnalysisRuleViolation")
|
||||||
|
public class FlowAnalysisRuleViolationDTO {
|
||||||
|
public FlowAnalysisRuleViolationDTO() {
|
||||||
|
}
|
||||||
|
|
||||||
|
private String enforcementPolicy;
|
||||||
|
private String scope;
|
||||||
|
private String subjectId;
|
||||||
|
private String subjectDisplayName;
|
||||||
|
private String groupId;
|
||||||
|
private String ruleId;
|
||||||
|
private String issueId;
|
||||||
|
private String violationMessage;
|
||||||
|
|
||||||
|
private PermissionsDTO subjectPermissionDto;
|
||||||
|
|
||||||
|
private boolean enabled;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the enforcement policy of the rule that produced this result
|
||||||
|
*/
|
||||||
|
public String getEnforcementPolicy() {
|
||||||
|
return enforcementPolicy;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setEnforcementPolicy(String enforcementPolicy) {
|
||||||
|
this.enforcementPolicy = enforcementPolicy;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the scope of the result
|
||||||
|
*/
|
||||||
|
public String getScope() {
|
||||||
|
return scope;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setScope(String scope) {
|
||||||
|
this.scope = scope;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the id of the subject that violated the rule
|
||||||
|
*/
|
||||||
|
public String getSubjectId() {
|
||||||
|
return subjectId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setSubjectId(String subjectId) {
|
||||||
|
this.subjectId = subjectId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the displayed name of the subject that violated the rule
|
||||||
|
*/
|
||||||
|
public String getSubjectDisplayName() {
|
||||||
|
return subjectDisplayName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setSubjectDisplayName(String subjectDisplayName) {
|
||||||
|
this.subjectDisplayName = subjectDisplayName;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return group id - if this violation is a result of a component analysis, then the id of the group of the component.
|
||||||
|
* If this violation is a result of a group analysis, then the id of that group itself.
|
||||||
|
*/
|
||||||
|
public String getGroupId() {
|
||||||
|
return groupId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setGroupId(String groupId) {
|
||||||
|
this.groupId = groupId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the id of the rule that produced this result
|
||||||
|
*/
|
||||||
|
public String getRuleId() {
|
||||||
|
return ruleId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setRuleId(String ruleId) {
|
||||||
|
this.ruleId = ruleId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a rule-defined id that corresponds to a unique type of issue recognized by the rule
|
||||||
|
*/
|
||||||
|
public String getIssueId() {
|
||||||
|
return issueId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setIssueId(String issueId) {
|
||||||
|
this.issueId = issueId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the violation message
|
||||||
|
*/
|
||||||
|
public String getViolationMessage() {
|
||||||
|
return violationMessage;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setViolationMessage(String violationMessage) {
|
||||||
|
this.violationMessage = violationMessage;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return true if this result should be in effect, false otherwise
|
||||||
|
*/
|
||||||
|
public boolean isEnabled() {
|
||||||
|
return enabled;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setEnabled(boolean enabled) {
|
||||||
|
this.enabled = enabled;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a permission object for the subject that violated the rule
|
||||||
|
*/
|
||||||
|
public PermissionsDTO getSubjectPermissionDto() {
|
||||||
|
return subjectPermissionDto;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setSubjectPermissionDto(PermissionsDTO subjectPermissionDto) {
|
||||||
|
this.subjectPermissionDto = subjectPermissionDto;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,37 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.nifi.web.api.dto.status;
|
||||||
|
|
||||||
|
import io.swagger.annotations.ApiModelProperty;
|
||||||
|
|
||||||
|
import javax.xml.bind.annotation.XmlType;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* DTO for serializing the status of a FlowAnalysisRule.
|
||||||
|
*/
|
||||||
|
@XmlType(name = "flowAnalysisRuleStatus")
|
||||||
|
public class FlowAnalysisRuleStatusDTO extends ComponentStatusDTO {
|
||||||
|
|
||||||
|
@ApiModelProperty(value = "The run status of this FlowAnalysisRule",
|
||||||
|
accessMode = ApiModelProperty.AccessMode.READ_ONLY,
|
||||||
|
allowableValues = "ENABLED, DISABLED")
|
||||||
|
@Override
|
||||||
|
public String getRunStatus() {
|
||||||
|
return super.getRunStatus();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,44 @@
|
||||||
|
/*
|
||||||
|
* 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.AnalyzeFlowRequestDTO;
|
||||||
|
|
||||||
|
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 an AnalyzeRequestDTO.
|
||||||
|
*/
|
||||||
|
@XmlRootElement(name = "dropRequestEntity")
|
||||||
|
public class AnalyzeFlowRequestEntity extends Entity {
|
||||||
|
|
||||||
|
private AnalyzeFlowRequestDTO analyzeFlowRequest;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The AnalyzeFlowRequestDTO that is being serialized.
|
||||||
|
*
|
||||||
|
* @return The AnalyzeFlowRequestDTO object
|
||||||
|
*/
|
||||||
|
public AnalyzeFlowRequestDTO getAnalyzeFlowRequest() {
|
||||||
|
return analyzeFlowRequest;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setAnalyzeFlowRequest(AnalyzeFlowRequestDTO analyzeFlowRequest) {
|
||||||
|
this.analyzeFlowRequest = analyzeFlowRequest;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -31,6 +31,7 @@ public class ControllerBulletinsEntity extends Entity {
|
||||||
private List<BulletinEntity> bulletins;
|
private List<BulletinEntity> bulletins;
|
||||||
private List<BulletinEntity> controllerServiceBulletins;
|
private List<BulletinEntity> controllerServiceBulletins;
|
||||||
private List<BulletinEntity> reportingTaskBulletins;
|
private List<BulletinEntity> reportingTaskBulletins;
|
||||||
|
private List<BulletinEntity> flowAnalysisRuleBulletins;
|
||||||
private List<BulletinEntity> parameterProviderBulletins;
|
private List<BulletinEntity> parameterProviderBulletins;
|
||||||
private List<BulletinEntity> flowRegistryClientBulletins;
|
private List<BulletinEntity> flowRegistryClientBulletins;
|
||||||
|
|
||||||
|
@ -70,6 +71,18 @@ public class ControllerBulletinsEntity extends Entity {
|
||||||
this.reportingTaskBulletins = reportingTaskBulletins;
|
this.reportingTaskBulletins = reportingTaskBulletins;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Flow Analysis Rule bulletins to be reported to the user
|
||||||
|
*/
|
||||||
|
@ApiModelProperty("Flow Analysis Rule bulletins to be reported to the user.")
|
||||||
|
public List<BulletinEntity> getFlowAnalysisRuleBulletins() {
|
||||||
|
return flowAnalysisRuleBulletins;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setFlowAnalysisRuleBulletins(List<BulletinEntity> flowAnalysisRuleBulletins) {
|
||||||
|
this.flowAnalysisRuleBulletins = flowAnalysisRuleBulletins;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return Parameter provider bulletins to be reported to the user
|
* @return Parameter provider bulletins to be reported to the user
|
||||||
*/
|
*/
|
||||||
|
@ -100,6 +113,7 @@ public class ControllerBulletinsEntity extends Entity {
|
||||||
other.setBulletins(getBulletins() == null ? null : new ArrayList<>(getBulletins()));
|
other.setBulletins(getBulletins() == null ? null : new ArrayList<>(getBulletins()));
|
||||||
other.setControllerServiceBulletins(getControllerServiceBulletins() == null ? null : new ArrayList<>(getControllerServiceBulletins()));
|
other.setControllerServiceBulletins(getControllerServiceBulletins() == null ? null : new ArrayList<>(getControllerServiceBulletins()));
|
||||||
other.setReportingTaskBulletins(getReportingTaskBulletins() == null ? null : new ArrayList<>(getReportingTaskBulletins()));
|
other.setReportingTaskBulletins(getReportingTaskBulletins() == null ? null : new ArrayList<>(getReportingTaskBulletins()));
|
||||||
|
other.setFlowAnalysisRuleBulletins(getFlowAnalysisRuleBulletins() == null ? null : new ArrayList<>(getFlowAnalysisRuleBulletins()));
|
||||||
other.setParameterProviderBulletins(getParameterProviderBulletins() == null ? null : new ArrayList<>(getParameterProviderBulletins()));
|
other.setParameterProviderBulletins(getParameterProviderBulletins() == null ? null : new ArrayList<>(getParameterProviderBulletins()));
|
||||||
other.setFlowRegistryClientBulletins(getFlowRegistryClientBulletins() == null ? null : new ArrayList<>(getFlowRegistryClientBulletins()));
|
other.setFlowRegistryClientBulletins(getFlowRegistryClientBulletins() == null ? null : new ArrayList<>(getFlowRegistryClientBulletins()));
|
||||||
return other;
|
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 org.apache.nifi.web.api.dto.FlowAnalysisRuleDTO;
|
||||||
|
import org.apache.nifi.web.api.dto.FlowAnalysisRuleViolationDTO;
|
||||||
|
|
||||||
|
import javax.xml.bind.annotation.XmlRootElement;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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 collection of {@link FlowAnalysisRuleDTO} and another collection of {@link FlowAnalysisRuleViolationDTO}.
|
||||||
|
*/
|
||||||
|
@XmlRootElement(name = "flowAnalysisResultEntity")
|
||||||
|
public class FlowAnalysisResultEntity extends Entity {
|
||||||
|
public FlowAnalysisResultEntity() {
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<FlowAnalysisRuleDTO> rules = new ArrayList<>();
|
||||||
|
private List<FlowAnalysisRuleViolationDTO> ruleViolations = new ArrayList<>();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return set of flow analysis rules that are being serialized
|
||||||
|
*/
|
||||||
|
public List<FlowAnalysisRuleDTO> getRules() {
|
||||||
|
return rules;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setRules(List<FlowAnalysisRuleDTO> rules) {
|
||||||
|
this.rules = rules;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return set of flow analysis results that are being serialized
|
||||||
|
*/
|
||||||
|
public List<FlowAnalysisRuleViolationDTO> getRuleViolations() {
|
||||||
|
return ruleViolations;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setRuleViolations(List<FlowAnalysisRuleViolationDTO> ruleViolations) {
|
||||||
|
this.ruleViolations = ruleViolations;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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.web.api.entity;
|
||||||
|
|
||||||
|
import io.swagger.annotations.ApiModelProperty;
|
||||||
|
import org.apache.nifi.web.api.dto.PermissionsDTO;
|
||||||
|
import org.apache.nifi.web.api.dto.FlowAnalysisRuleDTO;
|
||||||
|
import org.apache.nifi.web.api.dto.status.FlowAnalysisRuleStatusDTO;
|
||||||
|
|
||||||
|
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 flow analysis rule.
|
||||||
|
*/
|
||||||
|
@XmlRootElement(name = "flowAnalysisRuleEntity")
|
||||||
|
public class FlowAnalysisRuleEntity extends ComponentEntity implements Permissible<FlowAnalysisRuleDTO>, OperationPermissible {
|
||||||
|
private FlowAnalysisRuleDTO component;
|
||||||
|
private PermissionsDTO operatePermissions;
|
||||||
|
private FlowAnalysisRuleStatusDTO status;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return flow analysis rule that is being serialized
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public FlowAnalysisRuleDTO getComponent() {
|
||||||
|
return component;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setComponent(FlowAnalysisRuleDTO component) {
|
||||||
|
this.component = component;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The permissions for this component operations
|
||||||
|
*/
|
||||||
|
@ApiModelProperty(
|
||||||
|
value = "The permissions for this component operations."
|
||||||
|
)
|
||||||
|
@Override
|
||||||
|
public PermissionsDTO getOperatePermissions() {
|
||||||
|
return operatePermissions;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setOperatePermissions(PermissionsDTO permissions) {
|
||||||
|
this.operatePermissions = permissions;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The status for this FlowAnalysisRule
|
||||||
|
*/
|
||||||
|
@ApiModelProperty(
|
||||||
|
value = "The status for this FlowAnalysisRule.",
|
||||||
|
readOnly = true
|
||||||
|
)
|
||||||
|
public FlowAnalysisRuleStatusDTO getStatus() {
|
||||||
|
return status;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setStatus(FlowAnalysisRuleStatusDTO status) {
|
||||||
|
this.status = status;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,48 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.nifi.web.api.entity;
|
||||||
|
|
||||||
|
import io.swagger.annotations.ApiModelProperty;
|
||||||
|
|
||||||
|
import javax.xml.bind.annotation.XmlType;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Run status for a given FlowAnalysisRule.
|
||||||
|
*/
|
||||||
|
@XmlType(name = "flowAnalysisRuleRunStatus")
|
||||||
|
public class FlowAnalysisRuleRunStatusEntity extends ComponentRunStatusEntity {
|
||||||
|
|
||||||
|
private static final String[] SUPPORTED_STATE = {"ENABLED", "DISABLED"};
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String[] getSupportedState() {
|
||||||
|
return SUPPORTED_STATE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* State of this FlowAnalysisRule.
|
||||||
|
* @return The state
|
||||||
|
*/
|
||||||
|
@ApiModelProperty(
|
||||||
|
value = "The state of the FlowAnalysisRule.",
|
||||||
|
allowableValues = "ENABLED, DISABLED"
|
||||||
|
)
|
||||||
|
public String getState() {
|
||||||
|
return super.getState();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -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 set of flow analysis rule types.
|
||||||
|
*/
|
||||||
|
@XmlRootElement(name = "flowAnalysisRuleTypesEntity")
|
||||||
|
public class FlowAnalysisRuleTypesEntity extends Entity {
|
||||||
|
|
||||||
|
private Set<DocumentedTypeDTO> flowAnalysisRuleTypes;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return set of reporting task types that are being serialized
|
||||||
|
*/
|
||||||
|
public Set<DocumentedTypeDTO> getFlowAnalysisRuleTypes() {
|
||||||
|
return flowAnalysisRuleTypes;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setFlowAnalysisRuleTypes(Set<DocumentedTypeDTO> enforcementPolicies) {
|
||||||
|
this.flowAnalysisRuleTypes = enforcementPolicies;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -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 set of flow analysis rules.
|
||||||
|
*/
|
||||||
|
@XmlRootElement(name = "flowAnalysisRulesEntity")
|
||||||
|
public class FlowAnalysisRulesEntity extends Entity {
|
||||||
|
|
||||||
|
private Set<FlowAnalysisRuleEntity> flowAnalysisRules;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return set of flow analysis rules that are being serialized
|
||||||
|
*/
|
||||||
|
public Set<FlowAnalysisRuleEntity> getFlowAnalysisRules() {
|
||||||
|
return flowAnalysisRules;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setFlowAnalysisRules(Set<FlowAnalysisRuleEntity> flowAnalysisRules) {
|
||||||
|
this.flowAnalysisRules = flowAnalysisRules;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,87 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.nifi.web.api.entity;
|
||||||
|
|
||||||
|
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.
|
||||||
|
* Used to manage flow analysis rule violations.
|
||||||
|
*/
|
||||||
|
@XmlRootElement(name = "ruleViolationEntity")
|
||||||
|
public class RuleViolationEntity extends Entity {
|
||||||
|
private String scope;
|
||||||
|
private String subjectId;
|
||||||
|
private String ruleId;
|
||||||
|
private String issueId;
|
||||||
|
private Boolean enabled;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the scope of the analysis that produced this result.
|
||||||
|
*/
|
||||||
|
public String getScope() {
|
||||||
|
return scope;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setScope(String scope) {
|
||||||
|
this.scope = scope;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the id of the subject that violated the rule.
|
||||||
|
*/
|
||||||
|
public String getSubjectId() {
|
||||||
|
return subjectId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setSubjectId(String subjectId) {
|
||||||
|
this.subjectId = subjectId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the id of the rule that produced this result
|
||||||
|
*/
|
||||||
|
public String getRuleId() {
|
||||||
|
return ruleId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setRuleId(String ruleId) {
|
||||||
|
this.ruleId = ruleId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a rule-defined id that corresponds to a unique type of issue recognized by the rule.
|
||||||
|
*/
|
||||||
|
public String getIssueId() {
|
||||||
|
return issueId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setIssueId(String issueId) {
|
||||||
|
this.issueId = issueId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return true if this violation should be in effect, false otherwise
|
||||||
|
*/
|
||||||
|
public Boolean getEnabled() {
|
||||||
|
return enabled;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setEnabled(Boolean enabled) {
|
||||||
|
this.enabled = enabled;
|
||||||
|
}
|
||||||
|
}
|
|
@ -22,6 +22,7 @@ import org.apache.nifi.components.ConfigurableComponent;
|
||||||
import org.apache.nifi.controller.ControllerService;
|
import org.apache.nifi.controller.ControllerService;
|
||||||
import org.apache.nifi.documentation.html.HtmlDocumentationWriter;
|
import org.apache.nifi.documentation.html.HtmlDocumentationWriter;
|
||||||
import org.apache.nifi.documentation.html.HtmlProcessorDocumentationWriter;
|
import org.apache.nifi.documentation.html.HtmlProcessorDocumentationWriter;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRule;
|
||||||
import org.apache.nifi.nar.ExtensionDefinition;
|
import org.apache.nifi.nar.ExtensionDefinition;
|
||||||
import org.apache.nifi.nar.ExtensionManager;
|
import org.apache.nifi.nar.ExtensionManager;
|
||||||
import org.apache.nifi.nar.ExtensionMapping;
|
import org.apache.nifi.nar.ExtensionMapping;
|
||||||
|
@ -63,6 +64,7 @@ public class DocGenerator {
|
||||||
documentConfigurableComponent(extensionManager.getExtensions(Processor.class), explodedNiFiDocsDir, extensionManager);
|
documentConfigurableComponent(extensionManager.getExtensions(Processor.class), explodedNiFiDocsDir, extensionManager);
|
||||||
documentConfigurableComponent(extensionManager.getExtensions(ControllerService.class), explodedNiFiDocsDir, extensionManager);
|
documentConfigurableComponent(extensionManager.getExtensions(ControllerService.class), explodedNiFiDocsDir, extensionManager);
|
||||||
documentConfigurableComponent(extensionManager.getExtensions(ReportingTask.class), explodedNiFiDocsDir, extensionManager);
|
documentConfigurableComponent(extensionManager.getExtensions(ReportingTask.class), explodedNiFiDocsDir, extensionManager);
|
||||||
|
documentConfigurableComponent(extensionManager.getExtensions(FlowAnalysisRule.class), explodedNiFiDocsDir, extensionManager);
|
||||||
documentConfigurableComponent(extensionManager.getExtensions(ParameterProvider.class), explodedNiFiDocsDir, extensionManager);
|
documentConfigurableComponent(extensionManager.getExtensions(ParameterProvider.class), explodedNiFiDocsDir, extensionManager);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -575,6 +575,9 @@ public final class ResourceFactory {
|
||||||
case ReportingTask:
|
case ReportingTask:
|
||||||
componentType = "Reporting Task";
|
componentType = "Reporting Task";
|
||||||
break;
|
break;
|
||||||
|
case FlowAnalysisRule:
|
||||||
|
componentType = "Flow Analysis Rule";
|
||||||
|
break;
|
||||||
case Label:
|
case Label:
|
||||||
componentType = "Label";
|
componentType = "Label";
|
||||||
break;
|
break;
|
||||||
|
|
|
@ -34,6 +34,7 @@ public enum ResourceType {
|
||||||
Proxy("/proxy"),
|
Proxy("/proxy"),
|
||||||
RemoteProcessGroup("/remote-process-groups"),
|
RemoteProcessGroup("/remote-process-groups"),
|
||||||
ReportingTask("/reporting-tasks"),
|
ReportingTask("/reporting-tasks"),
|
||||||
|
FlowAnalysisRule("/controller/flow-analysis-rules"),
|
||||||
Resource("/resources"),
|
Resource("/resources"),
|
||||||
SiteToSite("/site-to-site"),
|
SiteToSite("/site-to-site"),
|
||||||
DataTransfer("/data-transfer"),
|
DataTransfer("/data-transfer"),
|
||||||
|
|
|
@ -34,6 +34,10 @@ import org.apache.nifi.cluster.coordination.http.endpoints.CountersEndpointMerge
|
||||||
import org.apache.nifi.cluster.coordination.http.endpoints.CurrentUserEndpointMerger;
|
import org.apache.nifi.cluster.coordination.http.endpoints.CurrentUserEndpointMerger;
|
||||||
import org.apache.nifi.cluster.coordination.http.endpoints.DropAllFlowFilesRequestEndpointMerger;
|
import org.apache.nifi.cluster.coordination.http.endpoints.DropAllFlowFilesRequestEndpointMerger;
|
||||||
import org.apache.nifi.cluster.coordination.http.endpoints.DropRequestEndpointMerger;
|
import org.apache.nifi.cluster.coordination.http.endpoints.DropRequestEndpointMerger;
|
||||||
|
import org.apache.nifi.cluster.coordination.http.endpoints.FlowAnalysisEndpointMerger;
|
||||||
|
import org.apache.nifi.cluster.coordination.http.endpoints.FlowAnalysisRuleEndpointMerger;
|
||||||
|
import org.apache.nifi.cluster.coordination.http.endpoints.FlowAnalysisRuleTypesEndpointMerger;
|
||||||
|
import org.apache.nifi.cluster.coordination.http.endpoints.FlowAnalysisRulesEndpointMerger;
|
||||||
import org.apache.nifi.cluster.coordination.http.endpoints.FlowConfigurationEndpointMerger;
|
import org.apache.nifi.cluster.coordination.http.endpoints.FlowConfigurationEndpointMerger;
|
||||||
import org.apache.nifi.cluster.coordination.http.endpoints.FlowMerger;
|
import org.apache.nifi.cluster.coordination.http.endpoints.FlowMerger;
|
||||||
import org.apache.nifi.cluster.coordination.http.endpoints.FlowRegistryClientEndpointMerger;
|
import org.apache.nifi.cluster.coordination.http.endpoints.FlowRegistryClientEndpointMerger;
|
||||||
|
@ -75,6 +79,7 @@ import org.apache.nifi.cluster.coordination.http.endpoints.ReplayLastEventEndpoi
|
||||||
import org.apache.nifi.cluster.coordination.http.endpoints.ReportingTaskEndpointMerger;
|
import org.apache.nifi.cluster.coordination.http.endpoints.ReportingTaskEndpointMerger;
|
||||||
import org.apache.nifi.cluster.coordination.http.endpoints.ReportingTaskTypesEndpointMerger;
|
import org.apache.nifi.cluster.coordination.http.endpoints.ReportingTaskTypesEndpointMerger;
|
||||||
import org.apache.nifi.cluster.coordination.http.endpoints.ReportingTasksEndpointMerger;
|
import org.apache.nifi.cluster.coordination.http.endpoints.ReportingTasksEndpointMerger;
|
||||||
|
import org.apache.nifi.cluster.coordination.http.endpoints.RuleViolationEndpointMerger;
|
||||||
import org.apache.nifi.cluster.coordination.http.endpoints.RuntimeManifestEndpointMerger;
|
import org.apache.nifi.cluster.coordination.http.endpoints.RuntimeManifestEndpointMerger;
|
||||||
import org.apache.nifi.cluster.coordination.http.endpoints.SearchUsersEndpointMerger;
|
import org.apache.nifi.cluster.coordination.http.endpoints.SearchUsersEndpointMerger;
|
||||||
import org.apache.nifi.cluster.coordination.http.endpoints.StatusHistoryEndpointMerger;
|
import org.apache.nifi.cluster.coordination.http.endpoints.StatusHistoryEndpointMerger;
|
||||||
|
@ -144,6 +149,10 @@ public class StandardHttpResponseMapper implements HttpResponseMapper {
|
||||||
endpointMergers.add(new ControllerServiceReferenceEndpointMerger());
|
endpointMergers.add(new ControllerServiceReferenceEndpointMerger());
|
||||||
endpointMergers.add(new ReportingTaskEndpointMerger());
|
endpointMergers.add(new ReportingTaskEndpointMerger());
|
||||||
endpointMergers.add(new ReportingTasksEndpointMerger());
|
endpointMergers.add(new ReportingTasksEndpointMerger());
|
||||||
|
endpointMergers.add(new FlowAnalysisRuleEndpointMerger());
|
||||||
|
endpointMergers.add(new FlowAnalysisRulesEndpointMerger());
|
||||||
|
endpointMergers.add(new FlowAnalysisEndpointMerger());
|
||||||
|
endpointMergers.add(new RuleViolationEndpointMerger());
|
||||||
endpointMergers.add(new DropRequestEndpointMerger());
|
endpointMergers.add(new DropRequestEndpointMerger());
|
||||||
endpointMergers.add(new DropAllFlowFilesRequestEndpointMerger());
|
endpointMergers.add(new DropAllFlowFilesRequestEndpointMerger());
|
||||||
endpointMergers.add(new ListFlowFilesEndpointMerger());
|
endpointMergers.add(new ListFlowFilesEndpointMerger());
|
||||||
|
@ -156,6 +165,7 @@ public class StandardHttpResponseMapper implements HttpResponseMapper {
|
||||||
endpointMergers.add(new ProcessorTypesEndpointMerger());
|
endpointMergers.add(new ProcessorTypesEndpointMerger());
|
||||||
endpointMergers.add(new ControllerServiceTypesEndpointMerger());
|
endpointMergers.add(new ControllerServiceTypesEndpointMerger());
|
||||||
endpointMergers.add(new ReportingTaskTypesEndpointMerger());
|
endpointMergers.add(new ReportingTaskTypesEndpointMerger());
|
||||||
|
endpointMergers.add(new FlowAnalysisRuleTypesEndpointMerger());
|
||||||
endpointMergers.add(new PrioritizerTypesEndpointMerger());
|
endpointMergers.add(new PrioritizerTypesEndpointMerger());
|
||||||
endpointMergers.add(new ControllerConfigurationEndpointMerger());
|
endpointMergers.add(new ControllerConfigurationEndpointMerger());
|
||||||
endpointMergers.add(new CurrentUserEndpointMerger());
|
endpointMergers.add(new CurrentUserEndpointMerger());
|
||||||
|
|
|
@ -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.cluster.coordination.http.endpoints;
|
||||||
|
|
||||||
|
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||||
|
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||||
|
import org.apache.nifi.web.api.dto.AnalyzeFlowRequestDTO;
|
||||||
|
import org.apache.nifi.web.api.entity.AnalyzeFlowRequestEntity;
|
||||||
|
|
||||||
|
import java.net.URI;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.regex.Pattern;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
public class AnalyzeFlowRequestEndpointMerger extends AbstractSingleDTOEndpoint<AnalyzeFlowRequestEntity, AnalyzeFlowRequestDTO> {
|
||||||
|
public static final Pattern ANALYZE_FLOW_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/flow-analysis/[a-f0-9\\-]{36}");
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean canHandle(URI uri, String method) {
|
||||||
|
if (
|
||||||
|
("POST".equalsIgnoreCase(method) || "GET".equalsIgnoreCase(method) || "DELETE".equalsIgnoreCase(method))
|
||||||
|
&& ANALYZE_FLOW_URI_PATTERN.matcher(uri.getPath()).matches()
|
||||||
|
) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Class<AnalyzeFlowRequestEntity> getEntityClass() {
|
||||||
|
return AnalyzeFlowRequestEntity.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected AnalyzeFlowRequestDTO getDto(AnalyzeFlowRequestEntity entity) {
|
||||||
|
AnalyzeFlowRequestDTO dto = entity.getAnalyzeFlowRequest();
|
||||||
|
|
||||||
|
return dto;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void mergeResponses(
|
||||||
|
AnalyzeFlowRequestDTO clientDto,
|
||||||
|
Map<NodeIdentifier, AnalyzeFlowRequestDTO> dtoMap,
|
||||||
|
Set<NodeResponse> successfulResponses,
|
||||||
|
Set<NodeResponse> problematicResponses
|
||||||
|
) {
|
||||||
|
Collection<String> failureReasons = new HashSet<>();
|
||||||
|
if (clientDto.getFailureReason() != null) {
|
||||||
|
failureReasons.add(clientDto.getFailureReason());
|
||||||
|
}
|
||||||
|
|
||||||
|
for (final AnalyzeFlowRequestDTO requestDto : dtoMap.values()) {
|
||||||
|
if (!requestDto.isComplete()) {
|
||||||
|
clientDto.setComplete(false);
|
||||||
|
}
|
||||||
|
if (requestDto.getFailureReason() != null) {
|
||||||
|
failureReasons.add(requestDto.getFailureReason());
|
||||||
|
}
|
||||||
|
if (requestDto.getLastUpdated() != null && (clientDto.getLastUpdated() == null || requestDto.getLastUpdated().after(clientDto.getLastUpdated()))) {
|
||||||
|
clientDto.setLastUpdated(requestDto.getLastUpdated());
|
||||||
|
}
|
||||||
|
if (requestDto.getPercentCompleted() < clientDto.getPercentCompleted()) {
|
||||||
|
clientDto.setPercentCompleted(requestDto.getPercentCompleted());
|
||||||
|
clientDto.setState(requestDto.getState());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
String failureReason = failureReasons.stream()
|
||||||
|
.filter(Objects::nonNull)
|
||||||
|
.collect(Collectors.joining("\n"));
|
||||||
|
if (!failureReason.isEmpty()) {
|
||||||
|
clientDto.setFailureReason(failureReason);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -37,6 +37,7 @@ public class ComponentStateEndpointMerger extends AbstractSingleDTOEndpoint<Comp
|
||||||
public static final Pattern PROCESSOR_STATE_URI_PATTERN = Pattern.compile("/nifi-api/processors/[a-f0-9\\-]{36}/state");
|
public static final Pattern PROCESSOR_STATE_URI_PATTERN = Pattern.compile("/nifi-api/processors/[a-f0-9\\-]{36}/state");
|
||||||
public static final Pattern CONTROLLER_SERVICE_STATE_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/[a-f0-9\\-]{36}/state");
|
public static final Pattern CONTROLLER_SERVICE_STATE_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/[a-f0-9\\-]{36}/state");
|
||||||
public static final Pattern REPORTING_TASK_STATE_URI_PATTERN = Pattern.compile("/nifi-api/reporting-tasks/[a-f0-9\\-]{36}/state");
|
public static final Pattern REPORTING_TASK_STATE_URI_PATTERN = Pattern.compile("/nifi-api/reporting-tasks/[a-f0-9\\-]{36}/state");
|
||||||
|
public static final Pattern FLOW_ANALYSIS_RULE_STATE_URI_PATTERN = Pattern.compile("/nifi-api/controller/flow-analysis-rules/[a-f0-9\\-]{36}/state");
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean canHandle(URI uri, String method) {
|
public boolean canHandle(URI uri, String method) {
|
||||||
|
@ -46,7 +47,8 @@ public class ComponentStateEndpointMerger extends AbstractSingleDTOEndpoint<Comp
|
||||||
|
|
||||||
return PROCESSOR_STATE_URI_PATTERN.matcher(uri.getPath()).matches()
|
return PROCESSOR_STATE_URI_PATTERN.matcher(uri.getPath()).matches()
|
||||||
|| CONTROLLER_SERVICE_STATE_URI_PATTERN.matcher(uri.getPath()).matches()
|
|| CONTROLLER_SERVICE_STATE_URI_PATTERN.matcher(uri.getPath()).matches()
|
||||||
|| REPORTING_TASK_STATE_URI_PATTERN.matcher(uri.getPath()).matches();
|
|| REPORTING_TASK_STATE_URI_PATTERN.matcher(uri.getPath()).matches()
|
||||||
|
|| FLOW_ANALYSIS_RULE_STATE_URI_PATTERN.matcher(uri.getPath()).matches();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -57,6 +57,7 @@ public class ControllerBulletinsEndpointMerger extends AbstractSingleEntityEndpo
|
||||||
final Map<NodeIdentifier, List<BulletinEntity>> bulletinDtos = new HashMap<>();
|
final Map<NodeIdentifier, List<BulletinEntity>> bulletinDtos = new HashMap<>();
|
||||||
final Map<NodeIdentifier, List<BulletinEntity>> controllerServiceBulletinDtos = new HashMap<>();
|
final Map<NodeIdentifier, List<BulletinEntity>> controllerServiceBulletinDtos = new HashMap<>();
|
||||||
final Map<NodeIdentifier, List<BulletinEntity>> reportingTaskBulletinDtos = new HashMap<>();
|
final Map<NodeIdentifier, List<BulletinEntity>> reportingTaskBulletinDtos = new HashMap<>();
|
||||||
|
final Map<NodeIdentifier, List<BulletinEntity>> flowAnalysisRuleBulletinDtos = new HashMap<>();
|
||||||
final Map<NodeIdentifier, List<BulletinEntity>> flowRegistryClientBulletinDtos = new HashMap<>();
|
final Map<NodeIdentifier, List<BulletinEntity>> flowRegistryClientBulletinDtos = new HashMap<>();
|
||||||
final Map<NodeIdentifier, List<BulletinEntity>> parameterProviderBulletinDtos = new HashMap<>();
|
final Map<NodeIdentifier, List<BulletinEntity>> parameterProviderBulletinDtos = new HashMap<>();
|
||||||
for (final Map.Entry<NodeIdentifier, ControllerBulletinsEntity> entry : entityMap.entrySet()) {
|
for (final Map.Entry<NodeIdentifier, ControllerBulletinsEntity> entry : entityMap.entrySet()) {
|
||||||
|
@ -92,6 +93,15 @@ public class ControllerBulletinsEndpointMerger extends AbstractSingleEntityEndpo
|
||||||
reportingTaskBulletinDtos.computeIfAbsent(nodeIdentifier, nodeId -> new ArrayList<>()).add(bulletin);
|
reportingTaskBulletinDtos.computeIfAbsent(nodeIdentifier, nodeId -> new ArrayList<>()).add(bulletin);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
if (entity.getFlowAnalysisRuleBulletins() != null) {
|
||||||
|
entity.getFlowAnalysisRuleBulletins().forEach(bulletin -> {
|
||||||
|
if (bulletin.getNodeAddress() == null) {
|
||||||
|
bulletin.setNodeAddress(nodeAddress);
|
||||||
|
}
|
||||||
|
|
||||||
|
flowAnalysisRuleBulletinDtos.computeIfAbsent(nodeIdentifier, nodeId -> new ArrayList<>()).add(bulletin);
|
||||||
|
});
|
||||||
|
}
|
||||||
if (entity.getFlowRegistryClientBulletins() != null) {
|
if (entity.getFlowRegistryClientBulletins() != null) {
|
||||||
entity.getFlowRegistryClientBulletins().forEach(bulletin -> {
|
entity.getFlowRegistryClientBulletins().forEach(bulletin -> {
|
||||||
if (bulletin.getNodeAddress() == null) {
|
if (bulletin.getNodeAddress() == null) {
|
||||||
|
@ -115,12 +125,14 @@ public class ControllerBulletinsEndpointMerger extends AbstractSingleEntityEndpo
|
||||||
clientEntity.setBulletins(BulletinMerger.mergeBulletins(bulletinDtos, entityMap.size()));
|
clientEntity.setBulletins(BulletinMerger.mergeBulletins(bulletinDtos, entityMap.size()));
|
||||||
clientEntity.setControllerServiceBulletins(BulletinMerger.mergeBulletins(controllerServiceBulletinDtos, entityMap.size()));
|
clientEntity.setControllerServiceBulletins(BulletinMerger.mergeBulletins(controllerServiceBulletinDtos, entityMap.size()));
|
||||||
clientEntity.setReportingTaskBulletins(BulletinMerger.mergeBulletins(reportingTaskBulletinDtos, entityMap.size()));
|
clientEntity.setReportingTaskBulletins(BulletinMerger.mergeBulletins(reportingTaskBulletinDtos, entityMap.size()));
|
||||||
|
clientEntity.setFlowAnalysisRuleBulletins(BulletinMerger.mergeBulletins(flowAnalysisRuleBulletinDtos, entityMap.size()));
|
||||||
clientEntity.setFlowRegistryClientBulletins(BulletinMerger.mergeBulletins(flowRegistryClientBulletinDtos, entityMap.size()));
|
clientEntity.setFlowRegistryClientBulletins(BulletinMerger.mergeBulletins(flowRegistryClientBulletinDtos, entityMap.size()));
|
||||||
|
|
||||||
// sort the bulletins
|
// sort the bulletins
|
||||||
Collections.sort(clientEntity.getBulletins(), BULLETIN_COMPARATOR);
|
Collections.sort(clientEntity.getBulletins(), BULLETIN_COMPARATOR);
|
||||||
Collections.sort(clientEntity.getControllerServiceBulletins(), BULLETIN_COMPARATOR);
|
Collections.sort(clientEntity.getControllerServiceBulletins(), BULLETIN_COMPARATOR);
|
||||||
Collections.sort(clientEntity.getReportingTaskBulletins(), BULLETIN_COMPARATOR);
|
Collections.sort(clientEntity.getReportingTaskBulletins(), BULLETIN_COMPARATOR);
|
||||||
|
Collections.sort(clientEntity.getFlowAnalysisRuleBulletins(), BULLETIN_COMPARATOR);
|
||||||
Collections.sort(clientEntity.getParameterProviderBulletins(), BULLETIN_COMPARATOR);
|
Collections.sort(clientEntity.getParameterProviderBulletins(), BULLETIN_COMPARATOR);
|
||||||
|
|
||||||
// prune the response to only include the max number of bulletins
|
// prune the response to only include the max number of bulletins
|
||||||
|
@ -133,6 +145,9 @@ public class ControllerBulletinsEndpointMerger extends AbstractSingleEntityEndpo
|
||||||
if (clientEntity.getReportingTaskBulletins().size() > MAX_BULLETINS_PER_COMPONENT) {
|
if (clientEntity.getReportingTaskBulletins().size() > MAX_BULLETINS_PER_COMPONENT) {
|
||||||
clientEntity.setReportingTaskBulletins(clientEntity.getReportingTaskBulletins().subList(0, MAX_BULLETINS_PER_COMPONENT));
|
clientEntity.setReportingTaskBulletins(clientEntity.getReportingTaskBulletins().subList(0, MAX_BULLETINS_PER_COMPONENT));
|
||||||
}
|
}
|
||||||
|
if (clientEntity.getFlowAnalysisRuleBulletins().size() > MAX_BULLETINS_PER_COMPONENT) {
|
||||||
|
clientEntity.setFlowAnalysisRuleBulletins(clientEntity.getFlowAnalysisRuleBulletins().subList(0, MAX_BULLETINS_PER_COMPONENT));
|
||||||
|
}
|
||||||
if (clientEntity.getParameterProviderBulletins().size() > MAX_BULLETINS_PER_COMPONENT) {
|
if (clientEntity.getParameterProviderBulletins().size() > MAX_BULLETINS_PER_COMPONENT) {
|
||||||
clientEntity.setParameterProviderBulletins(clientEntity.getParameterProviderBulletins().subList(0, MAX_BULLETINS_PER_COMPONENT));
|
clientEntity.setParameterProviderBulletins(clientEntity.getParameterProviderBulletins().subList(0, MAX_BULLETINS_PER_COMPONENT));
|
||||||
}
|
}
|
||||||
|
|
|
@ -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.cluster.coordination.http.endpoints;
|
||||||
|
|
||||||
|
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||||
|
import org.apache.nifi.cluster.manager.FlowAnalysisResultEntityMerger;
|
||||||
|
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||||
|
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||||
|
import org.apache.nifi.web.api.entity.FlowAnalysisResultEntity;
|
||||||
|
|
||||||
|
import java.net.URI;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
|
public class FlowAnalysisEndpointMerger extends AbstractSingleEntityEndpoint<FlowAnalysisResultEntity> implements EndpointResponseMerger {
|
||||||
|
public static final String GET_ALL_FLOW_ANALYSIS_RESULTS_URI = "/nifi-api/flow/flow-analysis/result";
|
||||||
|
public static final Pattern GET_GROUP_FLOW_ANALYSIS_RESULTS_URI_PATTERN = Pattern.compile("/nifi-api/flow/flow-analysis/result/[a-f0-9\\-]{36}");
|
||||||
|
|
||||||
|
private final FlowAnalysisResultEntityMerger flowAnalysisResultEntityMerger = new FlowAnalysisResultEntityMerger();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean canHandle(URI uri, String method) {
|
||||||
|
if ("GET".equalsIgnoreCase(method)
|
||||||
|
&& (GET_ALL_FLOW_ANALYSIS_RESULTS_URI.equals(uri.getPath()) || GET_GROUP_FLOW_ANALYSIS_RESULTS_URI_PATTERN.matcher(uri.getPath()).matches())
|
||||||
|
) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Class<FlowAnalysisResultEntity> getEntityClass() {
|
||||||
|
return FlowAnalysisResultEntity.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void mergeResponses(
|
||||||
|
FlowAnalysisResultEntity clientEntity,
|
||||||
|
Map<NodeIdentifier, FlowAnalysisResultEntity> entityMap,
|
||||||
|
Set<NodeResponse> successfulResponses,
|
||||||
|
Set<NodeResponse> problematicResponses
|
||||||
|
) {
|
||||||
|
flowAnalysisResultEntityMerger.merge(clientEntity, entityMap);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,98 @@
|
||||||
|
/*
|
||||||
|
* 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.manager.FlowAnalysisRuleEntityMerger;
|
||||||
|
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||||
|
import org.apache.nifi.web.api.entity.FlowAnalysisRuleEntity;
|
||||||
|
|
||||||
|
import java.net.URI;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
|
public class FlowAnalysisRuleEndpointMerger extends AbstractSingleEntityEndpoint<FlowAnalysisRuleEntity> implements EndpointResponseMerger {
|
||||||
|
private static final Collection<Endpoint> SUPPORTED_ENDPOINTS = Arrays.asList(
|
||||||
|
new Endpoint("/nifi-api/controller/flow-analysis-rules", "POST"),
|
||||||
|
new Endpoint(Pattern.compile("/nifi-api/controller/flow-analysis-rules/[a-f0-9\\-]{36}"), "GET", "PUT", "DELETE"),
|
||||||
|
new Endpoint(Pattern.compile("/nifi-api/controller/flow-analysis-rules/[a-f0-9\\-]{36}/run-status"), "PUT")
|
||||||
|
);
|
||||||
|
|
||||||
|
|
||||||
|
private final FlowAnalysisRuleEntityMerger flowAnalysisRuleEntityMerger = new FlowAnalysisRuleEntityMerger();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean canHandle(URI uri, String method) {
|
||||||
|
boolean canHandle = SUPPORTED_ENDPOINTS.stream()
|
||||||
|
.filter(supportedEndpoint -> supportedEndpoint.canHandle(uri, method))
|
||||||
|
.findAny()
|
||||||
|
.isPresent();
|
||||||
|
|
||||||
|
return canHandle;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Class<FlowAnalysisRuleEntity> getEntityClass() {
|
||||||
|
return FlowAnalysisRuleEntity.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void mergeResponses(
|
||||||
|
FlowAnalysisRuleEntity clientEntity,
|
||||||
|
Map<NodeIdentifier, FlowAnalysisRuleEntity> entityMap,
|
||||||
|
Set<NodeResponse> successfulResponses,
|
||||||
|
Set<NodeResponse> problematicResponses
|
||||||
|
) {
|
||||||
|
flowAnalysisRuleEntityMerger.merge(clientEntity, entityMap);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class Endpoint {
|
||||||
|
final List<String> httpMethods;
|
||||||
|
final String uri;
|
||||||
|
final Pattern uriPattern;
|
||||||
|
|
||||||
|
public Endpoint(final String uri, final String... httpMethods) {
|
||||||
|
this.httpMethods = Arrays.asList(httpMethods);
|
||||||
|
this.uri = uri;
|
||||||
|
this.uriPattern = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Endpoint(Pattern uriPattern, String... httpMethods) {
|
||||||
|
this.httpMethods = Arrays.asList(httpMethods);
|
||||||
|
this.uri = null;
|
||||||
|
this.uriPattern = uriPattern;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean canHandle(URI uri, String method) {
|
||||||
|
boolean canHandle =
|
||||||
|
httpMethods.stream().filter(httpMethod -> httpMethod.equalsIgnoreCase(method)).findAny().isPresent()
|
||||||
|
&& (
|
||||||
|
this.uri != null && this.uri.equals(uri.getPath())
|
||||||
|
||
|
||||||
|
this.uriPattern != null && this.uriPattern.matcher(uri.getPath()).matches()
|
||||||
|
);
|
||||||
|
|
||||||
|
return canHandle;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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.FlowAnalysisRuleTypesEntity;
|
||||||
|
|
||||||
|
import java.net.URI;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
public class FlowAnalysisRuleTypesEndpointMerger extends AbstractNodeStatusEndpoint<FlowAnalysisRuleTypesEntity, Set<DocumentedTypeDTO>> {
|
||||||
|
public static final String FLOW_ANALYSIS_RULE_TYPES_URI_PATTERN = "/nifi-api/flow/flow-analysis-rule-types";
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean canHandle(URI uri, String method) {
|
||||||
|
return "GET".equalsIgnoreCase(method) && FLOW_ANALYSIS_RULE_TYPES_URI_PATTERN.equals(uri.getPath());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Class<FlowAnalysisRuleTypesEntity> getEntityClass() {
|
||||||
|
return FlowAnalysisRuleTypesEntity.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Set<DocumentedTypeDTO> getDto(FlowAnalysisRuleTypesEntity entity) {
|
||||||
|
return entity.getFlowAnalysisRuleTypes();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void mergeResponses(Set<DocumentedTypeDTO> clientDto, Map<NodeIdentifier, Set<DocumentedTypeDTO>> dtoMap, NodeIdentifier selectedNodeId) {
|
||||||
|
DocumentedTypesMerger.mergeDocumentedTypes(clientDto, dtoMap);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,71 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||||
|
|
||||||
|
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||||
|
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||||
|
import org.apache.nifi.cluster.manager.FlowAnalysisRulesEntityMerger;
|
||||||
|
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||||
|
import org.apache.nifi.web.api.entity.FlowAnalysisRuleEntity;
|
||||||
|
import org.apache.nifi.web.api.entity.FlowAnalysisRulesEntity;
|
||||||
|
|
||||||
|
import java.net.URI;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
public class FlowAnalysisRulesEndpointMerger implements EndpointResponseMerger {
|
||||||
|
public static final String FLOW_ANALYSIS_RULES_URI = "/nifi-api/controller/flow-analysis-rules";
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean canHandle(URI uri, String method) {
|
||||||
|
return "GET".equalsIgnoreCase(method) && FLOW_ANALYSIS_RULES_URI.equals(uri.getPath());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final NodeResponse merge(final URI uri, final String method, final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses, final NodeResponse clientResponse) {
|
||||||
|
if (!canHandle(uri, method)) {
|
||||||
|
throw new IllegalArgumentException("Cannot use Endpoint Mapper of type " + getClass().getSimpleName() + " to map responses for URI " + uri + ", HTTP Method " + method);
|
||||||
|
}
|
||||||
|
|
||||||
|
final FlowAnalysisRulesEntity responseEntity = clientResponse.getClientResponse().readEntity(FlowAnalysisRulesEntity.class);
|
||||||
|
final Set<FlowAnalysisRuleEntity> flowAnalysisRulesEntities = responseEntity.getFlowAnalysisRules();
|
||||||
|
|
||||||
|
final Map<String, Map<NodeIdentifier, FlowAnalysisRuleEntity>> entityMap = new HashMap<>();
|
||||||
|
for (final NodeResponse nodeResponse : successfulResponses) {
|
||||||
|
final FlowAnalysisRulesEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().readEntity(FlowAnalysisRulesEntity.class);
|
||||||
|
final Set<FlowAnalysisRuleEntity> nodeFlowAnalysisRuleEntities = nodeResponseEntity.getFlowAnalysisRules();
|
||||||
|
|
||||||
|
for (final FlowAnalysisRuleEntity nodeFlowAnalysisRuleEntity : nodeFlowAnalysisRuleEntities) {
|
||||||
|
final NodeIdentifier nodeId = nodeResponse.getNodeId();
|
||||||
|
Map<NodeIdentifier, FlowAnalysisRuleEntity> innerMap = entityMap.get(nodeId);
|
||||||
|
if (innerMap == null) {
|
||||||
|
innerMap = new HashMap<>();
|
||||||
|
entityMap.put(nodeFlowAnalysisRuleEntity.getId(), innerMap);
|
||||||
|
}
|
||||||
|
|
||||||
|
innerMap.put(nodeResponse.getNodeId(), nodeFlowAnalysisRuleEntity);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
FlowAnalysisRulesEntityMerger.mergeFlowAnalysisRules(flowAnalysisRulesEntities, entityMap);
|
||||||
|
|
||||||
|
// create a new client response
|
||||||
|
return new NodeResponse(clientResponse, responseEntity);
|
||||||
|
}
|
||||||
|
}
|
|
@ -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.RuleViolationEntity;
|
||||||
|
|
||||||
|
import java.net.URI;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
public class RuleViolationEndpointMerger extends AbstractSingleEntityEndpoint<RuleViolationEntity> implements EndpointResponseMerger {
|
||||||
|
public static final String UPDATE_RULE_VIOLATION_URI = "/nifi-api/controller/analyze-flow/update-rule-violation";
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean canHandle(URI uri, String method) {
|
||||||
|
if ("PUT".equalsIgnoreCase(method) && UPDATE_RULE_VIOLATION_URI.equals(uri.getPath())) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Class<RuleViolationEntity> getEntityClass() {
|
||||||
|
return RuleViolationEntity.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void mergeResponses(RuleViolationEntity clientEntity, Map<NodeIdentifier, RuleViolationEntity> entityMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
|
||||||
|
// Nothing to do, if there are no issues the entities are the same
|
||||||
|
}
|
||||||
|
}
|
|
@ -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_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_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_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_FLOW_ANALYSIS_RULE_URI_PATTERN = Pattern.compile("/nifi-api/controller/flow-analysis-rules/[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})?");
|
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
|
@Override
|
||||||
|
@ -45,6 +46,7 @@ public class VerifyConfigEndpointMerger extends AbstractSingleEntityEndpoint<Ver
|
||||||
return VERIFY_PROCESSOR_CONFIG_URI_PATTERN.matcher(uri.getPath()).matches()
|
return VERIFY_PROCESSOR_CONFIG_URI_PATTERN.matcher(uri.getPath()).matches()
|
||||||
|| VERIFY_CONTROLLER_SERVICE_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_FLOW_ANALYSIS_RULE_URI_PATTERN.matcher(uri.getPath()).matches()
|
||||||
|| VERIFY_PARAMETER_PROVIDER_CONFIG_URI_PATTERN.matcher(uri.getPath()).matches();
|
|| VERIFY_PARAMETER_PROVIDER_CONFIG_URI_PATTERN.matcher(uri.getPath()).matches();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,74 @@
|
||||||
|
/*
|
||||||
|
* 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.manager;
|
||||||
|
|
||||||
|
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||||
|
import org.apache.nifi.validation.RuleViolationKey;
|
||||||
|
import org.apache.nifi.web.api.dto.FlowAnalysisRuleDTO;
|
||||||
|
import org.apache.nifi.web.api.dto.FlowAnalysisRuleViolationDTO;
|
||||||
|
import org.apache.nifi.web.api.entity.FlowAnalysisResultEntity;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
public class FlowAnalysisResultEntityMerger {
|
||||||
|
public void merge(FlowAnalysisResultEntity clientEntity, Map<NodeIdentifier, FlowAnalysisResultEntity> entityMap) {
|
||||||
|
List<FlowAnalysisRuleDTO> aggregateRules = clientEntity.getRules();
|
||||||
|
entityMap.values().stream()
|
||||||
|
.map(FlowAnalysisResultEntity::getRules)
|
||||||
|
.forEach(aggregateRules::addAll);
|
||||||
|
|
||||||
|
Map<RuleViolationKey, FlowAnalysisRuleViolationDTO> mergedViolations = clientEntity.getRuleViolations().stream().collect(Collectors.toMap(
|
||||||
|
violation -> new RuleViolationKey(
|
||||||
|
violation.getScope(),
|
||||||
|
violation.getSubjectId(),
|
||||||
|
violation.getRuleId(),
|
||||||
|
violation.getIssueId()
|
||||||
|
),
|
||||||
|
violation -> violation
|
||||||
|
));
|
||||||
|
|
||||||
|
for (final Map.Entry<NodeIdentifier, FlowAnalysisResultEntity> entry : entityMap.entrySet()) {
|
||||||
|
final FlowAnalysisResultEntity entity = entry.getValue();
|
||||||
|
|
||||||
|
entity.getRuleViolations().forEach(violation -> mergedViolations
|
||||||
|
.compute(
|
||||||
|
new RuleViolationKey(
|
||||||
|
violation.getScope(),
|
||||||
|
violation.getSubjectId(),
|
||||||
|
violation.getRuleId(),
|
||||||
|
violation.getIssueId()
|
||||||
|
),
|
||||||
|
(ruleViolationKey, storedViolation) -> {
|
||||||
|
if (storedViolation != null) {
|
||||||
|
PermissionsDtoMerger.mergePermissions(violation.getSubjectPermissionDto(), storedViolation.getSubjectPermissionDto());
|
||||||
|
}
|
||||||
|
|
||||||
|
return violation;
|
||||||
|
}
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
List<FlowAnalysisRuleViolationDTO> authorizedViolations = mergedViolations.values().stream()
|
||||||
|
.filter(violation -> violation.getSubjectPermissionDto().getCanRead())
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
clientEntity.setRuleViolations(authorizedViolations);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,109 @@
|
||||||
|
/*
|
||||||
|
* 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.manager;
|
||||||
|
|
||||||
|
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||||
|
import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
|
||||||
|
import org.apache.nifi.web.api.dto.FlowAnalysisRuleDTO;
|
||||||
|
import org.apache.nifi.web.api.entity.FlowAnalysisRuleEntity;
|
||||||
|
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
public class FlowAnalysisRuleEntityMerger implements ComponentEntityMerger<FlowAnalysisRuleEntity> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void merge(FlowAnalysisRuleEntity clientEntity, Map<NodeIdentifier, FlowAnalysisRuleEntity> entityMap) {
|
||||||
|
ComponentEntityMerger.super.merge(clientEntity, entityMap);
|
||||||
|
for (Map.Entry<NodeIdentifier, FlowAnalysisRuleEntity> entry : entityMap.entrySet()) {
|
||||||
|
final FlowAnalysisRuleEntity entityStatus = entry.getValue();
|
||||||
|
if (clientEntity != entityStatus) {
|
||||||
|
StatusMerger.merge(clientEntity.getStatus(), entityStatus.getStatus());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Merges the FlowAnalysisRuleEntity responses.
|
||||||
|
*
|
||||||
|
* @param clientEntity the entity being returned to the client
|
||||||
|
* @param entityMap all node responses
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void mergeComponents(final FlowAnalysisRuleEntity clientEntity, final Map<NodeIdentifier, FlowAnalysisRuleEntity> entityMap) {
|
||||||
|
final FlowAnalysisRuleDTO clientDto = clientEntity.getComponent();
|
||||||
|
final Map<NodeIdentifier, FlowAnalysisRuleDTO> dtoMap = new HashMap<>();
|
||||||
|
for (final Map.Entry<NodeIdentifier, FlowAnalysisRuleEntity> entry : entityMap.entrySet()) {
|
||||||
|
final FlowAnalysisRuleEntity nodeFlowAnalysisRuleEntity = entry.getValue();
|
||||||
|
final FlowAnalysisRuleDTO nodeFlowAnalysisRuleDto = nodeFlowAnalysisRuleEntity.getComponent();
|
||||||
|
dtoMap.put(entry.getKey(), nodeFlowAnalysisRuleDto);
|
||||||
|
}
|
||||||
|
|
||||||
|
mergeDtos(clientDto, dtoMap);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void mergeDtos(final FlowAnalysisRuleDTO clientDto, final Map<NodeIdentifier, FlowAnalysisRuleDTO> dtoMap) {
|
||||||
|
// if unauthorized for the client dto, simple return
|
||||||
|
if (clientDto == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
final Map<String, Set<NodeIdentifier>> validationErrorMap = new HashMap<>();
|
||||||
|
final Map<String, Map<NodeIdentifier, PropertyDescriptorDTO>> propertyDescriptorMap = new HashMap<>();
|
||||||
|
|
||||||
|
for (final Map.Entry<NodeIdentifier, FlowAnalysisRuleDTO> nodeEntry : dtoMap.entrySet()) {
|
||||||
|
final FlowAnalysisRuleDTO nodeFlowAnalysisRule = nodeEntry.getValue();
|
||||||
|
|
||||||
|
// consider the node flow analysis rule if authorized
|
||||||
|
if (nodeFlowAnalysisRule != null) {
|
||||||
|
final NodeIdentifier nodeId = nodeEntry.getKey();
|
||||||
|
|
||||||
|
// merge the validation errors
|
||||||
|
ErrorMerger.mergeErrors(validationErrorMap, nodeId, nodeFlowAnalysisRule.getValidationErrors());
|
||||||
|
|
||||||
|
// aggregate the property descriptors
|
||||||
|
if (nodeFlowAnalysisRule.getDescriptors() != null) {
|
||||||
|
nodeFlowAnalysisRule.getDescriptors().values().stream().forEach(propertyDescriptor -> {
|
||||||
|
propertyDescriptorMap.computeIfAbsent(propertyDescriptor.getName(), nodeIdToPropertyDescriptor -> new HashMap<>()).put(nodeId, propertyDescriptor);
|
||||||
|
});
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// merge property descriptors
|
||||||
|
for (Map<NodeIdentifier, PropertyDescriptorDTO> propertyDescriptorByNodeId : propertyDescriptorMap.values()) {
|
||||||
|
final Collection<PropertyDescriptorDTO> nodePropertyDescriptors = propertyDescriptorByNodeId.values();
|
||||||
|
if (!nodePropertyDescriptors.isEmpty()) {
|
||||||
|
// get the name of the property descriptor and find that descriptor being returned to the client
|
||||||
|
final PropertyDescriptorDTO propertyDescriptor = nodePropertyDescriptors.iterator().next();
|
||||||
|
final PropertyDescriptorDTO clientPropertyDescriptor = clientDto.getDescriptors().get(propertyDescriptor.getName());
|
||||||
|
PropertyDescriptorDtoMerger.merge(clientPropertyDescriptor, propertyDescriptorByNodeId);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
final Set<String> validationStatuses = dtoMap.values().stream()
|
||||||
|
.map(FlowAnalysisRuleDTO::getValidationStatus)
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
clientDto.setValidationStatus(ErrorMerger.mergeValidationStatus(validationStatuses));
|
||||||
|
|
||||||
|
// set the merged the validation errors
|
||||||
|
clientDto.setValidationErrors(ErrorMerger.normalizedMergedErrors(validationErrorMap, dtoMap.size()));
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,40 @@
|
||||||
|
/*
|
||||||
|
* 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.manager;
|
||||||
|
|
||||||
|
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||||
|
import org.apache.nifi.web.api.entity.FlowAnalysisRuleEntity;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
public class FlowAnalysisRulesEntityMerger {
|
||||||
|
|
||||||
|
private static final FlowAnalysisRuleEntityMerger flowAnalysisRuleEntityMerger = new FlowAnalysisRuleEntityMerger();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Merges multiple FlowAnalysisRuleEntity responses.
|
||||||
|
*
|
||||||
|
* @param flowAnalysisRuleEntities entities being returned to the client
|
||||||
|
* @param entityMap all node responses
|
||||||
|
*/
|
||||||
|
public static void mergeFlowAnalysisRules(final Set<FlowAnalysisRuleEntity> flowAnalysisRuleEntities, final Map<String, Map<NodeIdentifier, FlowAnalysisRuleEntity>> entityMap) {
|
||||||
|
for (final FlowAnalysisRuleEntity entity : flowAnalysisRuleEntities) {
|
||||||
|
flowAnalysisRuleEntityMerger.merge(entity, entityMap.get(entity.getId()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -45,6 +45,7 @@ import org.apache.nifi.web.api.dto.status.ConnectionStatusPredictionsSnapshotDTO
|
||||||
import org.apache.nifi.web.api.dto.status.ConnectionStatusSnapshotDTO;
|
import org.apache.nifi.web.api.dto.status.ConnectionStatusSnapshotDTO;
|
||||||
import org.apache.nifi.web.api.dto.status.ControllerServiceStatusDTO;
|
import org.apache.nifi.web.api.dto.status.ControllerServiceStatusDTO;
|
||||||
import org.apache.nifi.web.api.dto.status.ControllerStatusDTO;
|
import org.apache.nifi.web.api.dto.status.ControllerStatusDTO;
|
||||||
|
import org.apache.nifi.web.api.dto.status.FlowAnalysisRuleStatusDTO;
|
||||||
import org.apache.nifi.web.api.dto.status.NodeConnectionStatusSnapshotDTO;
|
import org.apache.nifi.web.api.dto.status.NodeConnectionStatusSnapshotDTO;
|
||||||
import org.apache.nifi.web.api.dto.status.NodePortStatusSnapshotDTO;
|
import org.apache.nifi.web.api.dto.status.NodePortStatusSnapshotDTO;
|
||||||
import org.apache.nifi.web.api.dto.status.NodeProcessGroupStatusSnapshotDTO;
|
import org.apache.nifi.web.api.dto.status.NodeProcessGroupStatusSnapshotDTO;
|
||||||
|
@ -1060,4 +1061,16 @@ public class StatusMerger {
|
||||||
target.setValidationStatus(ValidationStatus.INVALID.name());
|
target.setValidationStatus(ValidationStatus.INVALID.name());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static void merge(final FlowAnalysisRuleStatusDTO target, final FlowAnalysisRuleStatusDTO toMerge) {
|
||||||
|
if (target == null || toMerge == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (ValidationStatus.VALIDATING.name().equalsIgnoreCase(toMerge.getValidationStatus())) {
|
||||||
|
target.setValidationStatus(ValidationStatus.VALIDATING.name());
|
||||||
|
} else if (ValidationStatus.INVALID.name().equalsIgnoreCase(toMerge.getRunStatus())) {
|
||||||
|
target.setValidationStatus(ValidationStatus.INVALID.name());
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,163 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||||
|
|
||||||
|
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||||
|
import org.apache.nifi.util.EqualsWrapper;
|
||||||
|
import org.apache.nifi.web.api.dto.AnalyzeFlowRequestDTO;
|
||||||
|
import org.junit.jupiter.api.Assertions;
|
||||||
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
|
import org.junit.jupiter.api.Test;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.function.Function;
|
||||||
|
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
|
||||||
|
public class AnalyzeFlowRequestEndpointMergerTest {
|
||||||
|
|
||||||
|
private AnalyzeFlowRequestEndpointMerger testSubject;
|
||||||
|
|
||||||
|
@BeforeEach
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
testSubject = new AnalyzeFlowRequestEndpointMerger();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAllRequestsWaiting() throws Exception {
|
||||||
|
// GIVEN
|
||||||
|
AnalyzeFlowRequestDTO clientDto = createRequest(false, null, "WAITING");
|
||||||
|
|
||||||
|
Map<NodeIdentifier, AnalyzeFlowRequestDTO> dtoMap = new HashMap<>();
|
||||||
|
dtoMap.put(mock(NodeIdentifier.class), createRequest(false, null, "WAITING"));
|
||||||
|
dtoMap.put(mock(NodeIdentifier.class), createRequest(false, null, "WAITING"));
|
||||||
|
|
||||||
|
AnalyzeFlowRequestDTO expected = createRequest(false, null, "WAITING");
|
||||||
|
|
||||||
|
// WHEN
|
||||||
|
testSubject.mergeResponses(clientDto, dtoMap, null, null);
|
||||||
|
|
||||||
|
// THEN
|
||||||
|
this.assertEquals(expected, clientDto);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testClientRequestWaitingOthersComplete() throws Exception {
|
||||||
|
// GIVEN
|
||||||
|
AnalyzeFlowRequestDTO clientDto = createRequest(false, null, "WAITING");
|
||||||
|
|
||||||
|
Map<NodeIdentifier, AnalyzeFlowRequestDTO> dtoMap = new HashMap<>();
|
||||||
|
dtoMap.put(mock(NodeIdentifier.class), createRequest(true, null, "COMPLETE"));
|
||||||
|
dtoMap.put(mock(NodeIdentifier.class), createRequest(true, null, "COMPLETE"));
|
||||||
|
|
||||||
|
AnalyzeFlowRequestDTO expected = createRequest(false, null, "WAITING");
|
||||||
|
|
||||||
|
// WHEN
|
||||||
|
testSubject.mergeResponses(clientDto, dtoMap, null, null);
|
||||||
|
|
||||||
|
// THEN
|
||||||
|
this.assertEquals(expected, clientDto);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testOneNonClientRequestCompleteOthersWaiting() throws Exception {
|
||||||
|
// GIVEN
|
||||||
|
AnalyzeFlowRequestDTO clientDto = createRequest(false, null, "WAITING");
|
||||||
|
|
||||||
|
Map<NodeIdentifier, AnalyzeFlowRequestDTO> dtoMap = new HashMap<>();
|
||||||
|
dtoMap.put(mock(NodeIdentifier.class), createRequest(false, null, "WAITING"));
|
||||||
|
dtoMap.put(mock(NodeIdentifier.class), createRequest(true, null, "COMPLETE"));
|
||||||
|
|
||||||
|
AnalyzeFlowRequestDTO expected = createRequest(false, null, "WAITING");
|
||||||
|
|
||||||
|
// WHEN
|
||||||
|
testSubject.mergeResponses(clientDto, dtoMap, null, null);
|
||||||
|
|
||||||
|
// THEN
|
||||||
|
this.assertEquals(expected, clientDto);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAllRequestsComplete() throws Exception {
|
||||||
|
// GIVEN
|
||||||
|
AnalyzeFlowRequestDTO clientDto = createRequest(true, null, "COMPLETE");
|
||||||
|
|
||||||
|
Map<NodeIdentifier, AnalyzeFlowRequestDTO> dtoMap = new HashMap<>();
|
||||||
|
dtoMap.put(mock(NodeIdentifier.class), createRequest(true, null, "COMPLETE"));
|
||||||
|
dtoMap.put(mock(NodeIdentifier.class), createRequest(true, null, "COMPLETE"));
|
||||||
|
|
||||||
|
AnalyzeFlowRequestDTO expected = createRequest(true, null, "COMPLETE");
|
||||||
|
|
||||||
|
// WHEN
|
||||||
|
testSubject.mergeResponses(clientDto, dtoMap, null, null);
|
||||||
|
|
||||||
|
// THEN
|
||||||
|
this.assertEquals(expected, clientDto);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMergeFailures() throws Exception {
|
||||||
|
// GIVEN
|
||||||
|
AnalyzeFlowRequestDTO clientDto = createRequest(true, "failure1", "FAILURE");
|
||||||
|
|
||||||
|
Map<NodeIdentifier, AnalyzeFlowRequestDTO> dtoMap = new HashMap<>();
|
||||||
|
dtoMap.put(mock(NodeIdentifier.class), createRequest(true, "failure2", "FAILURE"));
|
||||||
|
dtoMap.put(mock(NodeIdentifier.class), createRequest(true, "failure3", "FAILURE"));
|
||||||
|
|
||||||
|
Set<String> expectedFailures = new HashSet<>(Arrays.asList("failure1", "failure2", "failure3"));
|
||||||
|
|
||||||
|
// WHEN
|
||||||
|
testSubject.mergeResponses(clientDto, dtoMap, null, null);
|
||||||
|
|
||||||
|
// THEN
|
||||||
|
HashSet<String> actualFailures = new HashSet<>(Arrays.asList(clientDto.getFailureReason().split("\n")));
|
||||||
|
|
||||||
|
Assertions.assertEquals(expectedFailures, actualFailures);
|
||||||
|
}
|
||||||
|
|
||||||
|
private AnalyzeFlowRequestDTO createRequest(
|
||||||
|
Boolean complete,
|
||||||
|
String failureReason,
|
||||||
|
String state
|
||||||
|
) {
|
||||||
|
AnalyzeFlowRequestDTO request = new AnalyzeFlowRequestDTO();
|
||||||
|
|
||||||
|
request.setComplete(complete);
|
||||||
|
request.setFailureReason(failureReason);
|
||||||
|
request.setState(state);
|
||||||
|
|
||||||
|
return request;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void assertEquals(AnalyzeFlowRequestDTO expected, AnalyzeFlowRequestDTO clientDto) {
|
||||||
|
List<Function<AnalyzeFlowRequestDTO, Object>> equalityCheckers = Arrays.asList(
|
||||||
|
AnalyzeFlowRequestDTO::isComplete,
|
||||||
|
AnalyzeFlowRequestDTO::getFailureReason,
|
||||||
|
AnalyzeFlowRequestDTO::getState
|
||||||
|
);
|
||||||
|
|
||||||
|
Assertions.assertEquals(
|
||||||
|
new EqualsWrapper<>(expected, equalityCheckers),
|
||||||
|
new EqualsWrapper<>(clientDto, equalityCheckers)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,258 @@
|
||||||
|
/*
|
||||||
|
* 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.manager;
|
||||||
|
|
||||||
|
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||||
|
import org.apache.nifi.util.EqualsWrapper;
|
||||||
|
import org.apache.nifi.web.api.dto.FlowAnalysisRuleDTO;
|
||||||
|
import org.apache.nifi.web.api.dto.FlowAnalysisRuleViolationDTO;
|
||||||
|
import org.apache.nifi.web.api.dto.PermissionsDTO;
|
||||||
|
import org.apache.nifi.web.api.entity.FlowAnalysisResultEntity;
|
||||||
|
import org.jetbrains.annotations.NotNull;
|
||||||
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
|
import org.junit.jupiter.api.Test;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.function.Function;
|
||||||
|
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
|
||||||
|
public class FlowAnalysisResultEntityMergerTest {
|
||||||
|
public static final NodeIdentifier NODE_ID_1 = nodeIdOf("id1");
|
||||||
|
public static final NodeIdentifier NODE_ID_2 = nodeIdOf("id2");
|
||||||
|
|
||||||
|
private FlowAnalysisResultEntityMerger testSubject;
|
||||||
|
|
||||||
|
@BeforeEach
|
||||||
|
void setUp() {
|
||||||
|
testSubject = new FlowAnalysisResultEntityMerger();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void differentViolationsAreMerged() {
|
||||||
|
// GIVEN
|
||||||
|
FlowAnalysisResultEntity clientEntity = resultEntityOf(
|
||||||
|
listOf(ruleOf("ruleId")),
|
||||||
|
listOf(ruleViolationOf("ruleId", true, true))
|
||||||
|
);
|
||||||
|
|
||||||
|
Map<NodeIdentifier, FlowAnalysisResultEntity> entityMap = resultEntityMapOf(
|
||||||
|
resultEntityOf(
|
||||||
|
listOf(ruleOf("ruleId1")),
|
||||||
|
listOf(ruleViolationOf("ruleId1", true, true))
|
||||||
|
),
|
||||||
|
resultEntityOf(
|
||||||
|
listOf(ruleOf("ruleId2")),
|
||||||
|
listOf(ruleViolationOf("ruleId2", true, true))
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
FlowAnalysisResultEntity expectedClientEntity = resultEntityOf(
|
||||||
|
listOf(ruleOf("ruleId"), ruleOf("ruleId1"), ruleOf("ruleId2")),
|
||||||
|
listOf(
|
||||||
|
ruleViolationOf("ruleId", true, true),
|
||||||
|
ruleViolationOf("ruleId1", true, true),
|
||||||
|
ruleViolationOf("ruleId2", true, true)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
testMerge(clientEntity, entityMap, expectedClientEntity);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void violationThatCannotBeReadOnAnyNodeIsOmitted() {
|
||||||
|
// GIVEN
|
||||||
|
String ruleId = "ruleWithViolationThatCantBeReadOnOneNode";
|
||||||
|
|
||||||
|
FlowAnalysisResultEntity clientEntity = resultEntityOf(
|
||||||
|
listOf(ruleOf(ruleId)),
|
||||||
|
listOf(ruleViolationOf(ruleId, true, true))
|
||||||
|
);
|
||||||
|
|
||||||
|
Map<NodeIdentifier, FlowAnalysisResultEntity> entityMap = resultEntityMapOf(
|
||||||
|
resultEntityOf(
|
||||||
|
listOf(ruleOf(ruleId)),
|
||||||
|
listOf(ruleViolationOf(ruleId, false, true))
|
||||||
|
),
|
||||||
|
resultEntityOf(
|
||||||
|
listOf(ruleOf(ruleId)),
|
||||||
|
listOf(ruleViolationOf(ruleId, true, true))
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
FlowAnalysisResultEntity expectedClientEntity = resultEntityOf(
|
||||||
|
listOf(ruleOf(ruleId)),
|
||||||
|
listOf()
|
||||||
|
);
|
||||||
|
|
||||||
|
testMerge(clientEntity, entityMap, expectedClientEntity);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void evenWhenViolationIsOmittedTheRuleIsNot() {
|
||||||
|
// GIVEN
|
||||||
|
FlowAnalysisResultEntity clientEntity = resultEntityOf(
|
||||||
|
listOf(),
|
||||||
|
listOf()
|
||||||
|
);
|
||||||
|
|
||||||
|
Map<NodeIdentifier, FlowAnalysisResultEntity> entityMap = resultEntityMapOf(
|
||||||
|
resultEntityOf(
|
||||||
|
listOf(ruleOf("notOmittedRuleButOmittedViolation")),
|
||||||
|
listOf(ruleViolationOf("notOmittedRuleButOmittedViolation", false, true))
|
||||||
|
),
|
||||||
|
resultEntityOf(
|
||||||
|
listOf(),
|
||||||
|
listOf()
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
FlowAnalysisResultEntity expectedClientEntity = resultEntityOf(
|
||||||
|
listOf(ruleOf("notOmittedRuleButOmittedViolation")),
|
||||||
|
listOf()
|
||||||
|
);
|
||||||
|
|
||||||
|
testMerge(clientEntity, entityMap, expectedClientEntity);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void violationThatCannotBeWrittenIsNotOmitted() {
|
||||||
|
// GIVEN
|
||||||
|
String ruleId = "ruleWithViolationThatCantBeWrittenOnOneNode";
|
||||||
|
|
||||||
|
FlowAnalysisResultEntity clientEntity = resultEntityOf(
|
||||||
|
listOf(ruleOf(ruleId)),
|
||||||
|
listOf(ruleViolationOf(ruleId, true, false))
|
||||||
|
);
|
||||||
|
|
||||||
|
Map<NodeIdentifier, FlowAnalysisResultEntity> entityMap = resultEntityMapOf(
|
||||||
|
resultEntityOf(
|
||||||
|
listOf(ruleOf(ruleId)),
|
||||||
|
listOf(ruleViolationOf(ruleId, true, false))
|
||||||
|
),
|
||||||
|
resultEntityOf(
|
||||||
|
listOf(ruleOf(ruleId)),
|
||||||
|
listOf(ruleViolationOf(ruleId, true, false))
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
FlowAnalysisResultEntity expectedClientEntity = clientEntity;
|
||||||
|
|
||||||
|
testMerge(clientEntity, entityMap, expectedClientEntity);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testMerge(FlowAnalysisResultEntity clientEntity, Map<NodeIdentifier, FlowAnalysisResultEntity> entityMap, FlowAnalysisResultEntity expectedClientEntity) {
|
||||||
|
// GIVEN
|
||||||
|
List<Function<FlowAnalysisRuleDTO, Object>> rulePropertiesProviders = Arrays.asList(FlowAnalysisRuleDTO::getId);
|
||||||
|
List<Function<FlowAnalysisRuleViolationDTO, Object>> list = Arrays.asList(
|
||||||
|
FlowAnalysisRuleViolationDTO::getRuleId,
|
||||||
|
FlowAnalysisRuleViolationDTO::isEnabled,
|
||||||
|
ruleViolation -> ruleViolation.getSubjectPermissionDto().getCanRead(),
|
||||||
|
ruleViolation -> ruleViolation.getSubjectPermissionDto().getCanWrite()
|
||||||
|
);
|
||||||
|
List<Function<FlowAnalysisResultEntity, Object>> resultEntityEqualsPropertiesProviders = Arrays.asList(
|
||||||
|
resultEntity -> new HashSet<>(EqualsWrapper.wrapList(resultEntity.getRules(), rulePropertiesProviders)),
|
||||||
|
resultEntity -> new HashSet<>(EqualsWrapper.wrapList(resultEntity.getRuleViolations(), list))
|
||||||
|
);
|
||||||
|
|
||||||
|
// WHEN
|
||||||
|
testSubject.merge(clientEntity, entityMap);
|
||||||
|
|
||||||
|
// THEN
|
||||||
|
assertEquals(new EqualsWrapper<>(
|
||||||
|
expectedClientEntity,
|
||||||
|
resultEntityEqualsPropertiesProviders
|
||||||
|
), new EqualsWrapper<>(
|
||||||
|
clientEntity,
|
||||||
|
resultEntityEqualsPropertiesProviders
|
||||||
|
));
|
||||||
|
}
|
||||||
|
|
||||||
|
@NotNull
|
||||||
|
private static NodeIdentifier nodeIdOf(String nodeId) {
|
||||||
|
NodeIdentifier nodeIdentifier = new NodeIdentifier(nodeId, "unimportant", 1, "unimportant", 1, "unimportant", 1, 1, false);
|
||||||
|
return nodeIdentifier;
|
||||||
|
}
|
||||||
|
|
||||||
|
@NotNull
|
||||||
|
private static FlowAnalysisRuleDTO ruleOf(String ruleId) {
|
||||||
|
FlowAnalysisRuleDTO rule = new FlowAnalysisRuleDTO();
|
||||||
|
|
||||||
|
rule.setId(ruleId);
|
||||||
|
|
||||||
|
return rule;
|
||||||
|
}
|
||||||
|
|
||||||
|
@NotNull
|
||||||
|
private static FlowAnalysisRuleViolationDTO ruleViolationOf(
|
||||||
|
String ruleId,
|
||||||
|
boolean canRead,
|
||||||
|
boolean canWrite
|
||||||
|
) {
|
||||||
|
FlowAnalysisRuleViolationDTO ruleViolation = new FlowAnalysisRuleViolationDTO();
|
||||||
|
|
||||||
|
ruleViolation.setRuleId(ruleId);
|
||||||
|
ruleViolation.setSubjectPermissionDto(permissionOf(canRead, canWrite));
|
||||||
|
|
||||||
|
return ruleViolation;
|
||||||
|
}
|
||||||
|
|
||||||
|
@NotNull
|
||||||
|
private static PermissionsDTO permissionOf(boolean canRead, boolean canWrite) {
|
||||||
|
PermissionsDTO subjectPermissionDto = new PermissionsDTO();
|
||||||
|
|
||||||
|
subjectPermissionDto.setCanRead(canRead);
|
||||||
|
subjectPermissionDto.setCanWrite(canWrite);
|
||||||
|
|
||||||
|
return subjectPermissionDto;
|
||||||
|
}
|
||||||
|
|
||||||
|
@NotNull
|
||||||
|
private static FlowAnalysisResultEntity resultEntityOf(List<FlowAnalysisRuleDTO> rules, List<FlowAnalysisRuleViolationDTO> ruleViolations) {
|
||||||
|
FlowAnalysisResultEntity clientEntity = new FlowAnalysisResultEntity();
|
||||||
|
|
||||||
|
clientEntity.setRules(rules);
|
||||||
|
clientEntity.setRuleViolations(ruleViolations);
|
||||||
|
|
||||||
|
return clientEntity;
|
||||||
|
}
|
||||||
|
|
||||||
|
@NotNull
|
||||||
|
private static Map<NodeIdentifier, FlowAnalysisResultEntity> resultEntityMapOf(FlowAnalysisResultEntity clientEntity1, FlowAnalysisResultEntity clientEntity2) {
|
||||||
|
Map<NodeIdentifier, FlowAnalysisResultEntity> entityMap = new HashMap<>();
|
||||||
|
|
||||||
|
entityMap.put(NODE_ID_1, clientEntity1);
|
||||||
|
entityMap.put(NODE_ID_2, clientEntity2);
|
||||||
|
|
||||||
|
return entityMap;
|
||||||
|
}
|
||||||
|
|
||||||
|
@NotNull
|
||||||
|
private static <T> List<T> listOf(T... items) {
|
||||||
|
List<T> itemSet = new ArrayList<>();
|
||||||
|
for (T item : items) {
|
||||||
|
itemSet.add(item);
|
||||||
|
|
||||||
|
}
|
||||||
|
return itemSet;
|
||||||
|
}
|
||||||
|
}
|
|
@ -26,9 +26,11 @@ import org.apache.nifi.connectable.Connection;
|
||||||
import org.apache.nifi.connectable.Funnel;
|
import org.apache.nifi.connectable.Funnel;
|
||||||
import org.apache.nifi.connectable.Port;
|
import org.apache.nifi.connectable.Port;
|
||||||
import org.apache.nifi.controller.ParameterProviderNode;
|
import org.apache.nifi.controller.ParameterProviderNode;
|
||||||
|
import org.apache.nifi.controller.FlowAnalysisRuleNode;
|
||||||
import org.apache.nifi.controller.ProcessScheduler;
|
import org.apache.nifi.controller.ProcessScheduler;
|
||||||
import org.apache.nifi.controller.ProcessorNode;
|
import org.apache.nifi.controller.ProcessorNode;
|
||||||
import org.apache.nifi.controller.ReportingTaskNode;
|
import org.apache.nifi.controller.ReportingTaskNode;
|
||||||
|
import org.apache.nifi.controller.flowanalysis.FlowAnalyzer;
|
||||||
import org.apache.nifi.controller.repository.FlowFileEventRepository;
|
import org.apache.nifi.controller.repository.FlowFileEventRepository;
|
||||||
import org.apache.nifi.controller.service.ControllerServiceNode;
|
import org.apache.nifi.controller.service.ControllerServiceNode;
|
||||||
import org.apache.nifi.controller.service.ControllerServiceProvider;
|
import org.apache.nifi.controller.service.ControllerServiceProvider;
|
||||||
|
@ -49,6 +51,7 @@ import org.apache.nifi.registry.flow.FlowRegistryClientNode;
|
||||||
import org.apache.nifi.remote.PublicPort;
|
import org.apache.nifi.remote.PublicPort;
|
||||||
import org.apache.nifi.remote.RemoteGroupPort;
|
import org.apache.nifi.remote.RemoteGroupPort;
|
||||||
import org.apache.nifi.util.ReflectionUtils;
|
import org.apache.nifi.util.ReflectionUtils;
|
||||||
|
import org.apache.nifi.validation.RuleViolationsManager;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
@ -74,6 +77,7 @@ public abstract class AbstractFlowManager implements FlowManager {
|
||||||
private final ConcurrentMap<String, Port> allOutputPorts = new ConcurrentHashMap<>();
|
private final ConcurrentMap<String, Port> allOutputPorts = new ConcurrentHashMap<>();
|
||||||
private final ConcurrentMap<String, Funnel> allFunnels = new ConcurrentHashMap<>();
|
private final ConcurrentMap<String, Funnel> allFunnels = new ConcurrentHashMap<>();
|
||||||
private final ConcurrentMap<String, ReportingTaskNode> allReportingTasks = new ConcurrentHashMap<>();
|
private final ConcurrentMap<String, ReportingTaskNode> allReportingTasks = new ConcurrentHashMap<>();
|
||||||
|
private final ConcurrentMap<String, FlowAnalysisRuleNode> allFlowAnalysisRules = new ConcurrentHashMap<>();
|
||||||
private final ConcurrentMap<String, ParameterProviderNode> allParameterProviders = new ConcurrentHashMap<>();
|
private final ConcurrentMap<String, ParameterProviderNode> allParameterProviders = new ConcurrentHashMap<>();
|
||||||
private final ConcurrentMap<String, FlowRegistryClientNode> allFlowRegistryClients = new ConcurrentHashMap<>();
|
private final ConcurrentMap<String, FlowRegistryClientNode> allFlowRegistryClients = new ConcurrentHashMap<>();
|
||||||
|
|
||||||
|
@ -83,6 +87,8 @@ public abstract class AbstractFlowManager implements FlowManager {
|
||||||
|
|
||||||
private volatile ControllerServiceProvider controllerServiceProvider;
|
private volatile ControllerServiceProvider controllerServiceProvider;
|
||||||
private volatile PythonBridge pythonBridge;
|
private volatile PythonBridge pythonBridge;
|
||||||
|
private volatile FlowAnalyzer flowAnalyzer;
|
||||||
|
private volatile RuleViolationsManager ruleViolationsManager;
|
||||||
private volatile ProcessGroup rootGroup;
|
private volatile ProcessGroup rootGroup;
|
||||||
|
|
||||||
private final ThreadLocal<Boolean> withParameterContextResolution = ThreadLocal.withInitial(() -> false);
|
private final ThreadLocal<Boolean> withParameterContextResolution = ThreadLocal.withInitial(() -> false);
|
||||||
|
@ -94,9 +100,16 @@ public abstract class AbstractFlowManager implements FlowManager {
|
||||||
this.flowInitializedCheck = flowInitializedCheck;
|
this.flowInitializedCheck = flowInitializedCheck;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void initialize(final ControllerServiceProvider controllerServiceProvider, final PythonBridge pythonBridge) {
|
public void initialize(
|
||||||
|
final ControllerServiceProvider controllerServiceProvider,
|
||||||
|
final PythonBridge pythonBridge,
|
||||||
|
final FlowAnalyzer flowAnalyzer,
|
||||||
|
final RuleViolationsManager ruleViolationsManager
|
||||||
|
) {
|
||||||
this.controllerServiceProvider = controllerServiceProvider;
|
this.controllerServiceProvider = controllerServiceProvider;
|
||||||
this.pythonBridge = pythonBridge;
|
this.pythonBridge = pythonBridge;
|
||||||
|
this.flowAnalyzer = flowAnalyzer;
|
||||||
|
this.ruleViolationsManager = ruleViolationsManager;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ProcessGroup getGroup(final String id) {
|
public ProcessGroup getGroup(final String id) {
|
||||||
|
@ -108,7 +121,10 @@ public abstract class AbstractFlowManager implements FlowManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void onProcessGroupRemoved(final ProcessGroup group) {
|
public void onProcessGroupRemoved(final ProcessGroup group) {
|
||||||
allProcessGroups.remove(group.getIdentifier());
|
String identifier = group.getIdentifier();
|
||||||
|
allProcessGroups.remove(identifier);
|
||||||
|
|
||||||
|
ruleViolationsManager.removeRuleViolationsForSubject(identifier);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void onProcessorAdded(final ProcessorNode procNode) {
|
public void onProcessorAdded(final ProcessorNode procNode) {
|
||||||
|
@ -120,6 +136,8 @@ public abstract class AbstractFlowManager implements FlowManager {
|
||||||
flowFileEventRepository.purgeTransferEvents(identifier);
|
flowFileEventRepository.purgeTransferEvents(identifier);
|
||||||
allProcessors.remove(identifier);
|
allProcessors.remove(identifier);
|
||||||
pythonBridge.onProcessorRemoved(identifier, procNode.getComponentType(), procNode.getBundleCoordinate().getVersion());
|
pythonBridge.onProcessorRemoved(identifier, procNode.getComponentType(), procNode.getBundleCoordinate().getVersion());
|
||||||
|
|
||||||
|
ruleViolationsManager.removeRuleViolationsForSubject(identifier);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Set<ProcessorNode> findAllProcessors(final Predicate<ProcessorNode> filter) {
|
public Set<ProcessorNode> findAllProcessors(final Predicate<ProcessorNode> filter) {
|
||||||
|
@ -178,6 +196,8 @@ public abstract class AbstractFlowManager implements FlowManager {
|
||||||
String identifier = connection.getIdentifier();
|
String identifier = connection.getIdentifier();
|
||||||
flowFileEventRepository.purgeTransferEvents(identifier);
|
flowFileEventRepository.purgeTransferEvents(identifier);
|
||||||
allConnections.remove(identifier);
|
allConnections.remove(identifier);
|
||||||
|
|
||||||
|
ruleViolationsManager.removeRuleViolationsForSubject(identifier);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Connection getConnection(final String id) {
|
public Connection getConnection(final String id) {
|
||||||
|
@ -226,6 +246,7 @@ public abstract class AbstractFlowManager implements FlowManager {
|
||||||
componentCounts.put("Processors", allProcessors.size());
|
componentCounts.put("Processors", allProcessors.size());
|
||||||
componentCounts.put("Controller Services", getAllControllerServices().size());
|
componentCounts.put("Controller Services", getAllControllerServices().size());
|
||||||
componentCounts.put("Reporting Tasks", getAllReportingTasks().size());
|
componentCounts.put("Reporting Tasks", getAllReportingTasks().size());
|
||||||
|
componentCounts.put("Flow Analysis Rules", getAllFlowAnalysisRules().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("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("Remote Process Groups", getRootGroup().findAllRemoteProcessGroups().size());
|
||||||
componentCounts.put("Parameter Providers", getAllParameterProviders().size());
|
componentCounts.put("Parameter Providers", getAllParameterProviders().size());
|
||||||
|
@ -286,6 +307,7 @@ public abstract class AbstractFlowManager implements FlowManager {
|
||||||
|
|
||||||
getRootControllerServices().forEach(this::removeRootControllerService);
|
getRootControllerServices().forEach(this::removeRootControllerService);
|
||||||
getAllReportingTasks().forEach(this::removeReportingTask);
|
getAllReportingTasks().forEach(this::removeReportingTask);
|
||||||
|
getAllFlowAnalysisRules().forEach(this::removeFlowAnalysisRule);
|
||||||
getAllParameterProviders().forEach(this::removeParameterProvider);
|
getAllParameterProviders().forEach(this::removeParameterProvider);
|
||||||
|
|
||||||
getAllFlowRegistryClients().forEach(this::removeFlowRegistryClientNode);
|
getAllFlowRegistryClients().forEach(this::removeFlowRegistryClientNode);
|
||||||
|
@ -306,6 +328,10 @@ public abstract class AbstractFlowManager implements FlowManager {
|
||||||
reportingTask.verifyCanDelete();
|
reportingTask.verifyCanDelete();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
for (final FlowAnalysisRuleNode flowAnalysisRule : getAllFlowAnalysisRules()) {
|
||||||
|
flowAnalysisRule.verifyCanDelete();
|
||||||
|
}
|
||||||
|
|
||||||
for (final ParameterProviderNode parameterProvider : getAllParameterProviders()) {
|
for (final ParameterProviderNode parameterProvider : getAllParameterProviders()) {
|
||||||
parameterProvider.verifyCanDelete();
|
parameterProvider.verifyCanDelete();
|
||||||
}
|
}
|
||||||
|
@ -333,6 +359,8 @@ public abstract class AbstractFlowManager implements FlowManager {
|
||||||
String identifier = inputPort.getIdentifier();
|
String identifier = inputPort.getIdentifier();
|
||||||
flowFileEventRepository.purgeTransferEvents(identifier);
|
flowFileEventRepository.purgeTransferEvents(identifier);
|
||||||
allInputPorts.remove(identifier);
|
allInputPorts.remove(identifier);
|
||||||
|
|
||||||
|
ruleViolationsManager.removeRuleViolationsForSubject(identifier);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Port getInputPort(final String id) {
|
public Port getInputPort(final String id) {
|
||||||
|
@ -347,6 +375,8 @@ public abstract class AbstractFlowManager implements FlowManager {
|
||||||
String identifier = outputPort.getIdentifier();
|
String identifier = outputPort.getIdentifier();
|
||||||
flowFileEventRepository.purgeTransferEvents(identifier);
|
flowFileEventRepository.purgeTransferEvents(identifier);
|
||||||
allOutputPorts.remove(identifier);
|
allOutputPorts.remove(identifier);
|
||||||
|
|
||||||
|
ruleViolationsManager.removeRuleViolationsForSubject(identifier);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Port getOutputPort(final String id) {
|
public Port getOutputPort(final String id) {
|
||||||
|
@ -361,6 +391,8 @@ public abstract class AbstractFlowManager implements FlowManager {
|
||||||
String identifier = funnel.getIdentifier();
|
String identifier = funnel.getIdentifier();
|
||||||
flowFileEventRepository.purgeTransferEvents(identifier);
|
flowFileEventRepository.purgeTransferEvents(identifier);
|
||||||
allFunnels.remove(identifier);
|
allFunnels.remove(identifier);
|
||||||
|
|
||||||
|
ruleViolationsManager.removeRuleViolationsForSubject(identifier);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Funnel getFunnel(final String id) {
|
public Funnel getFunnel(final String id) {
|
||||||
|
@ -435,6 +467,58 @@ public abstract class AbstractFlowManager implements FlowManager {
|
||||||
allReportingTasks.put(taskNode.getIdentifier(), taskNode);
|
allReportingTasks.put(taskNode.getIdentifier(), taskNode);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FlowAnalysisRuleNode createFlowAnalysisRule(final String type, final String id, final BundleCoordinate bundleCoordinate, final boolean firstTimeAdded) {
|
||||||
|
return createFlowAnalysisRule(type, id, bundleCoordinate, Collections.emptySet(), firstTimeAdded, true, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FlowAnalysisRuleNode getFlowAnalysisRuleNode(final String taskId) {
|
||||||
|
return allFlowAnalysisRules.get(taskId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void removeFlowAnalysisRule(final FlowAnalysisRuleNode flowAnalysisRuleNode) {
|
||||||
|
final FlowAnalysisRuleNode existing = allFlowAnalysisRules.get(flowAnalysisRuleNode.getIdentifier());
|
||||||
|
if (existing == null || existing != flowAnalysisRuleNode) {
|
||||||
|
throw new IllegalStateException("Flow Analysis Rule " + flowAnalysisRuleNode + " does not exist in this Flow");
|
||||||
|
}
|
||||||
|
|
||||||
|
flowAnalysisRuleNode.verifyCanDelete();
|
||||||
|
|
||||||
|
final Class<?> taskClass = flowAnalysisRuleNode.getFlowAnalysisRule().getClass();
|
||||||
|
try (final NarCloseable x = NarCloseable.withComponentNarLoader(getExtensionManager(), taskClass, flowAnalysisRuleNode.getFlowAnalysisRule().getIdentifier())) {
|
||||||
|
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, flowAnalysisRuleNode.getFlowAnalysisRule(), flowAnalysisRuleNode.getConfigurationContext());
|
||||||
|
}
|
||||||
|
|
||||||
|
for (final Map.Entry<PropertyDescriptor, String> entry : flowAnalysisRuleNode.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(flowAnalysisRuleNode, descriptor);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
allFlowAnalysisRules.remove(flowAnalysisRuleNode.getIdentifier());
|
||||||
|
LogRepositoryFactory.removeRepository(flowAnalysisRuleNode.getIdentifier());
|
||||||
|
|
||||||
|
getExtensionManager().removeInstanceClassLoader(flowAnalysisRuleNode.getIdentifier());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<FlowAnalysisRuleNode> getAllFlowAnalysisRules() {
|
||||||
|
return new HashSet<>(allFlowAnalysisRules.values());
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void onFlowAnalysisRuleAdded(final FlowAnalysisRuleNode flowAnalysisRuleNode) {
|
||||||
|
allFlowAnalysisRules.put(flowAnalysisRuleNode.getIdentifier(), flowAnalysisRuleNode);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ParameterProviderNode getParameterProvider(final String id) {
|
public ParameterProviderNode getParameterProvider(final String id) {
|
||||||
return id == null ? null : allParameterProviders.get(id);
|
return id == null ? null : allParameterProviders.get(id);
|
||||||
|
@ -607,4 +691,14 @@ public abstract class AbstractFlowManager implements FlowManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
protected abstract Authorizable getParameterContextParent();
|
protected abstract Authorizable getParameterContextParent();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FlowAnalyzer getFlowAnalyzer() {
|
||||||
|
return flowAnalyzer;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RuleViolationsManager getRuleViolationsManager() {
|
||||||
|
return ruleViolationsManager;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,104 @@
|
||||||
|
/*
|
||||||
|
* 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.flowanalysis;
|
||||||
|
|
||||||
|
import org.apache.nifi.attribute.expression.language.PreparedQuery;
|
||||||
|
import org.apache.nifi.attribute.expression.language.Query;
|
||||||
|
import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
|
||||||
|
import org.apache.nifi.components.PropertyDescriptor;
|
||||||
|
import org.apache.nifi.components.PropertyValue;
|
||||||
|
import org.apache.nifi.components.resource.ResourceContext;
|
||||||
|
import org.apache.nifi.components.resource.StandardResourceContext;
|
||||||
|
import org.apache.nifi.components.resource.StandardResourceReferenceFactory;
|
||||||
|
import org.apache.nifi.controller.FlowAnalysisRuleNode;
|
||||||
|
import org.apache.nifi.controller.service.ControllerServiceProvider;
|
||||||
|
import org.apache.nifi.parameter.ParameterLookup;
|
||||||
|
import org.apache.nifi.registry.VariableRegistry;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRuleContext;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRule;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public abstract class AbstractFlowAnalysisRuleContext implements FlowAnalysisRuleContext {
|
||||||
|
private final FlowAnalysisRuleNode flowAnalysisRuleNode;
|
||||||
|
private final Map<PropertyDescriptor, String> properties;
|
||||||
|
private final ControllerServiceProvider serviceProvider;
|
||||||
|
private final Map<PropertyDescriptor, PreparedQuery> preparedQueries;
|
||||||
|
private final ParameterLookup parameterLookup;
|
||||||
|
private final VariableRegistry variableRegistry;
|
||||||
|
|
||||||
|
public AbstractFlowAnalysisRuleContext(
|
||||||
|
FlowAnalysisRuleNode flowAnalysisRule,
|
||||||
|
Map<PropertyDescriptor, String> properties,
|
||||||
|
ControllerServiceProvider controllerServiceProvider,
|
||||||
|
ParameterLookup parameterLookup,
|
||||||
|
VariableRegistry variableRegistry
|
||||||
|
) {
|
||||||
|
this.flowAnalysisRuleNode = flowAnalysisRule;
|
||||||
|
this.properties = Collections.unmodifiableMap(properties);
|
||||||
|
this.serviceProvider = controllerServiceProvider;
|
||||||
|
this.preparedQueries = new HashMap<>();
|
||||||
|
this.parameterLookup = parameterLookup;
|
||||||
|
this.variableRegistry = variableRegistry;
|
||||||
|
|
||||||
|
for (final Map.Entry<PropertyDescriptor, String> entry : properties.entrySet()) {
|
||||||
|
final PropertyDescriptor desc = entry.getKey();
|
||||||
|
String value = entry.getValue();
|
||||||
|
if (value == null) {
|
||||||
|
value = desc.getDefaultValue();
|
||||||
|
}
|
||||||
|
|
||||||
|
final PreparedQuery pq = Query.prepare(value);
|
||||||
|
preparedQueries.put(desc, pq);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected FlowAnalysisRule getFlowAnalysisRule() {
|
||||||
|
return flowAnalysisRuleNode.getFlowAnalysisRule();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<PropertyDescriptor, String> getProperties() {
|
||||||
|
return properties;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, String> getAllProperties() {
|
||||||
|
final Map<String,String> propValueMap = new LinkedHashMap<>();
|
||||||
|
for (final Map.Entry<PropertyDescriptor, String> entry : getProperties().entrySet()) {
|
||||||
|
propValueMap.put(entry.getKey().getName(), entry.getValue());
|
||||||
|
}
|
||||||
|
return propValueMap;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public PropertyValue getProperty(final PropertyDescriptor property) {
|
||||||
|
final PropertyDescriptor descriptor = flowAnalysisRuleNode.getPropertyDescriptor(property.getName());
|
||||||
|
if (descriptor == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
final String configuredValue = properties.get(property);
|
||||||
|
final ResourceContext resourceContext = new StandardResourceContext(new StandardResourceReferenceFactory(), descriptor);
|
||||||
|
return new StandardPropertyValue(resourceContext, configuredValue == null ? descriptor.getDefaultValue() : configuredValue, serviceProvider, parameterLookup, preparedQueries.get(property),
|
||||||
|
variableRegistry);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,376 @@
|
||||||
|
/*
|
||||||
|
* 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.flowanalysis;
|
||||||
|
|
||||||
|
import org.apache.nifi.annotation.lifecycle.OnDisabled;
|
||||||
|
import org.apache.nifi.annotation.lifecycle.OnEnabled;
|
||||||
|
import org.apache.nifi.bundle.Bundle;
|
||||||
|
import org.apache.nifi.bundle.BundleCoordinate;
|
||||||
|
import org.apache.nifi.components.ConfigVerificationResult;
|
||||||
|
import org.apache.nifi.components.ConfigurableComponent;
|
||||||
|
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.FlowAnalysisRuleNode;
|
||||||
|
import org.apache.nifi.controller.LoggableComponent;
|
||||||
|
import org.apache.nifi.controller.ReloadComponent;
|
||||||
|
import org.apache.nifi.controller.TerminationAwareLogger;
|
||||||
|
import org.apache.nifi.controller.ValidationContextFactory;
|
||||||
|
import org.apache.nifi.controller.service.ControllerServiceNode;
|
||||||
|
import org.apache.nifi.controller.service.ControllerServiceProvider;
|
||||||
|
import org.apache.nifi.controller.service.StandardConfigurationContext;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRule;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRuleState;
|
||||||
|
import org.apache.nifi.flowanalysis.EnforcementPolicy;
|
||||||
|
import org.apache.nifi.flowanalysis.VerifiableFlowAnalysisRule;
|
||||||
|
import org.apache.nifi.groups.ProcessGroup;
|
||||||
|
import org.apache.nifi.logging.ComponentLog;
|
||||||
|
import org.apache.nifi.logging.StandardLoggingContext;
|
||||||
|
import org.apache.nifi.nar.ExtensionManager;
|
||||||
|
import org.apache.nifi.nar.InstanceClassLoader;
|
||||||
|
import org.apache.nifi.nar.NarCloseable;
|
||||||
|
import org.apache.nifi.parameter.ParameterLookup;
|
||||||
|
import org.apache.nifi.processor.SimpleProcessLogger;
|
||||||
|
import org.apache.nifi.registry.ComponentVariableRegistry;
|
||||||
|
import org.apache.nifi.util.CharacterFilterUtils;
|
||||||
|
import org.apache.nifi.util.FormatUtils;
|
||||||
|
import org.apache.nifi.util.ReflectionUtils;
|
||||||
|
import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
|
||||||
|
import org.apache.nifi.validation.RuleViolationsManager;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import java.lang.annotation.Annotation;
|
||||||
|
import java.lang.reflect.InvocationTargetException;
|
||||||
|
import java.net.URL;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
|
||||||
|
public abstract class AbstractFlowAnalysisRuleNode extends AbstractComponentNode implements FlowAnalysisRuleNode {
|
||||||
|
|
||||||
|
private final Logger log = LoggerFactory.getLogger(getClass());
|
||||||
|
|
||||||
|
private final AtomicReference<FlowAnalysisRuleDetails> flowAnalysisRuleRef;
|
||||||
|
private final ControllerServiceLookup serviceLookup;
|
||||||
|
private final RuleViolationsManager ruleViolationsManager;
|
||||||
|
|
||||||
|
private volatile String comment;
|
||||||
|
private EnforcementPolicy enforcementPolicy;
|
||||||
|
|
||||||
|
private volatile FlowAnalysisRuleState state = FlowAnalysisRuleState.DISABLED;
|
||||||
|
|
||||||
|
public AbstractFlowAnalysisRuleNode(final LoggableComponent<FlowAnalysisRule> flowAnalysisRule, final String id,
|
||||||
|
final ControllerServiceProvider controllerServiceProvider,
|
||||||
|
final ValidationContextFactory validationContextFactory,
|
||||||
|
final RuleViolationsManager ruleViolationsManager,
|
||||||
|
final ComponentVariableRegistry variableRegistry,
|
||||||
|
final ReloadComponent reloadComponent, final ExtensionManager extensionManager, final ValidationTrigger validationTrigger) {
|
||||||
|
|
||||||
|
this(flowAnalysisRule, id, controllerServiceProvider, validationContextFactory, ruleViolationsManager,
|
||||||
|
flowAnalysisRule.getComponent().getClass().getSimpleName(), flowAnalysisRule.getComponent().getClass().getCanonicalName(),
|
||||||
|
variableRegistry, reloadComponent, extensionManager, validationTrigger, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
public AbstractFlowAnalysisRuleNode(final LoggableComponent<FlowAnalysisRule> flowAnalysisRule, final String id, final ControllerServiceProvider controllerServiceProvider,
|
||||||
|
final ValidationContextFactory validationContextFactory, final RuleViolationsManager ruleViolationsManager,
|
||||||
|
final String componentType, final String componentCanonicalClass, final ComponentVariableRegistry variableRegistry,
|
||||||
|
final ReloadComponent reloadComponent, final ExtensionManager extensionManager, final ValidationTrigger validationTrigger,
|
||||||
|
final boolean isExtensionMissing) {
|
||||||
|
|
||||||
|
super(id, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent,
|
||||||
|
extensionManager, validationTrigger, isExtensionMissing);
|
||||||
|
this.flowAnalysisRuleRef = new AtomicReference<>(new FlowAnalysisRuleDetails(flowAnalysisRule));
|
||||||
|
this.serviceLookup = controllerServiceProvider;
|
||||||
|
this.ruleViolationsManager = ruleViolationsManager;
|
||||||
|
this.enforcementPolicy = EnforcementPolicy.WARN;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public EnforcementPolicy getEnforcementPolicy() {
|
||||||
|
return enforcementPolicy;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setEnforcementPolicy(EnforcementPolicy enforcementPolicy) {
|
||||||
|
this.enforcementPolicy = enforcementPolicy;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ConfigurableComponent getComponent() {
|
||||||
|
return flowAnalysisRuleRef.get().getFlowAnalysisRule();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public BundleCoordinate getBundleCoordinate() {
|
||||||
|
return flowAnalysisRuleRef.get().getBundleCoordinate();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public TerminationAwareLogger getLogger() {
|
||||||
|
return flowAnalysisRuleRef.get().getComponentLog();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FlowAnalysisRule getFlowAnalysisRule() {
|
||||||
|
return flowAnalysisRuleRef.get().getFlowAnalysisRule();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setFlowAnalysisRule(final LoggableComponent<FlowAnalysisRule> flowAnalysisRule) {
|
||||||
|
if (isEnabled()) {
|
||||||
|
throw new IllegalStateException("Cannot modify Flow Analysis Rule configuration while it is enabled");
|
||||||
|
}
|
||||||
|
this.flowAnalysisRuleRef.set(new FlowAnalysisRuleDetails(flowAnalysisRule));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void reload(final Set<URL> additionalUrls) throws FlowAnalysisRuleInstantiationException {
|
||||||
|
if (isEnabled()) {
|
||||||
|
throw new IllegalStateException("Cannot reload Flow Analysis Rule while it is enabled");
|
||||||
|
}
|
||||||
|
String additionalResourcesFingerprint = ClassLoaderUtils.generateAdditionalUrlsFingerprint(additionalUrls, determineClasloaderIsolationKey());
|
||||||
|
setAdditionalResourcesFingerprint(additionalResourcesFingerprint);
|
||||||
|
getReloadComponent().reload(this, getCanonicalClassName(), getBundleCoordinate(), additionalUrls);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isEnabled() {
|
||||||
|
return FlowAnalysisRuleState.ENABLED.equals(state);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isValidationNecessary() {
|
||||||
|
return !isEnabled() || getValidationStatus() != ValidationStatus.VALID;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ConfigurationContext getConfigurationContext() {
|
||||||
|
return new StandardConfigurationContext(this, serviceLookup, null, getVariableRegistry());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void verifyModifiable() throws IllegalStateException {
|
||||||
|
if (isEnabled()) {
|
||||||
|
throw new IllegalStateException("Cannot modify Flow Analysis Rule while it is enabled");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FlowAnalysisRuleState getState() {
|
||||||
|
return state;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getComments() {
|
||||||
|
return comment;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setComments(final String comment) {
|
||||||
|
this.comment = CharacterFilterUtils.filterInvalidXmlCharacters(comment);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void verifyCanDelete() {
|
||||||
|
if (isEnabled()) {
|
||||||
|
throw new IllegalStateException("Cannot delete " + getFlowAnalysisRule().getIdentifier() + " because it is enabled");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void verifyCanDisable() {
|
||||||
|
if (!isEnabled()) {
|
||||||
|
throw new IllegalStateException("Cannot disable " + getFlowAnalysisRule().getIdentifier() + " because it is already disabled");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void verifyCanEnable() {
|
||||||
|
if (getValidationStatus() == ValidationStatus.INVALID) {
|
||||||
|
throw new IllegalStateException("Cannot enable " + getFlowAnalysisRule().getIdentifier() + " because it is in INVALID status");
|
||||||
|
}
|
||||||
|
|
||||||
|
if (isEnabled()) {
|
||||||
|
throw new IllegalStateException("Cannot enable " + getFlowAnalysisRule().getIdentifier() + " because it is not disabled");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void verifyCanEnable(final Set<ControllerServiceNode> ignoredServices) {
|
||||||
|
if (isEnabled()) {
|
||||||
|
throw new IllegalStateException("Cannot enable " + getFlowAnalysisRule().getIdentifier() + " because it is not disabled");
|
||||||
|
}
|
||||||
|
|
||||||
|
final Collection<ValidationResult> validationResults = getValidationErrors(ignoredServices);
|
||||||
|
if (!validationResults.isEmpty()) {
|
||||||
|
throw new IllegalStateException(this + " cannot be enabled because it is not currently valid");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void verifyCanUpdate() {
|
||||||
|
if (isEnabled()) {
|
||||||
|
throw new IllegalStateException("Cannot update " + getFlowAnalysisRule().getIdentifier() + " because it is currently enabled");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void verifyCanClearState() {
|
||||||
|
verifyCanUpdate();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getProcessGroupIdentifier() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ParameterLookup getParameterLookup() {
|
||||||
|
return ParameterLookup.EMPTY;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
FlowAnalysisRule flowAnalysisRule = flowAnalysisRuleRef.get().getFlowAnalysisRule();
|
||||||
|
try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getExtensionManager(), flowAnalysisRule.getClass(), flowAnalysisRule.getIdentifier())) {
|
||||||
|
return getFlowAnalysisRule().toString();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void enable() {
|
||||||
|
verifyCanEnable();
|
||||||
|
setState(FlowAnalysisRuleState.ENABLED, OnEnabled.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void disable() {
|
||||||
|
verifyCanDisable();
|
||||||
|
setState(FlowAnalysisRuleState.DISABLED, OnDisabled.class);
|
||||||
|
|
||||||
|
ruleViolationsManager.removeRuleViolationsForRule(getIdentifier());
|
||||||
|
ruleViolationsManager.cleanUp();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void setState(FlowAnalysisRuleState newState, Class<? extends Annotation> annotation) {
|
||||||
|
final ConfigurationContext configContext = new StandardConfigurationContext(this, this.serviceLookup, null, getVariableRegistry());
|
||||||
|
|
||||||
|
try (final NarCloseable nc = NarCloseable.withComponentNarLoader(getExtensionManager(), getFlowAnalysisRule().getClass(), getIdentifier())) {
|
||||||
|
ReflectionUtils.invokeMethodsWithAnnotation(annotation, getFlowAnalysisRule(), configContext);
|
||||||
|
|
||||||
|
this.state = newState;
|
||||||
|
|
||||||
|
log.debug("Successfully {} {}", newState.toString().toLowerCase(), this);
|
||||||
|
} catch (Exception e) {
|
||||||
|
final Throwable cause = e instanceof InvocationTargetException ? e.getCause() : e;
|
||||||
|
|
||||||
|
final ComponentLog componentLog = new SimpleProcessLogger(getIdentifier(), getFlowAnalysisRule(), new StandardLoggingContext(null));
|
||||||
|
|
||||||
|
componentLog.error("Failed to invoke {} method", cause);
|
||||||
|
|
||||||
|
log.error("Failed to invoke {} method of {} due to {}", annotation.getSimpleName(), getFlowAnalysisRule(), cause.toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void verifyCanPerformVerification() {
|
||||||
|
if (isEnabled()) {
|
||||||
|
throw new IllegalStateException("Cannot perform verification of " + this + " because Flow Analysis Rule is not fully stopped");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<ConfigVerificationResult> verifyConfiguration(final ConfigurationContext context, final ComponentLog logger, final ExtensionManager extensionManager) {
|
||||||
|
final List<ConfigVerificationResult> results = new ArrayList<>();
|
||||||
|
|
||||||
|
try {
|
||||||
|
verifyCanPerformVerification();
|
||||||
|
|
||||||
|
final long startNanos = System.nanoTime();
|
||||||
|
// Call super's verifyConfig, which will perform component validation
|
||||||
|
results.addAll(super.verifyConfig(context.getProperties(), context.getAnnotationData(), null));
|
||||||
|
final long validationComplete = System.nanoTime();
|
||||||
|
|
||||||
|
// If any invalid outcomes from validation, we do not want to perform additional verification, because we only run additional verification when the component is valid.
|
||||||
|
// This is done in order to make it much simpler to develop these verifications, since the developer doesn't have to worry about whether or not the given values are valid.
|
||||||
|
if (!results.isEmpty() && results.stream().anyMatch(result -> result.getOutcome() == ConfigVerificationResult.Outcome.FAILED)) {
|
||||||
|
return results;
|
||||||
|
}
|
||||||
|
|
||||||
|
final FlowAnalysisRule flowAnalysisRule = getFlowAnalysisRule();
|
||||||
|
if (flowAnalysisRule instanceof VerifiableFlowAnalysisRule) {
|
||||||
|
logger.debug("{} is a VerifiableFlowAnalysisRule. Will perform full verification of configuration.", this);
|
||||||
|
final VerifiableFlowAnalysisRule verifiable = (VerifiableFlowAnalysisRule) flowAnalysisRule;
|
||||||
|
|
||||||
|
// 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, flowAnalysisRule.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 VerifiableFlowAnalysisRule, 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 Flow Analysis Rule's configuration for {}", this, t);
|
||||||
|
|
||||||
|
results.add(new ConfigVerificationResult.Builder()
|
||||||
|
.outcome(ConfigVerificationResult.Outcome.FAILED)
|
||||||
|
.verificationStepName("Perform Verification")
|
||||||
|
.explanation("Encountered unexpected failure when attempting to perform verification: " + t)
|
||||||
|
.build());
|
||||||
|
}
|
||||||
|
|
||||||
|
return results;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Optional<ProcessGroup> getParentProcessGroup() {
|
||||||
|
return Optional.empty();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -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.flowanalysis;
|
||||||
|
|
||||||
|
import org.apache.nifi.bundle.BundleCoordinate;
|
||||||
|
import org.apache.nifi.controller.LoggableComponent;
|
||||||
|
import org.apache.nifi.controller.TerminationAwareLogger;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRule;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Holder for StandardFlowAnalysisRuleNode to atomically swap out the component.
|
||||||
|
*/
|
||||||
|
class FlowAnalysisRuleDetails {
|
||||||
|
|
||||||
|
private final FlowAnalysisRule flowAnalysisRule;
|
||||||
|
private final TerminationAwareLogger componentLog;
|
||||||
|
private final BundleCoordinate bundleCoordinate;
|
||||||
|
|
||||||
|
public FlowAnalysisRuleDetails(final LoggableComponent<FlowAnalysisRule> flowAnalysisRule) {
|
||||||
|
this.flowAnalysisRule = flowAnalysisRule.getComponent();
|
||||||
|
this.componentLog = flowAnalysisRule.getLogger();
|
||||||
|
this.bundleCoordinate = flowAnalysisRule.getBundleCoordinate();
|
||||||
|
}
|
||||||
|
|
||||||
|
public FlowAnalysisRule getFlowAnalysisRule() {
|
||||||
|
return flowAnalysisRule;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TerminationAwareLogger getComponentLog() {
|
||||||
|
return componentLog;
|
||||||
|
}
|
||||||
|
|
||||||
|
public BundleCoordinate getBundleCoordinate() {
|
||||||
|
return bundleCoordinate;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,110 @@
|
||||||
|
/*
|
||||||
|
* 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.flowanalysis;
|
||||||
|
|
||||||
|
import org.apache.nifi.controller.ControllerService;
|
||||||
|
import org.apache.nifi.controller.ControllerServiceLookup;
|
||||||
|
import org.apache.nifi.controller.NodeTypeProvider;
|
||||||
|
import org.apache.nifi.controller.kerberos.KerberosConfig;
|
||||||
|
import org.apache.nifi.controller.service.ControllerServiceProvider;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRuleInitializationContext;
|
||||||
|
import org.apache.nifi.logging.ComponentLog;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
public class StandardFlowAnalysisInitializationContext implements FlowAnalysisRuleInitializationContext, ControllerServiceLookup {
|
||||||
|
private final String id;
|
||||||
|
private final ComponentLog logger;
|
||||||
|
private final ControllerServiceProvider serviceProvider;
|
||||||
|
private final KerberosConfig kerberosConfig;
|
||||||
|
private final NodeTypeProvider nodeTypeProvider;
|
||||||
|
|
||||||
|
public StandardFlowAnalysisInitializationContext(
|
||||||
|
String id,
|
||||||
|
ComponentLog logger,
|
||||||
|
ControllerServiceProvider serviceProvider,
|
||||||
|
KerberosConfig kerberosConfig,
|
||||||
|
NodeTypeProvider nodeTypeProvider
|
||||||
|
) {
|
||||||
|
this.id = id;
|
||||||
|
this.serviceProvider = serviceProvider;
|
||||||
|
this.logger = logger;
|
||||||
|
this.kerberosConfig = kerberosConfig;
|
||||||
|
this.nodeTypeProvider = nodeTypeProvider;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getIdentifier() {
|
||||||
|
return id;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<String> getControllerServiceIdentifiers(final Class<? extends ControllerService> serviceType) {
|
||||||
|
return serviceProvider.getControllerServiceIdentifiers(serviceType, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ControllerService getControllerService(final String identifier) {
|
||||||
|
return serviceProvider.getControllerService(identifier);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isControllerServiceEnabled(final ControllerService service) {
|
||||||
|
return serviceProvider.isControllerServiceEnabled(service);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isControllerServiceEnabled(final String serviceIdentifier) {
|
||||||
|
return serviceProvider.isControllerServiceEnabled(serviceIdentifier);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isControllerServiceEnabling(final String serviceIdentifier) {
|
||||||
|
return serviceProvider.isControllerServiceEnabling(serviceIdentifier);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getControllerServiceName(final String serviceIdentifier) {
|
||||||
|
return serviceProvider.getControllerServiceName(serviceIdentifier);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ComponentLog getLogger() {
|
||||||
|
return logger;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getKerberosServicePrincipal() {
|
||||||
|
return kerberosConfig.getPrincipal();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public File getKerberosServiceKeytab() {
|
||||||
|
return kerberosConfig.getKeytabLocation();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public File getKerberosConfigurationFile() {
|
||||||
|
return kerberosConfig.getConfigFile();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public NodeTypeProvider getNodeTypeProvider() {
|
||||||
|
return nodeTypeProvider;
|
||||||
|
}
|
||||||
|
}
|
|
@ -833,4 +833,10 @@ public class StandardControllerServiceNode extends AbstractComponentNode impleme
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void performFlowAnalysisOnThis() {
|
||||||
|
Optional.ofNullable(getValidationContextFactory().getFlowAnalyzer())
|
||||||
|
.ifPresent(flowAnalyzer -> flowAnalyzer.analyzeControllerService(this));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.nifi.controller.service;
|
||||||
import org.apache.nifi.components.PropertyDescriptor;
|
import org.apache.nifi.components.PropertyDescriptor;
|
||||||
import org.apache.nifi.controller.ComponentNode;
|
import org.apache.nifi.controller.ComponentNode;
|
||||||
import org.apache.nifi.controller.ControllerService;
|
import org.apache.nifi.controller.ControllerService;
|
||||||
|
import org.apache.nifi.controller.FlowAnalysisRuleNode;
|
||||||
import org.apache.nifi.controller.ParameterProviderNode;
|
import org.apache.nifi.controller.ParameterProviderNode;
|
||||||
import org.apache.nifi.controller.ProcessScheduler;
|
import org.apache.nifi.controller.ProcessScheduler;
|
||||||
import org.apache.nifi.controller.ProcessorNode;
|
import org.apache.nifi.controller.ProcessorNode;
|
||||||
|
@ -113,6 +114,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
|
||||||
// or a service that references this controller service, etc.
|
// or a service that references this controller service, etc.
|
||||||
final List<ProcessorNode> processors = serviceNode.getReferences().findRecursiveReferences(ProcessorNode.class);
|
final List<ProcessorNode> processors = serviceNode.getReferences().findRecursiveReferences(ProcessorNode.class);
|
||||||
final List<ReportingTaskNode> reportingTasks = serviceNode.getReferences().findRecursiveReferences(ReportingTaskNode.class);
|
final List<ReportingTaskNode> reportingTasks = serviceNode.getReferences().findRecursiveReferences(ReportingTaskNode.class);
|
||||||
|
final List<FlowAnalysisRuleNode> flowAnalysisRuleNodes = serviceNode.getReferences().findRecursiveReferences(FlowAnalysisRuleNode.class);
|
||||||
|
|
||||||
// verify that we can start all components (that are not disabled) before doing anything
|
// verify that we can start all components (that are not disabled) before doing anything
|
||||||
for (final ProcessorNode node : processors) {
|
for (final ProcessorNode node : processors) {
|
||||||
|
@ -133,6 +135,15 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
|
||||||
node.verifyCanStart();
|
node.verifyCanStart();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
for (final FlowAnalysisRuleNode node : flowAnalysisRuleNodes) {
|
||||||
|
if (candidates != null && !candidates.contains(node)) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!node.isEnabled()) {
|
||||||
|
node.verifyCanEnable();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// start all of the components that are not disabled
|
// start all of the components that are not disabled
|
||||||
final Set<ComponentNode> updated = new HashSet<>();
|
final Set<ComponentNode> updated = new HashSet<>();
|
||||||
|
@ -156,6 +167,16 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
|
||||||
updated.add(node);
|
updated.add(node);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
for (final FlowAnalysisRuleNode node : flowAnalysisRuleNodes) {
|
||||||
|
if (candidates != null && !candidates.contains(node)) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!node.isEnabled()) {
|
||||||
|
node.enable();
|
||||||
|
updated.add(node);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
return updated;
|
return updated;
|
||||||
}
|
}
|
||||||
|
@ -166,6 +187,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
|
||||||
// or a service that references this controller service, etc.
|
// or a service that references this controller service, etc.
|
||||||
final List<ProcessorNode> processors = serviceNode.getReferences().findRecursiveReferences(ProcessorNode.class);
|
final List<ProcessorNode> processors = serviceNode.getReferences().findRecursiveReferences(ProcessorNode.class);
|
||||||
final List<ReportingTaskNode> reportingTasks = serviceNode.getReferences().findRecursiveReferences(ReportingTaskNode.class);
|
final List<ReportingTaskNode> reportingTasks = serviceNode.getReferences().findRecursiveReferences(ReportingTaskNode.class);
|
||||||
|
final List<FlowAnalysisRuleNode> flowAnalysisRuleNodes = serviceNode.getReferences().findRecursiveReferences(FlowAnalysisRuleNode.class);
|
||||||
|
|
||||||
final Map<ComponentNode, Future<Void>> updated = new HashMap<>();
|
final Map<ComponentNode, Future<Void>> updated = new HashMap<>();
|
||||||
|
|
||||||
|
@ -180,6 +202,11 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
|
||||||
node.verifyCanStop();
|
node.verifyCanStop();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
for (final FlowAnalysisRuleNode node : flowAnalysisRuleNodes) {
|
||||||
|
if (node.isEnabled()) {
|
||||||
|
node.verifyCanDisable();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// stop all of the components that are running
|
// stop all of the components that are running
|
||||||
for (final ProcessorNode node : processors) {
|
for (final ProcessorNode node : processors) {
|
||||||
|
@ -194,6 +221,20 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
|
||||||
updated.put(node, future);
|
updated.put(node, future);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
for (final FlowAnalysisRuleNode node : flowAnalysisRuleNodes) {
|
||||||
|
if (node.isEnabled()) {
|
||||||
|
final CompletableFuture<Void> future = new CompletableFuture<>();
|
||||||
|
processScheduler.submitFrameworkTask(() -> {
|
||||||
|
try {
|
||||||
|
node.disable();
|
||||||
|
future.complete(null);
|
||||||
|
} catch (final Exception e) {
|
||||||
|
future.completeExceptionally(e);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
updated.put(node, future);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
return updated;
|
return updated;
|
||||||
}
|
}
|
||||||
|
@ -504,6 +545,8 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
|
||||||
if (serviceNode == null) {
|
if (serviceNode == null) {
|
||||||
final ReportingTaskNode taskNode = flowManager.getReportingTaskNode(componentId);
|
final ReportingTaskNode taskNode = flowManager.getReportingTaskNode(componentId);
|
||||||
if (taskNode == null) {
|
if (taskNode == null) {
|
||||||
|
final FlowAnalysisRuleNode flowAnalysisRuleNode = flowManager.getFlowAnalysisRuleNode(componentId);
|
||||||
|
if (flowAnalysisRuleNode == null) {
|
||||||
final ParameterProviderNode parameterProviderNode = flowManager.getParameterProvider(componentId);
|
final ParameterProviderNode parameterProviderNode = flowManager.getParameterProvider(componentId);
|
||||||
if (parameterProviderNode == null) {
|
if (parameterProviderNode == null) {
|
||||||
final FlowRegistryClientNode flowRegistryClientNode = flowManager.getFlowRegistryClient(componentId);
|
final FlowRegistryClientNode flowRegistryClientNode = flowManager.getFlowRegistryClient(componentId);
|
||||||
|
@ -512,8 +555,9 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// We have confirmed that the component is a reporting task or parameter provider. We can only reference Controller Services
|
// We have confirmed that the component is a reporting task or a flow analysis rule or parameter provider. We can only reference Controller Services
|
||||||
// that are scoped at the FlowController level in this case.
|
// that are scoped at the FlowController level in this case.
|
||||||
final ControllerServiceNode rootServiceNode = flowManager.getRootControllerService(serviceIdentifier);
|
final ControllerServiceNode rootServiceNode = flowManager.getRootControllerService(serviceIdentifier);
|
||||||
return (rootServiceNode == null) ? null : rootServiceNode.getProxiedControllerService();
|
return (rootServiceNode == null) ? null : rootServiceNode.getProxiedControllerService();
|
||||||
|
@ -638,6 +682,8 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
|
||||||
LogRepositoryFactory.removeRepository(serviceNode.getIdentifier());
|
LogRepositoryFactory.removeRepository(serviceNode.getIdentifier());
|
||||||
extensionManager.removeInstanceClassLoader(serviceNode.getIdentifier());
|
extensionManager.removeInstanceClassLoader(serviceNode.getIdentifier());
|
||||||
serviceCache.remove(serviceNode.getIdentifier());
|
serviceCache.remove(serviceNode.getIdentifier());
|
||||||
|
|
||||||
|
flowManager.getRuleViolationsManager().removeRuleViolationsForSubject(serviceNode.getIdentifier());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -695,6 +741,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
|
||||||
public void verifyCanScheduleReferencingComponents(final ControllerServiceNode serviceNode) {
|
public void verifyCanScheduleReferencingComponents(final ControllerServiceNode serviceNode) {
|
||||||
final List<ControllerServiceNode> referencingServices = serviceNode.getReferences().findRecursiveReferences(ControllerServiceNode.class);
|
final List<ControllerServiceNode> referencingServices = serviceNode.getReferences().findRecursiveReferences(ControllerServiceNode.class);
|
||||||
final List<ReportingTaskNode> referencingReportingTasks = serviceNode.getReferences().findRecursiveReferences(ReportingTaskNode.class);
|
final List<ReportingTaskNode> referencingReportingTasks = serviceNode.getReferences().findRecursiveReferences(ReportingTaskNode.class);
|
||||||
|
final List<FlowAnalysisRuleNode> referencingFlowAnalysisRuleNodes = serviceNode.getReferences().findRecursiveReferences(FlowAnalysisRuleNode.class);
|
||||||
final List<ProcessorNode> referencingProcessors = serviceNode.getReferences().findRecursiveReferences(ProcessorNode.class);
|
final List<ProcessorNode> referencingProcessors = serviceNode.getReferences().findRecursiveReferences(ProcessorNode.class);
|
||||||
|
|
||||||
final Set<ControllerServiceNode> referencingServiceSet = new HashSet<>(referencingServices);
|
final Set<ControllerServiceNode> referencingServiceSet = new HashSet<>(referencingServices);
|
||||||
|
@ -705,6 +752,10 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
for (final FlowAnalysisRuleNode ruleNode : referencingFlowAnalysisRuleNodes) {
|
||||||
|
ruleNode.verifyCanEnable(referencingServiceSet);
|
||||||
|
}
|
||||||
|
|
||||||
for (final ProcessorNode procNode : referencingProcessors) {
|
for (final ProcessorNode procNode : referencingProcessors) {
|
||||||
if (procNode.getScheduledState() != ScheduledState.DISABLED) {
|
if (procNode.getScheduledState() != ScheduledState.DISABLED) {
|
||||||
procNode.verifyCanStart();
|
procNode.verifyCanStart();
|
||||||
|
|
|
@ -42,6 +42,7 @@ public class VolatileBulletinRepository implements BulletinRepository {
|
||||||
private static final String CONTROLLER_BULLETIN_STORE_KEY = "CONTROLLER";
|
private static final String CONTROLLER_BULLETIN_STORE_KEY = "CONTROLLER";
|
||||||
private static final String SERVICE_BULLETIN_STORE_KEY = "SERVICE";
|
private static final String SERVICE_BULLETIN_STORE_KEY = "SERVICE";
|
||||||
private static final String REPORTING_TASK_BULLETIN_STORE_KEY = "REPORTING_TASK";
|
private static final String REPORTING_TASK_BULLETIN_STORE_KEY = "REPORTING_TASK";
|
||||||
|
private static final String FLOW_ANALYSIS_RULE_BULLETIN_STORE_KEY = "FLOW_ANALYSIS_RULE";
|
||||||
private static final String FLOW_REGISTRY_CLIENT_STORE_KEY = "FLOW_REGISTRY_CLIENT";
|
private static final String FLOW_REGISTRY_CLIENT_STORE_KEY = "FLOW_REGISTRY_CLIENT";
|
||||||
private static final String PARAMETER_PROVIDER_BULLETIN_STORE_KEY = "PARAMETER_PROVIDER";
|
private static final String PARAMETER_PROVIDER_BULLETIN_STORE_KEY = "PARAMETER_PROVIDER";
|
||||||
|
|
||||||
|
@ -301,6 +302,8 @@ public class VolatileBulletinRepository implements BulletinRepository {
|
||||||
return SERVICE_BULLETIN_STORE_KEY;
|
return SERVICE_BULLETIN_STORE_KEY;
|
||||||
case REPORTING_TASK:
|
case REPORTING_TASK:
|
||||||
return REPORTING_TASK_BULLETIN_STORE_KEY;
|
return REPORTING_TASK_BULLETIN_STORE_KEY;
|
||||||
|
case FLOW_ANALYSIS_RULE:
|
||||||
|
return FLOW_ANALYSIS_RULE_BULLETIN_STORE_KEY;
|
||||||
case PARAMETER_PROVIDER:
|
case PARAMETER_PROVIDER:
|
||||||
return PARAMETER_PROVIDER_BULLETIN_STORE_KEY;
|
return PARAMETER_PROVIDER_BULLETIN_STORE_KEY;
|
||||||
case FLOW_REGISTRY_CLIENT:
|
case FLOW_REGISTRY_CLIENT:
|
||||||
|
@ -315,6 +318,7 @@ public class VolatileBulletinRepository implements BulletinRepository {
|
||||||
case FLOW_CONTROLLER:
|
case FLOW_CONTROLLER:
|
||||||
case CONTROLLER_SERVICE:
|
case CONTROLLER_SERVICE:
|
||||||
case REPORTING_TASK:
|
case REPORTING_TASK:
|
||||||
|
case FLOW_ANALYSIS_RULE:
|
||||||
case PARAMETER_PROVIDER:
|
case PARAMETER_PROVIDER:
|
||||||
case FLOW_REGISTRY_CLIENT:
|
case FLOW_REGISTRY_CLIENT:
|
||||||
return true;
|
return true;
|
||||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.nifi.connectable.Position;
|
||||||
import org.apache.nifi.connectable.Size;
|
import org.apache.nifi.connectable.Size;
|
||||||
import org.apache.nifi.controller.BackoffMechanism;
|
import org.apache.nifi.controller.BackoffMechanism;
|
||||||
import org.apache.nifi.controller.ComponentNode;
|
import org.apache.nifi.controller.ComponentNode;
|
||||||
|
import org.apache.nifi.controller.FlowAnalysisRuleNode;
|
||||||
import org.apache.nifi.controller.ParameterProviderNode;
|
import org.apache.nifi.controller.ParameterProviderNode;
|
||||||
import org.apache.nifi.controller.ProcessorNode;
|
import org.apache.nifi.controller.ProcessorNode;
|
||||||
import org.apache.nifi.controller.PropertyConfiguration;
|
import org.apache.nifi.controller.PropertyConfiguration;
|
||||||
|
@ -37,6 +38,7 @@ import org.apache.nifi.controller.ScheduledState;
|
||||||
import org.apache.nifi.controller.Template;
|
import org.apache.nifi.controller.Template;
|
||||||
import org.apache.nifi.controller.Triggerable;
|
import org.apache.nifi.controller.Triggerable;
|
||||||
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
|
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
|
||||||
|
import org.apache.nifi.controller.flowanalysis.FlowAnalysisRuleInstantiationException;
|
||||||
import org.apache.nifi.controller.label.Label;
|
import org.apache.nifi.controller.label.Label;
|
||||||
import org.apache.nifi.controller.queue.FlowFileQueue;
|
import org.apache.nifi.controller.queue.FlowFileQueue;
|
||||||
import org.apache.nifi.controller.queue.LoadBalanceCompression;
|
import org.apache.nifi.controller.queue.LoadBalanceCompression;
|
||||||
|
@ -57,6 +59,7 @@ import org.apache.nifi.flow.VersionedComponent;
|
||||||
import org.apache.nifi.flow.VersionedConnection;
|
import org.apache.nifi.flow.VersionedConnection;
|
||||||
import org.apache.nifi.flow.VersionedControllerService;
|
import org.apache.nifi.flow.VersionedControllerService;
|
||||||
import org.apache.nifi.flow.VersionedExternalFlow;
|
import org.apache.nifi.flow.VersionedExternalFlow;
|
||||||
|
import org.apache.nifi.flow.VersionedFlowAnalysisRule;
|
||||||
import org.apache.nifi.flow.VersionedFlowCoordinates;
|
import org.apache.nifi.flow.VersionedFlowCoordinates;
|
||||||
import org.apache.nifi.flow.VersionedFunnel;
|
import org.apache.nifi.flow.VersionedFunnel;
|
||||||
import org.apache.nifi.flow.VersionedLabel;
|
import org.apache.nifi.flow.VersionedLabel;
|
||||||
|
@ -3665,6 +3668,82 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void synchronize(final FlowAnalysisRuleNode flowAnalysisRule, final VersionedFlowAnalysisRule proposed, final FlowSynchronizationOptions synchronizationOptions)
|
||||||
|
throws FlowSynchronizationException, TimeoutException, InterruptedException, FlowAnalysisRuleInstantiationException {
|
||||||
|
|
||||||
|
if (flowAnalysisRule == null && proposed == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
synchronizationOptions.getComponentScheduler().pause();
|
||||||
|
try {
|
||||||
|
// If flow analysis rule is not null, make sure that it's disabled.
|
||||||
|
if (flowAnalysisRule != null && flowAnalysisRule.isEnabled()) {
|
||||||
|
flowAnalysisRule.disable();
|
||||||
|
}
|
||||||
|
|
||||||
|
if (proposed == null) {
|
||||||
|
flowAnalysisRule.verifyCanDelete();
|
||||||
|
context.getFlowManager().removeFlowAnalysisRule(flowAnalysisRule);
|
||||||
|
LOG.info("Successfully synchronized {} by removing it from the flow", flowAnalysisRule);
|
||||||
|
} else if (flowAnalysisRule == null) {
|
||||||
|
final FlowAnalysisRuleNode added = addFlowAnalysisRule(proposed);
|
||||||
|
LOG.info("Successfully synchronized {} by adding it to the flow", added);
|
||||||
|
} else {
|
||||||
|
updateFlowAnalysisRule(flowAnalysisRule, proposed);
|
||||||
|
LOG.info("Successfully synchronized {} by updating it to match proposed version", flowAnalysisRule);
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
synchronizationOptions.getComponentScheduler().resume();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private FlowAnalysisRuleNode addFlowAnalysisRule(final VersionedFlowAnalysisRule flowAnalysisRule) throws FlowAnalysisRuleInstantiationException {
|
||||||
|
final BundleCoordinate coordinate = toCoordinate(flowAnalysisRule.getBundle());
|
||||||
|
final FlowAnalysisRuleNode ruleNode = context.getFlowManager().createFlowAnalysisRule(flowAnalysisRule.getType(), flowAnalysisRule.getInstanceIdentifier(), coordinate, false);
|
||||||
|
updateFlowAnalysisRule(ruleNode, flowAnalysisRule);
|
||||||
|
return ruleNode;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void updateFlowAnalysisRule(final FlowAnalysisRuleNode flowAnalysisRule, final VersionedFlowAnalysisRule proposed)
|
||||||
|
throws FlowAnalysisRuleInstantiationException {
|
||||||
|
LOG.debug("Updating Flow Analysis Rule {}", flowAnalysisRule);
|
||||||
|
|
||||||
|
flowAnalysisRule.pauseValidationTrigger();
|
||||||
|
try {
|
||||||
|
flowAnalysisRule.setName(proposed.getName());
|
||||||
|
flowAnalysisRule.setComments(proposed.getComments());
|
||||||
|
flowAnalysisRule.setEnforcementPolicy(proposed.getEnforcementPolicy());
|
||||||
|
|
||||||
|
if (!isEqual(flowAnalysisRule.getBundleCoordinate(), proposed.getBundle())) {
|
||||||
|
final BundleCoordinate newBundleCoordinate = toCoordinate(proposed.getBundle());
|
||||||
|
final List<PropertyDescriptor> descriptors = new ArrayList<>(flowAnalysisRule.getProperties().keySet());
|
||||||
|
final Set<URL> additionalUrls = flowAnalysisRule.getAdditionalClasspathResources(descriptors);
|
||||||
|
context.getReloadComponent().reload(flowAnalysisRule, proposed.getType(), newBundleCoordinate, additionalUrls);
|
||||||
|
}
|
||||||
|
|
||||||
|
final Set<String> sensitiveDynamicPropertyNames = getSensitiveDynamicPropertyNames(flowAnalysisRule, proposed.getProperties(), proposed.getPropertyDescriptors().values());
|
||||||
|
flowAnalysisRule.setProperties(proposed.getProperties(), false, sensitiveDynamicPropertyNames);
|
||||||
|
|
||||||
|
switch (proposed.getScheduledState()) {
|
||||||
|
case DISABLED:
|
||||||
|
if (flowAnalysisRule.isEnabled()) {
|
||||||
|
flowAnalysisRule.disable();
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
case ENABLED:
|
||||||
|
if (!flowAnalysisRule.isEnabled()) {
|
||||||
|
flowAnalysisRule.enable();
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
notifyScheduledStateChange(flowAnalysisRule, syncOptions, proposed.getScheduledState());
|
||||||
|
} finally {
|
||||||
|
flowAnalysisRule.resumeValidationTrigger();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private <T extends org.apache.nifi.components.VersionedComponent & Connectable> boolean matchesId(final T component, final String id) {
|
private <T extends org.apache.nifi.components.VersionedComponent & Connectable> boolean matchesId(final T component, final String id) {
|
||||||
return id.equals(component.getIdentifier()) || id.equals(component.getVersionedComponentId().orElse(NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier())));
|
return id.equals(component.getIdentifier()) || id.equals(component.getVersionedComponentId().orElse(NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier())));
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,15 +20,18 @@ package org.apache.nifi.flow.synchronization;
|
||||||
import org.apache.nifi.connectable.Connection;
|
import org.apache.nifi.connectable.Connection;
|
||||||
import org.apache.nifi.connectable.Funnel;
|
import org.apache.nifi.connectable.Funnel;
|
||||||
import org.apache.nifi.connectable.Port;
|
import org.apache.nifi.connectable.Port;
|
||||||
|
import org.apache.nifi.controller.FlowAnalysisRuleNode;
|
||||||
import org.apache.nifi.controller.ProcessorNode;
|
import org.apache.nifi.controller.ProcessorNode;
|
||||||
import org.apache.nifi.controller.ReportingTaskNode;
|
import org.apache.nifi.controller.ReportingTaskNode;
|
||||||
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
|
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
|
||||||
|
import org.apache.nifi.controller.flowanalysis.FlowAnalysisRuleInstantiationException;
|
||||||
import org.apache.nifi.controller.label.Label;
|
import org.apache.nifi.controller.label.Label;
|
||||||
import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
|
import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
|
||||||
import org.apache.nifi.controller.service.ControllerServiceNode;
|
import org.apache.nifi.controller.service.ControllerServiceNode;
|
||||||
import org.apache.nifi.flow.VersionedConnection;
|
import org.apache.nifi.flow.VersionedConnection;
|
||||||
import org.apache.nifi.flow.VersionedControllerService;
|
import org.apache.nifi.flow.VersionedControllerService;
|
||||||
import org.apache.nifi.flow.VersionedExternalFlow;
|
import org.apache.nifi.flow.VersionedExternalFlow;
|
||||||
|
import org.apache.nifi.flow.VersionedFlowAnalysisRule;
|
||||||
import org.apache.nifi.flow.VersionedFunnel;
|
import org.apache.nifi.flow.VersionedFunnel;
|
||||||
import org.apache.nifi.flow.VersionedLabel;
|
import org.apache.nifi.flow.VersionedLabel;
|
||||||
import org.apache.nifi.flow.VersionedParameterContext;
|
import org.apache.nifi.flow.VersionedParameterContext;
|
||||||
|
@ -174,6 +177,20 @@ public interface VersionedComponentSynchronizer {
|
||||||
void synchronize(ReportingTaskNode reportingTask, VersionedReportingTask proposed, FlowSynchronizationOptions synchronizationOptions)
|
void synchronize(ReportingTaskNode reportingTask, VersionedReportingTask proposed, FlowSynchronizationOptions synchronizationOptions)
|
||||||
throws FlowSynchronizationException, TimeoutException, InterruptedException, ReportingTaskInstantiationException;
|
throws FlowSynchronizationException, TimeoutException, InterruptedException, ReportingTaskInstantiationException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Synchronizes the given Flow Analysis Rule to match the proposed one, or deletes the Flow Analysis Rule if the proposed is <code>null</code>.
|
||||||
|
* If the given Flow Analysis Rule is <code>null</code>, creates it.
|
||||||
|
*
|
||||||
|
* @param flowAnalysisRule the flow analysis rule to synchronize
|
||||||
|
* @param proposed the proposed/desired state
|
||||||
|
* @param synchronizationOptions options for how to synchronize the flow
|
||||||
|
*
|
||||||
|
* @throws IllegalStateException if the flow analysis rule cannot be updated due to the state of the flow
|
||||||
|
* @throws FlowSynchronizationException if unable to synchronize the flow analysis rule with the proposed version
|
||||||
|
* @throws TimeoutException if the flow analysis rule is being removed and takes longer to stop than the timeout allowed by the {@link FlowSynchronizationOptions synchronization options}.
|
||||||
|
*/
|
||||||
|
void synchronize(FlowAnalysisRuleNode flowAnalysisRule, VersionedFlowAnalysisRule proposed, FlowSynchronizationOptions synchronizationOptions)
|
||||||
|
throws FlowSynchronizationException, TimeoutException, InterruptedException, FlowAnalysisRuleInstantiationException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Synchronizes the given Remote Process Group to match the proposed one, or deletes the rpg if the proposed is <code>null</code>. If the given rpg is <code>null</code>, creates it and adds
|
* Synchronizes the given Remote Process Group to match the proposed one, or deletes the rpg if the proposed is <code>null</code>. If the given rpg is <code>null</code>, creates it and adds
|
||||||
|
|
|
@ -20,9 +20,11 @@ import org.apache.nifi.components.PropertyDescriptor;
|
||||||
import org.apache.nifi.components.ValidationContext;
|
import org.apache.nifi.components.ValidationContext;
|
||||||
import org.apache.nifi.controller.PropertyConfiguration;
|
import org.apache.nifi.controller.PropertyConfiguration;
|
||||||
import org.apache.nifi.controller.ValidationContextFactory;
|
import org.apache.nifi.controller.ValidationContextFactory;
|
||||||
|
import org.apache.nifi.controller.flowanalysis.FlowAnalyzer;
|
||||||
import org.apache.nifi.controller.service.ControllerServiceProvider;
|
import org.apache.nifi.controller.service.ControllerServiceProvider;
|
||||||
import org.apache.nifi.parameter.ParameterContext;
|
import org.apache.nifi.parameter.ParameterContext;
|
||||||
import org.apache.nifi.registry.VariableRegistry;
|
import org.apache.nifi.registry.VariableRegistry;
|
||||||
|
import org.apache.nifi.validation.RuleViolationsManager;
|
||||||
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
@ -30,10 +32,26 @@ public class StandardValidationContextFactory implements ValidationContextFactor
|
||||||
|
|
||||||
private final ControllerServiceProvider serviceProvider;
|
private final ControllerServiceProvider serviceProvider;
|
||||||
private final VariableRegistry variableRegistry;
|
private final VariableRegistry variableRegistry;
|
||||||
|
private final RuleViolationsManager ruleViolationsManager;
|
||||||
|
private final FlowAnalyzer flowAnalyzer;
|
||||||
|
|
||||||
public StandardValidationContextFactory(final ControllerServiceProvider serviceProvider, final VariableRegistry variableRegistry) {
|
public StandardValidationContextFactory(
|
||||||
|
final ControllerServiceProvider serviceProvider,
|
||||||
|
final VariableRegistry variableRegistry
|
||||||
|
) {
|
||||||
|
this(serviceProvider, variableRegistry, null, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public StandardValidationContextFactory(
|
||||||
|
final ControllerServiceProvider serviceProvider,
|
||||||
|
final VariableRegistry variableRegistry,
|
||||||
|
final RuleViolationsManager ruleViolationsManager,
|
||||||
|
final FlowAnalyzer flowAnalyzer
|
||||||
|
) {
|
||||||
this.serviceProvider = serviceProvider;
|
this.serviceProvider = serviceProvider;
|
||||||
this.variableRegistry = variableRegistry;
|
this.variableRegistry = variableRegistry;
|
||||||
|
this.ruleViolationsManager = ruleViolationsManager;
|
||||||
|
this.flowAnalyzer = flowAnalyzer;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -41,4 +59,14 @@ public class StandardValidationContextFactory implements ValidationContextFactor
|
||||||
final ParameterContext parameterContext, final boolean validateConnections) {
|
final ParameterContext parameterContext, final boolean validateConnections) {
|
||||||
return new StandardValidationContext(serviceProvider, properties, annotationData, groupId, componentId, variableRegistry, parameterContext, validateConnections);
|
return new StandardValidationContext(serviceProvider, properties, annotationData, groupId, componentId, variableRegistry, parameterContext, validateConnections);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RuleViolationsManager getRuleViolationsManager() {
|
||||||
|
return ruleViolationsManager;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FlowAnalyzer getFlowAnalyzer() {
|
||||||
|
return flowAnalyzer;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.nifi.connectable.Funnel;
|
||||||
import org.apache.nifi.connectable.Port;
|
import org.apache.nifi.connectable.Port;
|
||||||
import org.apache.nifi.controller.ComponentNode;
|
import org.apache.nifi.controller.ComponentNode;
|
||||||
import org.apache.nifi.controller.ControllerService;
|
import org.apache.nifi.controller.ControllerService;
|
||||||
|
import org.apache.nifi.controller.FlowAnalysisRuleNode;
|
||||||
import org.apache.nifi.controller.ParameterProviderNode;
|
import org.apache.nifi.controller.ParameterProviderNode;
|
||||||
import org.apache.nifi.controller.ProcessorNode;
|
import org.apache.nifi.controller.ProcessorNode;
|
||||||
import org.apache.nifi.controller.PropertyConfiguration;
|
import org.apache.nifi.controller.PropertyConfiguration;
|
||||||
|
@ -52,6 +53,7 @@ import org.apache.nifi.flow.PortType;
|
||||||
import org.apache.nifi.flow.Position;
|
import org.apache.nifi.flow.Position;
|
||||||
import org.apache.nifi.flow.VersionedConnection;
|
import org.apache.nifi.flow.VersionedConnection;
|
||||||
import org.apache.nifi.flow.VersionedControllerService;
|
import org.apache.nifi.flow.VersionedControllerService;
|
||||||
|
import org.apache.nifi.flow.VersionedFlowAnalysisRule;
|
||||||
import org.apache.nifi.flow.VersionedFlowCoordinates;
|
import org.apache.nifi.flow.VersionedFlowCoordinates;
|
||||||
import org.apache.nifi.flow.VersionedFlowRegistryClient;
|
import org.apache.nifi.flow.VersionedFlowRegistryClient;
|
||||||
import org.apache.nifi.flow.VersionedFunnel;
|
import org.apache.nifi.flow.VersionedFunnel;
|
||||||
|
@ -479,6 +481,26 @@ public class NiFiRegistryFlowMapper {
|
||||||
return versionedTask;
|
return versionedTask;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public VersionedFlowAnalysisRule mapFlowAnalysisRule(final FlowAnalysisRuleNode flowAnalysisRuleNode, final ControllerServiceProvider serviceProvider) {
|
||||||
|
final VersionedFlowAnalysisRule versionedRule = new VersionedFlowAnalysisRule();
|
||||||
|
versionedRule.setIdentifier(flowAnalysisRuleNode.getIdentifier());
|
||||||
|
if (flowMappingOptions.isMapInstanceIdentifiers()) {
|
||||||
|
versionedRule.setInstanceIdentifier(flowAnalysisRuleNode.getIdentifier());
|
||||||
|
}
|
||||||
|
versionedRule.setBundle(mapBundle(flowAnalysisRuleNode.getBundleCoordinate()));
|
||||||
|
versionedRule.setComments(flowAnalysisRuleNode.getComments());
|
||||||
|
versionedRule.setComponentType(ComponentType.FLOW_ANALYSIS_RULE);
|
||||||
|
versionedRule.setName(flowAnalysisRuleNode.getName());
|
||||||
|
|
||||||
|
versionedRule.setProperties(mapProperties(flowAnalysisRuleNode, serviceProvider));
|
||||||
|
versionedRule.setPropertyDescriptors(mapPropertyDescriptors(flowAnalysisRuleNode, serviceProvider, Collections.emptySet(), Collections.emptyMap()));
|
||||||
|
versionedRule.setEnforcementPolicy(flowAnalysisRuleNode.getEnforcementPolicy());
|
||||||
|
versionedRule.setType(flowAnalysisRuleNode.getCanonicalClassName());
|
||||||
|
versionedRule.setScheduledState(flowMappingOptions.getStateLookup().getState(flowAnalysisRuleNode));
|
||||||
|
|
||||||
|
return versionedRule;
|
||||||
|
}
|
||||||
|
|
||||||
public VersionedParameterProvider mapParameterProvider(final ParameterProviderNode parameterProviderNode, final ControllerServiceProvider serviceProvider) {
|
public VersionedParameterProvider mapParameterProvider(final ParameterProviderNode parameterProviderNode, final ControllerServiceProvider serviceProvider) {
|
||||||
final VersionedParameterProvider versionedParameterProvider = new VersionedParameterProvider();
|
final VersionedParameterProvider versionedParameterProvider = new VersionedParameterProvider();
|
||||||
versionedParameterProvider.setIdentifier(parameterProviderNode.getIdentifier());
|
versionedParameterProvider.setIdentifier(parameterProviderNode.getIdentifier());
|
||||||
|
@ -572,7 +594,7 @@ public class NiFiRegistryFlowMapper {
|
||||||
return mapped;
|
return mapped;
|
||||||
}
|
}
|
||||||
|
|
||||||
private String encrypt(final String value) {
|
protected String encrypt(final String value) {
|
||||||
if (value == null) {
|
if (value == null) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
@ -619,6 +641,7 @@ public class NiFiRegistryFlowMapper {
|
||||||
versionedDescriptor.setName(descriptor.getName());
|
versionedDescriptor.setName(descriptor.getName());
|
||||||
versionedDescriptor.setDisplayName(descriptor.getDisplayName());
|
versionedDescriptor.setDisplayName(descriptor.getDisplayName());
|
||||||
versionedDescriptor.setSensitive(descriptor.isSensitive());
|
versionedDescriptor.setSensitive(descriptor.isSensitive());
|
||||||
|
versionedDescriptor.setDynamic(descriptor.isDynamic());
|
||||||
|
|
||||||
final VersionedResourceDefinition versionedResourceDefinition = mapResourceDefinition(descriptor.getResourceDefinition());
|
final VersionedResourceDefinition versionedResourceDefinition = mapResourceDefinition(descriptor.getResourceDefinition());
|
||||||
versionedDescriptor.setResourceDefinition(versionedResourceDefinition);
|
versionedDescriptor.setResourceDefinition(versionedResourceDefinition);
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
|
|
||||||
package org.apache.nifi.registry.flow.mapping;
|
package org.apache.nifi.registry.flow.mapping;
|
||||||
|
|
||||||
|
import org.apache.nifi.flow.VersionedFlowAnalysisRule;
|
||||||
import org.apache.nifi.flow.VersionedFlowRegistryClient;
|
import org.apache.nifi.flow.VersionedFlowRegistryClient;
|
||||||
import org.apache.nifi.flow.VersionedControllerService;
|
import org.apache.nifi.flow.VersionedControllerService;
|
||||||
import org.apache.nifi.flow.VersionedParameterProvider;
|
import org.apache.nifi.flow.VersionedParameterProvider;
|
||||||
|
@ -34,21 +35,30 @@ public class StandardComparableDataFlow implements ComparableDataFlow {
|
||||||
private final VersionedProcessGroup contents;
|
private final VersionedProcessGroup contents;
|
||||||
private final Set<VersionedControllerService> controllerLevelServices;
|
private final Set<VersionedControllerService> controllerLevelServices;
|
||||||
private final Set<VersionedReportingTask> reportingTasks;
|
private final Set<VersionedReportingTask> reportingTasks;
|
||||||
|
private final Set<VersionedFlowAnalysisRule> flowAnalysisRules;
|
||||||
private final Set<VersionedParameterContext> parameterContexts;
|
private final Set<VersionedParameterContext> parameterContexts;
|
||||||
private final Set<VersionedParameterProvider> parameterProviders;
|
private final Set<VersionedParameterProvider> parameterProviders;
|
||||||
private final Set<VersionedFlowRegistryClient> flowRegistryClients;
|
private final Set<VersionedFlowRegistryClient> flowRegistryClients;
|
||||||
|
|
||||||
public StandardComparableDataFlow(final String name, final VersionedProcessGroup contents) {
|
public StandardComparableDataFlow(final String name, final VersionedProcessGroup contents) {
|
||||||
this(name, contents, Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), Collections.emptySet());
|
this(name, contents, Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), Collections.emptySet());
|
||||||
}
|
}
|
||||||
|
|
||||||
public StandardComparableDataFlow(final String name, final VersionedProcessGroup contents, final Set<VersionedControllerService> controllerLevelServices,
|
public StandardComparableDataFlow(
|
||||||
final Set<VersionedReportingTask> reportingTasks, final Set<VersionedParameterContext> parameterContexts,
|
final String name,
|
||||||
final Set<VersionedParameterProvider> parameterProviders, final Set<VersionedFlowRegistryClient> flowRegistryClients) {
|
final VersionedProcessGroup contents,
|
||||||
|
final Set<VersionedControllerService> controllerLevelServices,
|
||||||
|
final Set<VersionedReportingTask> reportingTasks,
|
||||||
|
final Set<VersionedFlowAnalysisRule> flowAnalysisRules,
|
||||||
|
final Set<VersionedParameterContext> parameterContexts,
|
||||||
|
final Set<VersionedParameterProvider> parameterProviders,
|
||||||
|
final Set<VersionedFlowRegistryClient> flowRegistryClients
|
||||||
|
) {
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.contents = contents;
|
this.contents = contents;
|
||||||
this.controllerLevelServices = controllerLevelServices == null ? Collections.emptySet() : new HashSet<>(controllerLevelServices);
|
this.controllerLevelServices = controllerLevelServices == null ? Collections.emptySet() : new HashSet<>(controllerLevelServices);
|
||||||
this.reportingTasks = reportingTasks == null ? Collections.emptySet() : new HashSet<>(reportingTasks);
|
this.reportingTasks = reportingTasks == null ? Collections.emptySet() : new HashSet<>(reportingTasks);
|
||||||
|
this.flowAnalysisRules = flowAnalysisRules == null ? Collections.emptySet() : new HashSet<>(flowAnalysisRules);
|
||||||
this.parameterContexts = parameterContexts == null ? Collections.emptySet() : new HashSet<>(parameterContexts);
|
this.parameterContexts = parameterContexts == null ? Collections.emptySet() : new HashSet<>(parameterContexts);
|
||||||
this.parameterProviders = parameterProviders == null ? Collections.emptySet() : new HashSet<>(parameterProviders);
|
this.parameterProviders = parameterProviders == null ? Collections.emptySet() : new HashSet<>(parameterProviders);
|
||||||
this.flowRegistryClients = flowRegistryClients == null ? Collections.emptySet() : new HashSet<>(flowRegistryClients);
|
this.flowRegistryClients = flowRegistryClients == null ? Collections.emptySet() : new HashSet<>(flowRegistryClients);
|
||||||
|
@ -74,6 +84,11 @@ public class StandardComparableDataFlow implements ComparableDataFlow {
|
||||||
return reportingTasks;
|
return reportingTasks;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<VersionedFlowAnalysisRule> getFlowAnalysisRules() {
|
||||||
|
return flowAnalysisRules;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Set<VersionedParameterContext> getParameterContexts() {
|
public Set<VersionedParameterContext> getParameterContexts() {
|
||||||
return parameterContexts;
|
return parameterContexts;
|
||||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.nifi.controller.service.ControllerServiceDisabledException;
|
||||||
import org.apache.nifi.controller.service.ControllerServiceNode;
|
import org.apache.nifi.controller.service.ControllerServiceNode;
|
||||||
import org.apache.nifi.controller.service.ControllerServiceProvider;
|
import org.apache.nifi.controller.service.ControllerServiceProvider;
|
||||||
import org.apache.nifi.controller.service.ControllerServiceState;
|
import org.apache.nifi.controller.service.ControllerServiceState;
|
||||||
|
import org.apache.nifi.flowanalysis.EnforcementPolicy;
|
||||||
import org.apache.nifi.flow.ExecutionEngine;
|
import org.apache.nifi.flow.ExecutionEngine;
|
||||||
import org.apache.nifi.groups.ProcessGroup;
|
import org.apache.nifi.groups.ProcessGroup;
|
||||||
import org.apache.nifi.nar.ExtensionManager;
|
import org.apache.nifi.nar.ExtensionManager;
|
||||||
|
@ -61,6 +62,8 @@ import org.apache.nifi.registry.ComponentVariableRegistry;
|
||||||
import org.apache.nifi.util.CharacterFilterUtils;
|
import org.apache.nifi.util.CharacterFilterUtils;
|
||||||
import org.apache.nifi.util.FormatUtils;
|
import org.apache.nifi.util.FormatUtils;
|
||||||
import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
|
import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
|
||||||
|
import org.apache.nifi.validation.RuleViolation;
|
||||||
|
import org.apache.nifi.validation.RuleViolationsManager;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
@ -758,6 +761,7 @@ public abstract class AbstractComponentNode implements ComponentNode {
|
||||||
|
|
||||||
protected Collection<ValidationResult> computeValidationErrors(final ValidationContext validationContext) {
|
protected Collection<ValidationResult> computeValidationErrors(final ValidationContext validationContext) {
|
||||||
Throwable failureCause = null;
|
Throwable failureCause = null;
|
||||||
|
|
||||||
try {
|
try {
|
||||||
if (!sensitiveDynamicPropertyNames.get().isEmpty() && !isSupportsSensitiveDynamicProperties()) {
|
if (!sensitiveDynamicPropertyNames.get().isEmpty() && !isSupportsSensitiveDynamicProperties()) {
|
||||||
return Collections.singletonList(
|
return Collections.singletonList(
|
||||||
|
@ -786,6 +790,24 @@ public abstract class AbstractComponentNode implements ComponentNode {
|
||||||
final Collection<ValidationResult> referencedServiceValidationResults = validateReferencedControllerServices(validationContext);
|
final Collection<ValidationResult> referencedServiceValidationResults = validateReferencedControllerServices(validationContext);
|
||||||
validationResults.addAll(referencedServiceValidationResults);
|
validationResults.addAll(referencedServiceValidationResults);
|
||||||
|
|
||||||
|
performFlowAnalysisOnThis();
|
||||||
|
|
||||||
|
RuleViolationsManager ruleViolationsManager = getValidationContextFactory().getRuleViolationsManager();
|
||||||
|
if (ruleViolationsManager != null) {
|
||||||
|
Collection<RuleViolation> ruleViolations = ruleViolationsManager.getRuleViolationsForSubject(getIdentifier());
|
||||||
|
for (RuleViolation ruleViolation : ruleViolations) {
|
||||||
|
if (ruleViolation.getEnforcementPolicy() == EnforcementPolicy.ENFORCE) {
|
||||||
|
validationResults.add(
|
||||||
|
new ValidationResult.Builder()
|
||||||
|
.subject(getComponent().getClass().getSimpleName())
|
||||||
|
.valid(false)
|
||||||
|
.explanation(ruleViolation.getViolationMessage())
|
||||||
|
.build()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
logger.debug("Computed validation errors with Validation Context {}; results = {}", validationContext, validationResults);
|
logger.debug("Computed validation errors with Validation Context {}; results = {}", validationContext, validationResults);
|
||||||
|
|
||||||
return validationResults;
|
return validationResults;
|
||||||
|
@ -994,6 +1016,9 @@ public abstract class AbstractComponentNode implements ComponentNode {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected void performFlowAnalysisOnThis() {
|
||||||
|
}
|
||||||
|
|
||||||
private ValidationResult createInvalidResult(final String serviceId, final String propertyName, final String explanation) {
|
private ValidationResult createInvalidResult(final String serviceId, final String propertyName, final String explanation) {
|
||||||
return new ValidationResult.Builder()
|
return new ValidationResult.Builder()
|
||||||
.input(serviceId)
|
.input(serviceId)
|
||||||
|
|
|
@ -0,0 +1,91 @@
|
||||||
|
/*
|
||||||
|
* 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.controller.service.ControllerServiceNode;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRuleContext;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRule;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRuleState;
|
||||||
|
import org.apache.nifi.flowanalysis.EnforcementPolicy;
|
||||||
|
import org.apache.nifi.logging.ComponentLog;
|
||||||
|
import org.apache.nifi.nar.ExtensionManager;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
public interface FlowAnalysisRuleNode extends ComponentNode {
|
||||||
|
FlowAnalysisRule getFlowAnalysisRule();
|
||||||
|
|
||||||
|
void setEnforcementPolicy(EnforcementPolicy enforcementPolicy);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the enforcement policy of the flow analysis rule
|
||||||
|
*/
|
||||||
|
EnforcementPolicy getEnforcementPolicy();
|
||||||
|
|
||||||
|
void setFlowAnalysisRule(LoggableComponent<FlowAnalysisRule> flowAnalysisRule);
|
||||||
|
|
||||||
|
FlowAnalysisRuleContext getFlowAnalysisRuleContext();
|
||||||
|
|
||||||
|
ConfigurationContext getConfigurationContext();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the current state of the flow analysis rule
|
||||||
|
*/
|
||||||
|
FlowAnalysisRuleState getState();
|
||||||
|
|
||||||
|
boolean isEnabled();
|
||||||
|
|
||||||
|
String getComments();
|
||||||
|
|
||||||
|
void setComments(String comment);
|
||||||
|
|
||||||
|
void verifyCanDisable();
|
||||||
|
|
||||||
|
void verifyCanEnable();
|
||||||
|
|
||||||
|
void verifyCanEnable(Set<ControllerServiceNode> ignoredServices);
|
||||||
|
|
||||||
|
void verifyCanDelete();
|
||||||
|
|
||||||
|
void verifyCanUpdate();
|
||||||
|
|
||||||
|
void verifyCanClearState();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Verifies that the Flow Analysis Rule is in a state in which it can verify a configuration by calling
|
||||||
|
* {@link #verifyConfiguration(ConfigurationContext, ComponentLog, ExtensionManager)}.
|
||||||
|
*
|
||||||
|
* @throws IllegalStateException if not in a state in which configuration can be verified
|
||||||
|
*/
|
||||||
|
void verifyCanPerformVerification();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Verifies that the given configuration is valid for the Flow Analysis Rule
|
||||||
|
*
|
||||||
|
* @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);
|
||||||
|
|
||||||
|
void enable();
|
||||||
|
|
||||||
|
void disable();
|
||||||
|
}
|
|
@ -37,6 +37,7 @@ import org.apache.nifi.scheduling.SchedulingStrategy;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.ScheduledExecutorService;
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
|
@ -284,6 +285,12 @@ public abstract class ProcessorNode extends AbstractComponentNode implements Con
|
||||||
*/
|
*/
|
||||||
public abstract ScheduledState getDesiredState();
|
public abstract ScheduledState getDesiredState();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void performFlowAnalysisOnThis() {
|
||||||
|
Optional.ofNullable(getValidationContextFactory().getFlowAnalyzer())
|
||||||
|
.ifPresent(flowAnalyzer -> flowAnalyzer.analyzeProcessor(this));
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method will be called once the processor's configuration has been restored (on startup, reload, e.g.)
|
* This method will be called once the processor's configuration has been restored (on startup, reload, e.g.)
|
||||||
*
|
*
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.nifi.controller;
|
||||||
import org.apache.nifi.bundle.BundleCoordinate;
|
import org.apache.nifi.bundle.BundleCoordinate;
|
||||||
import org.apache.nifi.controller.exception.ControllerServiceInstantiationException;
|
import org.apache.nifi.controller.exception.ControllerServiceInstantiationException;
|
||||||
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
|
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
|
||||||
|
import org.apache.nifi.controller.flowanalysis.FlowAnalysisRuleInstantiationException;
|
||||||
import org.apache.nifi.controller.parameter.ParameterProviderInstantiationException;
|
import org.apache.nifi.controller.parameter.ParameterProviderInstantiationException;
|
||||||
import org.apache.nifi.controller.flowrepository.FlowRepositoryClientInstantiationException;
|
import org.apache.nifi.controller.flowrepository.FlowRepositoryClientInstantiationException;
|
||||||
import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
|
import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
|
||||||
|
@ -69,6 +70,17 @@ public interface ReloadComponent {
|
||||||
void reload(ReportingTaskNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls)
|
void reload(ReportingTaskNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls)
|
||||||
throws ReportingTaskInstantiationException;
|
throws ReportingTaskInstantiationException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Changes the underlying FlowAnalysisRule held by the node to an instance of the new type.
|
||||||
|
*
|
||||||
|
* @param existingNode the FlowAnalysisRuleNode 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 FlowAnalysisRuleInstantiationException if unable to create an instance of the new type
|
||||||
|
*/
|
||||||
|
void reload(FlowAnalysisRuleNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls)
|
||||||
|
throws FlowAnalysisRuleInstantiationException;
|
||||||
/**
|
/**
|
||||||
* Changes the underlying ParameterProvider held by the node to an instance of the new type.
|
* Changes the underlying ParameterProvider held by the node to an instance of the new type.
|
||||||
*
|
*
|
||||||
|
|
|
@ -18,7 +18,9 @@ package org.apache.nifi.controller;
|
||||||
|
|
||||||
import org.apache.nifi.components.PropertyDescriptor;
|
import org.apache.nifi.components.PropertyDescriptor;
|
||||||
import org.apache.nifi.components.ValidationContext;
|
import org.apache.nifi.components.ValidationContext;
|
||||||
|
import org.apache.nifi.controller.flowanalysis.FlowAnalyzer;
|
||||||
import org.apache.nifi.parameter.ParameterContext;
|
import org.apache.nifi.parameter.ParameterContext;
|
||||||
|
import org.apache.nifi.validation.RuleViolationsManager;
|
||||||
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
@ -27,4 +29,7 @@ public interface ValidationContextFactory {
|
||||||
ValidationContext newValidationContext(Map<PropertyDescriptor, PropertyConfiguration> properties, String annotationData, String groupId, String componentId, ParameterContext parameterContext,
|
ValidationContext newValidationContext(Map<PropertyDescriptor, PropertyConfiguration> properties, String annotationData, String groupId, String componentId, ParameterContext parameterContext,
|
||||||
boolean validateConnections);
|
boolean validateConnections);
|
||||||
|
|
||||||
|
RuleViolationsManager getRuleViolationsManager();
|
||||||
|
|
||||||
|
FlowAnalyzer getFlowAnalyzer();
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,10 +21,12 @@ import org.apache.nifi.connectable.Connectable;
|
||||||
import org.apache.nifi.connectable.Connection;
|
import org.apache.nifi.connectable.Connection;
|
||||||
import org.apache.nifi.connectable.Funnel;
|
import org.apache.nifi.connectable.Funnel;
|
||||||
import org.apache.nifi.connectable.Port;
|
import org.apache.nifi.connectable.Port;
|
||||||
|
import org.apache.nifi.controller.FlowAnalysisRuleNode;
|
||||||
import org.apache.nifi.controller.ParameterProviderNode;
|
import org.apache.nifi.controller.ParameterProviderNode;
|
||||||
import org.apache.nifi.controller.ProcessorNode;
|
import org.apache.nifi.controller.ProcessorNode;
|
||||||
import org.apache.nifi.controller.ReportingTaskNode;
|
import org.apache.nifi.controller.ReportingTaskNode;
|
||||||
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
|
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
|
||||||
|
import org.apache.nifi.controller.flowanalysis.FlowAnalyzer;
|
||||||
import org.apache.nifi.controller.label.Label;
|
import org.apache.nifi.controller.label.Label;
|
||||||
import org.apache.nifi.controller.parameter.ParameterProviderLookup;
|
import org.apache.nifi.controller.parameter.ParameterProviderLookup;
|
||||||
import org.apache.nifi.controller.service.ControllerServiceNode;
|
import org.apache.nifi.controller.service.ControllerServiceNode;
|
||||||
|
@ -34,6 +36,7 @@ import org.apache.nifi.groups.RemoteProcessGroup;
|
||||||
import org.apache.nifi.parameter.Parameter;
|
import org.apache.nifi.parameter.Parameter;
|
||||||
import org.apache.nifi.parameter.ParameterContext;
|
import org.apache.nifi.parameter.ParameterContext;
|
||||||
import org.apache.nifi.parameter.ParameterContextManager;
|
import org.apache.nifi.parameter.ParameterContextManager;
|
||||||
|
import org.apache.nifi.validation.RuleViolationsManager;
|
||||||
import org.apache.nifi.parameter.ParameterProviderConfiguration;
|
import org.apache.nifi.parameter.ParameterProviderConfiguration;
|
||||||
import org.apache.nifi.registry.flow.FlowRegistryClientNode;
|
import org.apache.nifi.registry.flow.FlowRegistryClientNode;
|
||||||
import org.apache.nifi.web.api.dto.FlowSnippetDTO;
|
import org.apache.nifi.web.api.dto.FlowSnippetDTO;
|
||||||
|
@ -392,7 +395,7 @@ public interface FlowManager extends ParameterProviderLookup {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return the number of each type of component (Processor, Controller Service, Process Group, Funnel, Input Port, Output Port,
|
* @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)
|
* Parameter Provider, Reporting Task, Flow Analysis Rule, Remote Process Group)
|
||||||
*/
|
*/
|
||||||
Map<String, Integer> getComponentCounts();
|
Map<String, Integer> getComponentCounts();
|
||||||
|
|
||||||
|
@ -403,10 +406,39 @@ public interface FlowManager extends ParameterProviderLookup {
|
||||||
* Controller Services
|
* Controller Services
|
||||||
* Templates
|
* Templates
|
||||||
* Reporting Tasks
|
* Reporting Tasks
|
||||||
|
* Flow Analysis Rules
|
||||||
* Parameter Contexts
|
* Parameter Contexts
|
||||||
* Flow Registries
|
* Flow Registries
|
||||||
*
|
*
|
||||||
* @throws IllegalStateException if any of the components is not in a state that it can be deleted.
|
* @throws IllegalStateException if any of the components is not in a state that it can be deleted.
|
||||||
*/
|
*/
|
||||||
void purge();
|
void purge();
|
||||||
|
|
||||||
|
// Flow Analysis
|
||||||
|
FlowAnalysisRuleNode createFlowAnalysisRule(
|
||||||
|
final String type,
|
||||||
|
final String id,
|
||||||
|
final BundleCoordinate bundleCoordinate,
|
||||||
|
final boolean firstTimeAdded
|
||||||
|
);
|
||||||
|
|
||||||
|
FlowAnalysisRuleNode createFlowAnalysisRule(
|
||||||
|
final String type,
|
||||||
|
final String id,
|
||||||
|
final BundleCoordinate bundleCoordinate,
|
||||||
|
final Set<URL> additionalUrls,
|
||||||
|
final boolean firstTimeAdded,
|
||||||
|
final boolean register,
|
||||||
|
final String classloaderIsolationKey
|
||||||
|
);
|
||||||
|
|
||||||
|
FlowAnalysisRuleNode getFlowAnalysisRuleNode(final String taskId);
|
||||||
|
|
||||||
|
void removeFlowAnalysisRule(final FlowAnalysisRuleNode reportingTask);
|
||||||
|
|
||||||
|
Set<FlowAnalysisRuleNode> getAllFlowAnalysisRules();
|
||||||
|
|
||||||
|
FlowAnalyzer getFlowAnalyzer();
|
||||||
|
|
||||||
|
RuleViolationsManager getRuleViolationsManager();
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,6 +18,7 @@
|
||||||
package org.apache.nifi.controller.flow;
|
package org.apache.nifi.controller.flow;
|
||||||
|
|
||||||
import org.apache.nifi.flow.VersionedControllerService;
|
import org.apache.nifi.flow.VersionedControllerService;
|
||||||
|
import org.apache.nifi.flow.VersionedFlowAnalysisRule;
|
||||||
import org.apache.nifi.flow.VersionedFlowRegistryClient;
|
import org.apache.nifi.flow.VersionedFlowRegistryClient;
|
||||||
import org.apache.nifi.flow.VersionedParameterProvider;
|
import org.apache.nifi.flow.VersionedParameterProvider;
|
||||||
import org.apache.nifi.flow.VersionedProcessGroup;
|
import org.apache.nifi.flow.VersionedProcessGroup;
|
||||||
|
@ -35,6 +36,7 @@ public class VersionedDataflow {
|
||||||
private List<VersionedParameterProvider> parameterProviders;
|
private List<VersionedParameterProvider> parameterProviders;
|
||||||
private List<VersionedControllerService> controllerServices;
|
private List<VersionedControllerService> controllerServices;
|
||||||
private List<VersionedReportingTask> reportingTasks;
|
private List<VersionedReportingTask> reportingTasks;
|
||||||
|
private List<VersionedFlowAnalysisRule> flowAnalysisRules;
|
||||||
private Set<VersionedTemplate> templates;
|
private Set<VersionedTemplate> templates;
|
||||||
private VersionedProcessGroup rootGroup;
|
private VersionedProcessGroup rootGroup;
|
||||||
|
|
||||||
|
@ -86,6 +88,14 @@ public class VersionedDataflow {
|
||||||
this.reportingTasks = reportingTasks;
|
this.reportingTasks = reportingTasks;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public List<VersionedFlowAnalysisRule> getFlowAnalysisRules() {
|
||||||
|
return flowAnalysisRules;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setFlowAnalysisRules(List<VersionedFlowAnalysisRule> flowAnalysisRules) {
|
||||||
|
this.flowAnalysisRules = flowAnalysisRules;
|
||||||
|
}
|
||||||
|
|
||||||
public List<VersionedParameterProvider> getParameterProviders() {
|
public List<VersionedParameterProvider> getParameterProviders() {
|
||||||
return parameterProviders;
|
return parameterProviders;
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,31 @@
|
||||||
|
/*
|
||||||
|
* 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.flowanalysis;
|
||||||
|
|
||||||
|
public class FlowAnalysisRuleInstantiationException extends Exception {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1l;
|
||||||
|
|
||||||
|
public FlowAnalysisRuleInstantiationException(final String className, final Throwable t) {
|
||||||
|
super(className, t);
|
||||||
|
}
|
||||||
|
|
||||||
|
public FlowAnalysisRuleInstantiationException(final String message) {
|
||||||
|
super(message);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,99 @@
|
||||||
|
/*
|
||||||
|
* 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.flowanalysis;
|
||||||
|
|
||||||
|
import org.apache.nifi.bundle.BundleCoordinate;
|
||||||
|
import org.apache.nifi.controller.FlowAnalysisRuleNode;
|
||||||
|
import org.apache.nifi.nar.ExtensionManager;
|
||||||
|
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A FlowAnalysisRuleProvider is responsible for providing management of, and
|
||||||
|
* access to, Flow Analysis Rules
|
||||||
|
*/
|
||||||
|
public interface FlowAnalysisRuleProvider {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a new instance of a flow analysis rule
|
||||||
|
*
|
||||||
|
* @param type the type (fully qualified class name) of the flow analysis rule
|
||||||
|
* to instantiate
|
||||||
|
* @param id the identifier for the Flow Analysis Rule
|
||||||
|
* @param bundleCoordinate the bundle coordinate for the type of flow analysis rule
|
||||||
|
* @param firstTimeAdded whether or not this is the first time that the
|
||||||
|
* flow analysis rule is being added to the flow. I.e., this will be true only
|
||||||
|
* when the user adds the flow analysis rule to the flow, not when the flow is
|
||||||
|
* being restored after a restart of the software
|
||||||
|
*
|
||||||
|
* @return the FlowAnalysisRuleNode that is used to manage the flow analysis rule
|
||||||
|
*
|
||||||
|
* @throws FlowAnalysisRuleInstantiationException if unable to create the
|
||||||
|
* Flow Analysis Rule
|
||||||
|
*/
|
||||||
|
FlowAnalysisRuleNode createFlowAnalysisRule(String type, String id, BundleCoordinate bundleCoordinate, boolean firstTimeAdded) throws FlowAnalysisRuleInstantiationException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param identifier of node
|
||||||
|
* @return the flow analysis rule that has the given identifier, or
|
||||||
|
* <code>null</code> if no flow analysis rule exists with that ID
|
||||||
|
*/
|
||||||
|
FlowAnalysisRuleNode getFlowAnalysisRuleNode(String identifier);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a Set of all Flow Analysis Rules that exist for this service
|
||||||
|
* provider
|
||||||
|
*/
|
||||||
|
Set<FlowAnalysisRuleNode> getAllFlowAnalysisRules();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Removes the given flow analysis rule from the flow
|
||||||
|
*
|
||||||
|
* @param flowAnalysisRule
|
||||||
|
*
|
||||||
|
* @throws IllegalStateException if the flow analysis rule cannot be removed
|
||||||
|
* because it is not disabled, or if the flow analysis rule is not known in the
|
||||||
|
* flow
|
||||||
|
*/
|
||||||
|
void removeFlowAnalysisRule(FlowAnalysisRuleNode flowAnalysisRule);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Enables the flow analysis rule
|
||||||
|
*
|
||||||
|
* @param flowAnalysisRule
|
||||||
|
*
|
||||||
|
* @throws IllegalStateException if the FlowAnalysisRule's state is not
|
||||||
|
* DISABLED
|
||||||
|
*/
|
||||||
|
void enableFlowAnalysisRule(FlowAnalysisRuleNode flowAnalysisRule);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Disables the flow analysis rule
|
||||||
|
*
|
||||||
|
* @param flowAnalysisRule
|
||||||
|
*
|
||||||
|
* @throws IllegalStateException if the FlowAnalysisRule's state is not
|
||||||
|
* ENABLED
|
||||||
|
*/
|
||||||
|
void disableFlowAnalysisRule(FlowAnalysisRuleNode flowAnalysisRule);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the ExtensionManager instance used by this provider
|
||||||
|
*/
|
||||||
|
ExtensionManager getExtensionManager();
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,49 @@
|
||||||
|
/*
|
||||||
|
* 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.flowanalysis;
|
||||||
|
|
||||||
|
import org.apache.nifi.controller.ProcessorNode;
|
||||||
|
import org.apache.nifi.controller.service.ControllerServiceNode;
|
||||||
|
import org.apache.nifi.flow.VersionedProcessGroup;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Analyzes components, parts or the entirety of the flow.
|
||||||
|
*/
|
||||||
|
public interface FlowAnalyzer {
|
||||||
|
/**
|
||||||
|
* Analyzes a processor
|
||||||
|
*
|
||||||
|
* @param processorNode the processor (as a node) to be analyzed
|
||||||
|
*/
|
||||||
|
void analyzeProcessor(ProcessorNode processorNode);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Analyzes a controller service
|
||||||
|
*
|
||||||
|
* @param controllerServiceNode the controller service (as a node) to be analyzed
|
||||||
|
*/
|
||||||
|
void analyzeControllerService(ControllerServiceNode controllerServiceNode);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Analyze the flow or a part of it
|
||||||
|
*
|
||||||
|
* @param processGroup The process group (as a {@link org.apache.nifi.flow.VersionedComponent VersionedComponent})
|
||||||
|
* representing (a part of) the flow to be analyzed. Recursive - all child process groups will
|
||||||
|
* be analyzed as well.
|
||||||
|
*/
|
||||||
|
void analyzeProcessGroup(VersionedProcessGroup processGroup);
|
||||||
|
}
|
|
@ -20,8 +20,10 @@ package org.apache.nifi.registry.flow.mapping;
|
||||||
import java.nio.charset.StandardCharsets;
|
import java.nio.charset.StandardCharsets;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
|
import java.util.function.Function;
|
||||||
|
|
||||||
public interface ComponentIdLookup {
|
public interface ComponentIdLookup {
|
||||||
|
Function<String, String> DEFAULT_VERSIONED_UUID_GENERATOR = componentId -> UUID.nameUUIDFromBytes(componentId.getBytes(StandardCharsets.UTF_8)).toString();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Given a component identifier and an optional Versioned Component ID, returns the identifier to use for the component
|
* Given a component identifier and an optional Versioned Component ID, returns the identifier to use for the component
|
||||||
|
@ -29,20 +31,27 @@ public interface ComponentIdLookup {
|
||||||
* @param componentId the ID of the component
|
* @param componentId the ID of the component
|
||||||
* @return the ID to use for mapping a component to a Versioned Component
|
* @return the ID to use for mapping a component to a Versioned Component
|
||||||
*/
|
*/
|
||||||
String getComponentId(Optional<String> currentVersionedId, String componentId);
|
default String getComponentId(Optional<String> currentVersionedId, String componentId) {
|
||||||
|
return getComponentId(currentVersionedId, componentId, DEFAULT_VERSIONED_UUID_GENERATOR);
|
||||||
|
}
|
||||||
|
|
||||||
|
String getComponentId(Optional<String> currentVersionedId, String componentId, Function<String, String> versionedUuidGenerator);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Uses the Versioned Component ID, if it is present, or else generates a new Versioned Component ID based on the Component ID
|
* Uses the Versioned Component ID, if it is present, or else generates a new Versioned Component ID based on the Component ID
|
||||||
*/
|
*/
|
||||||
ComponentIdLookup VERSIONED_OR_GENERATE = new ComponentIdLookup() {
|
ComponentIdLookup VERSIONED_OR_GENERATE = new ComponentIdLookup() {
|
||||||
@Override
|
@Override
|
||||||
public String getComponentId(final Optional<String> currentVersionedId, final String componentId) {
|
public String getComponentId(
|
||||||
|
final Optional<String> currentVersionedId,
|
||||||
|
final String componentId,
|
||||||
|
final Function<String, String> versionedUuidGenerator
|
||||||
|
) {
|
||||||
if (currentVersionedId.isPresent()) {
|
if (currentVersionedId.isPresent()) {
|
||||||
return currentVersionedId.get();
|
return currentVersionedId.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
return UUID.nameUUIDFromBytes(componentId.getBytes(StandardCharsets.UTF_8)).toString();
|
return versionedUuidGenerator.apply(componentId);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -51,5 +60,5 @@ public interface ComponentIdLookup {
|
||||||
/**
|
/**
|
||||||
* Always uses the Component ID
|
* Always uses the Component ID
|
||||||
*/
|
*/
|
||||||
ComponentIdLookup USE_COMPONENT_ID = (versioned, componentId) -> componentId;
|
ComponentIdLookup USE_COMPONENT_ID = (versioned, componentId, versionedUuidGenerator) -> componentId;
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,10 +18,12 @@
|
||||||
package org.apache.nifi.registry.flow.mapping;
|
package org.apache.nifi.registry.flow.mapping;
|
||||||
|
|
||||||
import org.apache.nifi.connectable.Port;
|
import org.apache.nifi.connectable.Port;
|
||||||
|
import org.apache.nifi.controller.FlowAnalysisRuleNode;
|
||||||
import org.apache.nifi.controller.ProcessorNode;
|
import org.apache.nifi.controller.ProcessorNode;
|
||||||
import org.apache.nifi.controller.ReportingTaskNode;
|
import org.apache.nifi.controller.ReportingTaskNode;
|
||||||
import org.apache.nifi.controller.service.ControllerServiceNode;
|
import org.apache.nifi.controller.service.ControllerServiceNode;
|
||||||
import org.apache.nifi.flow.ScheduledState;
|
import org.apache.nifi.flow.ScheduledState;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRuleState;
|
||||||
import org.apache.nifi.groups.ProcessGroup;
|
import org.apache.nifi.groups.ProcessGroup;
|
||||||
import org.apache.nifi.groups.StatelessGroupScheduledState;
|
import org.apache.nifi.groups.StatelessGroupScheduledState;
|
||||||
|
|
||||||
|
@ -32,6 +34,8 @@ public interface VersionedComponentStateLookup {
|
||||||
|
|
||||||
ScheduledState getState(ReportingTaskNode taskNode);
|
ScheduledState getState(ReportingTaskNode taskNode);
|
||||||
|
|
||||||
|
ScheduledState getState(FlowAnalysisRuleNode ruleNode);
|
||||||
|
|
||||||
ScheduledState getState(ControllerServiceNode serviceNode);
|
ScheduledState getState(ControllerServiceNode serviceNode);
|
||||||
|
|
||||||
ScheduledState getState(ProcessGroup processGroup);
|
ScheduledState getState(ProcessGroup processGroup);
|
||||||
|
@ -56,6 +60,11 @@ public interface VersionedComponentStateLookup {
|
||||||
return taskNode.getScheduledState() == org.apache.nifi.controller.ScheduledState.DISABLED ? ScheduledState.DISABLED : ScheduledState.ENABLED;
|
return taskNode.getScheduledState() == org.apache.nifi.controller.ScheduledState.DISABLED ? ScheduledState.DISABLED : ScheduledState.ENABLED;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ScheduledState getState(final FlowAnalysisRuleNode ruleNode) {
|
||||||
|
return ruleNode.getState() == FlowAnalysisRuleState.DISABLED ? ScheduledState.DISABLED : ScheduledState.ENABLED;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ScheduledState getState(final ControllerServiceNode serviceNode) {
|
public ScheduledState getState(final ControllerServiceNode serviceNode) {
|
||||||
return ScheduledState.DISABLED;
|
return ScheduledState.DISABLED;
|
||||||
|
@ -86,6 +95,17 @@ public interface VersionedComponentStateLookup {
|
||||||
return map(taskNode.getScheduledState());
|
return map(taskNode.getScheduledState());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ScheduledState getState(final FlowAnalysisRuleNode ruleNode) {
|
||||||
|
switch (ruleNode.getState()) {
|
||||||
|
case DISABLED:
|
||||||
|
return ScheduledState.DISABLED;
|
||||||
|
case ENABLED:
|
||||||
|
default:
|
||||||
|
return ScheduledState.ENABLED;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ScheduledState getState(final ControllerServiceNode serviceNode) {
|
public ScheduledState getState(final ControllerServiceNode serviceNode) {
|
||||||
switch (serviceNode.getState()) {
|
switch (serviceNode.getState()) {
|
||||||
|
|
|
@ -0,0 +1,193 @@
|
||||||
|
/*
|
||||||
|
* 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.validation;
|
||||||
|
|
||||||
|
import org.apache.commons.lang3.builder.EqualsBuilder;
|
||||||
|
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
||||||
|
import org.apache.nifi.flowanalysis.EnforcementPolicy;
|
||||||
|
|
||||||
|
import java.util.StringJoiner;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A result of a rule violation produced during a flow analysis.
|
||||||
|
* Violations produced by previous analysis runs may be overridden by new ones.
|
||||||
|
* A violation is identified by the scope, subjectId, ruleId and issueId properties.
|
||||||
|
*/
|
||||||
|
public class RuleViolation {
|
||||||
|
private final EnforcementPolicy enforcementPolicy;
|
||||||
|
private final String scope;
|
||||||
|
private final String subjectId;
|
||||||
|
private final String subjectDisplayName;
|
||||||
|
private final String groupId;
|
||||||
|
private final String ruleId;
|
||||||
|
private final String issueId;
|
||||||
|
private final String violationMessage;
|
||||||
|
private final String violationExplanation;
|
||||||
|
|
||||||
|
private boolean available;
|
||||||
|
|
||||||
|
public RuleViolation(
|
||||||
|
EnforcementPolicy enforcementPolicy,
|
||||||
|
String scope,
|
||||||
|
String subjectId,
|
||||||
|
String subjectDisplayName,
|
||||||
|
String groupId,
|
||||||
|
String ruleId,
|
||||||
|
String issueId,
|
||||||
|
String violationMessage,
|
||||||
|
String violationExplanation
|
||||||
|
) {
|
||||||
|
this.enforcementPolicy = enforcementPolicy;
|
||||||
|
this.scope = scope;
|
||||||
|
this.subjectId = subjectId;
|
||||||
|
this.subjectDisplayName = subjectDisplayName;
|
||||||
|
this.groupId = groupId;
|
||||||
|
this.ruleId = ruleId;
|
||||||
|
this.issueId = issueId;
|
||||||
|
this.violationMessage = violationMessage;
|
||||||
|
this.violationExplanation = violationExplanation;
|
||||||
|
this.available = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the type of the rule that produced this violation
|
||||||
|
*/
|
||||||
|
public EnforcementPolicy getEnforcementPolicy() {
|
||||||
|
return enforcementPolicy;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the scope of the analysis that produced this violation. The id of the component or the process group that was
|
||||||
|
* the subject of the analysis.
|
||||||
|
*/
|
||||||
|
public String getScope() {
|
||||||
|
return scope;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the id of the subject that violated the rule (component or a process group).
|
||||||
|
*/
|
||||||
|
public String getSubjectId() {
|
||||||
|
return subjectId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the displayed name of the subject that violated the rule
|
||||||
|
*/
|
||||||
|
public String getSubjectDisplayName() {
|
||||||
|
return subjectDisplayName;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return group id - if this violation is a result of a component analysis, then the id of the group of the component.
|
||||||
|
* If this violation is a result of a group analysis, then the id of that group itself.
|
||||||
|
*/
|
||||||
|
public String getGroupId() {
|
||||||
|
return groupId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a rule-defined id that corresponds to a unique type of issue recognized by the rule.
|
||||||
|
*/
|
||||||
|
public String getIssueId() {
|
||||||
|
return issueId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the id of the rule that produced this violation
|
||||||
|
*/
|
||||||
|
public String getRuleId() {
|
||||||
|
return ruleId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the violation message
|
||||||
|
*/
|
||||||
|
public String getViolationMessage() {
|
||||||
|
return violationMessage;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a detailed explanation of the nature of the violation
|
||||||
|
*/
|
||||||
|
public String getViolationExplanation() {
|
||||||
|
return violationExplanation;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Used by the framework internally to manage lifecycle
|
||||||
|
*/
|
||||||
|
public boolean isAvailable() {
|
||||||
|
return available;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setAvailable(boolean available) {
|
||||||
|
this.available = available;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return new StringJoiner(",\n\t", RuleViolation.class.getSimpleName() + "[\n\t", "\n]")
|
||||||
|
.add("enforcementPolicy=" + enforcementPolicy)
|
||||||
|
.add("scope='" + scope + "'")
|
||||||
|
.add("subjectId='" + subjectId + "'")
|
||||||
|
.add("subjectDisplayName='" + subjectDisplayName + "'")
|
||||||
|
.add("groupId='" + groupId + "'")
|
||||||
|
.add("issueId='" + issueId + "'")
|
||||||
|
.add("ruleId='" + ruleId + "'")
|
||||||
|
.add("violationMessage='" + violationMessage + "'")
|
||||||
|
.add("violationExplanation='" + violationExplanation + "'")
|
||||||
|
.add("available=" + available)
|
||||||
|
.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (this == o) return true;
|
||||||
|
|
||||||
|
if (o == null || getClass() != o.getClass()) return false;
|
||||||
|
|
||||||
|
RuleViolation that = (RuleViolation) o;
|
||||||
|
|
||||||
|
return new EqualsBuilder()
|
||||||
|
.append(enforcementPolicy, that.enforcementPolicy)
|
||||||
|
.append(scope, that.scope)
|
||||||
|
.append(subjectId, that.subjectId)
|
||||||
|
.append(groupId, that.groupId)
|
||||||
|
.append(issueId, that.issueId)
|
||||||
|
.append(ruleId, that.ruleId)
|
||||||
|
.append(violationMessage, that.violationMessage)
|
||||||
|
.append(violationExplanation, that.violationExplanation)
|
||||||
|
.append(available, that.available)
|
||||||
|
.isEquals();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return new HashCodeBuilder(17, 37)
|
||||||
|
.append(enforcementPolicy)
|
||||||
|
.append(scope)
|
||||||
|
.append(subjectId)
|
||||||
|
.append(groupId)
|
||||||
|
.append(issueId)
|
||||||
|
.append(ruleId)
|
||||||
|
.append(violationMessage)
|
||||||
|
.append(violationExplanation)
|
||||||
|
.append(available)
|
||||||
|
.toHashCode();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,84 @@
|
||||||
|
/*
|
||||||
|
* 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.validation;
|
||||||
|
|
||||||
|
import org.apache.nifi.flow.VersionedComponent;
|
||||||
|
import org.apache.nifi.flow.VersionedProcessGroup;
|
||||||
|
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Manages {@link RuleViolation}s produced during flow analysis
|
||||||
|
*/
|
||||||
|
public interface RuleViolationsManager {
|
||||||
|
/**
|
||||||
|
* Add or update rule violations created during the analysis of a component
|
||||||
|
*
|
||||||
|
* @param subjectId The id of the component that was analyzed
|
||||||
|
* @param violations The violations to be added or updated
|
||||||
|
*/
|
||||||
|
void upsertComponentViolations(String subjectId, Collection<RuleViolation> violations);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add or update rule violations created during the analysis of a process group
|
||||||
|
*
|
||||||
|
* @param processGroup The process group that was analyzed
|
||||||
|
* @param violations Violations to be added that scoped to a process group (the one that was analyzed or one of it's children)
|
||||||
|
* @param componentToRuleViolations Violations to be added scoped to components under the analyzed process group (or one of it's children)
|
||||||
|
*/
|
||||||
|
void upsertGroupViolations(VersionedProcessGroup processGroup, Collection<RuleViolation> violations, Map<VersionedComponent, Collection<RuleViolation>> componentToRuleViolations);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns rule violations tied to a component or process group with a given id
|
||||||
|
*
|
||||||
|
* @param subjectId The id of the component or process group
|
||||||
|
* @return Violations tied to a component or process group with the given subjectId
|
||||||
|
*/
|
||||||
|
Collection<RuleViolation> getRuleViolationsForSubject(String subjectId);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a list of violations with the given groupId (non-recursive)
|
||||||
|
*
|
||||||
|
* @return Violations with the given groupId
|
||||||
|
*/
|
||||||
|
Collection<RuleViolation> getRuleViolationsForGroup(String groupId);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return All current rule violations
|
||||||
|
*/
|
||||||
|
Collection<RuleViolation> getAllRuleViolations();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Remove all rule violations tied to a component or process group with a given id
|
||||||
|
*
|
||||||
|
* @param subjectId The id of the component or process group
|
||||||
|
*/
|
||||||
|
void removeRuleViolationsForSubject(String subjectId);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Remove all rule violations produced by the rule with a given id
|
||||||
|
*
|
||||||
|
* @param ruleId The id of the rule
|
||||||
|
*/
|
||||||
|
void removeRuleViolationsForRule(String ruleId);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Removes empty entries from the map storing the rule violations
|
||||||
|
*/
|
||||||
|
void cleanUp();
|
||||||
|
}
|
|
@ -46,6 +46,10 @@ public class TriggerValidationTask implements Runnable {
|
||||||
validationTrigger.trigger(node);
|
validationTrigger.trigger(node);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
for (final ComponentNode node : flowManager.getAllFlowAnalysisRules()) {
|
||||||
|
validationTrigger.trigger(node);
|
||||||
|
}
|
||||||
|
|
||||||
for (final ComponentNode node : flowManager.getAllParameterProviders()) {
|
for (final ComponentNode node : flowManager.getAllParameterProviders()) {
|
||||||
validationTrigger.trigger(node);
|
validationTrigger.trigger(node);
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,6 +29,10 @@ import org.apache.nifi.components.state.StateManager;
|
||||||
import org.apache.nifi.components.state.StateManagerProvider;
|
import org.apache.nifi.components.state.StateManagerProvider;
|
||||||
import org.apache.nifi.components.validation.ValidationTrigger;
|
import org.apache.nifi.components.validation.ValidationTrigger;
|
||||||
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
|
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
|
||||||
|
import org.apache.nifi.controller.flowanalysis.FlowAnalysisRuleInstantiationException;
|
||||||
|
import org.apache.nifi.controller.flowanalysis.FlowAnalyzer;
|
||||||
|
import org.apache.nifi.controller.flowanalysis.StandardFlowAnalysisInitializationContext;
|
||||||
|
import org.apache.nifi.controller.flowanalysis.StandardFlowAnalysisRuleNode;
|
||||||
import org.apache.nifi.controller.flowrepository.FlowRepositoryClientInstantiationException;
|
import org.apache.nifi.controller.flowrepository.FlowRepositoryClientInstantiationException;
|
||||||
import org.apache.nifi.controller.kerberos.KerberosConfig;
|
import org.apache.nifi.controller.kerberos.KerberosConfig;
|
||||||
import org.apache.nifi.controller.parameter.ParameterProviderInstantiationException;
|
import org.apache.nifi.controller.parameter.ParameterProviderInstantiationException;
|
||||||
|
@ -43,6 +47,9 @@ import org.apache.nifi.controller.service.GhostControllerService;
|
||||||
import org.apache.nifi.controller.service.StandardControllerServiceInitializationContext;
|
import org.apache.nifi.controller.service.StandardControllerServiceInitializationContext;
|
||||||
import org.apache.nifi.controller.service.StandardControllerServiceInvocationHandler;
|
import org.apache.nifi.controller.service.StandardControllerServiceInvocationHandler;
|
||||||
import org.apache.nifi.controller.service.StandardControllerServiceNode;
|
import org.apache.nifi.controller.service.StandardControllerServiceNode;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRule;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRuleInitializationContext;
|
||||||
|
import org.apache.nifi.flowanalysis.GhostFlowAnalysisRule;
|
||||||
import org.apache.nifi.logging.ComponentLog;
|
import org.apache.nifi.logging.ComponentLog;
|
||||||
import org.apache.nifi.logging.LoggingContext;
|
import org.apache.nifi.logging.LoggingContext;
|
||||||
import org.apache.nifi.logging.StandardLoggingContext;
|
import org.apache.nifi.logging.StandardLoggingContext;
|
||||||
|
@ -76,6 +83,7 @@ import org.apache.nifi.reporting.InitializationException;
|
||||||
import org.apache.nifi.reporting.ReportingInitializationContext;
|
import org.apache.nifi.reporting.ReportingInitializationContext;
|
||||||
import org.apache.nifi.reporting.ReportingTask;
|
import org.apache.nifi.reporting.ReportingTask;
|
||||||
import org.apache.nifi.scheduling.SchedulingStrategy;
|
import org.apache.nifi.scheduling.SchedulingStrategy;
|
||||||
|
import org.apache.nifi.validation.RuleViolationsManager;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
@ -106,6 +114,8 @@ public class ExtensionBuilder {
|
||||||
private ReloadComponent reloadComponent;
|
private ReloadComponent reloadComponent;
|
||||||
private FlowController flowController;
|
private FlowController flowController;
|
||||||
private StateManagerProvider stateManagerProvider;
|
private StateManagerProvider stateManagerProvider;
|
||||||
|
private RuleViolationsManager ruleViolationsManager;
|
||||||
|
private FlowAnalyzer flowAnalyzer;
|
||||||
private String classloaderIsolationKey;
|
private String classloaderIsolationKey;
|
||||||
private SSLContext systemSslContext;
|
private SSLContext systemSslContext;
|
||||||
private PythonBridge pythonBridge;
|
private PythonBridge pythonBridge;
|
||||||
|
@ -183,6 +193,16 @@ public class ExtensionBuilder {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public ExtensionBuilder ruleViolationsManager(RuleViolationsManager ruleViolationsManager) {
|
||||||
|
this.ruleViolationsManager = ruleViolationsManager;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public ExtensionBuilder flowAnalyzer(FlowAnalyzer flowAnalyzer) {
|
||||||
|
this.flowAnalyzer = flowAnalyzer;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
public ExtensionBuilder extensionManager(final ExtensionManager extensionManager) {
|
public ExtensionBuilder extensionManager(final ExtensionManager extensionManager) {
|
||||||
this.extensionManager = extensionManager;
|
this.extensionManager = extensionManager;
|
||||||
return this;
|
return this;
|
||||||
|
@ -436,10 +456,57 @@ public class ExtensionBuilder {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public FlowAnalysisRuleNode buildFlowAnalysisRuleNode() {
|
||||||
|
if (identifier == null) {
|
||||||
|
throw new IllegalStateException("FlowAnalysisRule ID must be specified");
|
||||||
|
}
|
||||||
|
if (type == null) {
|
||||||
|
throw new IllegalStateException("FlowAnalysisRule Type must be specified");
|
||||||
|
}
|
||||||
|
if (bundleCoordinate == null) {
|
||||||
|
throw new IllegalStateException("Bundle Coordinate must be specified");
|
||||||
|
}
|
||||||
|
if (extensionManager == null) {
|
||||||
|
throw new IllegalStateException("Extension Manager must be specified");
|
||||||
|
}
|
||||||
|
if (serviceProvider == null) {
|
||||||
|
throw new IllegalStateException("Controller Service Provider must be specified");
|
||||||
|
}
|
||||||
|
if (nodeTypeProvider == null) {
|
||||||
|
throw new IllegalStateException("Node Type Provider must be specified");
|
||||||
|
}
|
||||||
|
if (variableRegistry == null) {
|
||||||
|
throw new IllegalStateException("Variable Registry must be specified");
|
||||||
|
}
|
||||||
|
if (reloadComponent == null) {
|
||||||
|
throw new IllegalStateException("Reload Component must be specified");
|
||||||
|
}
|
||||||
|
if (flowController == null) {
|
||||||
|
throw new IllegalStateException("FlowController must be specified");
|
||||||
|
}
|
||||||
|
|
||||||
|
boolean creationSuccessful = true;
|
||||||
|
LoggableComponent<FlowAnalysisRule> loggableComponent;
|
||||||
|
try {
|
||||||
|
loggableComponent = createLoggableFlowAnalysisRule();
|
||||||
|
} catch (final FlowAnalysisRuleInstantiationException rtie) {
|
||||||
|
logger.error("Could not create FlowAnalysisRule of type {} for ID {}; creating \"Ghost\" implementation", type, identifier, rtie);
|
||||||
|
final GhostFlowAnalysisRule ghostFlowAnalysisRule = new GhostFlowAnalysisRule();
|
||||||
|
ghostFlowAnalysisRule.setIdentifier(identifier);
|
||||||
|
ghostFlowAnalysisRule.setCanonicalClassName(type);
|
||||||
|
loggableComponent = new LoggableComponent<>(ghostFlowAnalysisRule, bundleCoordinate, null);
|
||||||
|
creationSuccessful = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
final FlowAnalysisRuleNode flowAnalysisRuleNode = createFlowAnalysisRuleNode(loggableComponent, creationSuccessful);
|
||||||
|
|
||||||
|
return flowAnalysisRuleNode;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
private ProcessorNode createProcessorNode(final LoggableComponent<Processor> processor, final String componentType, final boolean extensionMissing) {
|
private ProcessorNode createProcessorNode(final LoggableComponent<Processor> processor, final String componentType, final boolean extensionMissing) {
|
||||||
final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
|
final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
|
||||||
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(serviceProvider, componentVarRegistry);
|
final ValidationContextFactory validationContextFactory = createValidationContextFactory(serviceProvider, componentVarRegistry);
|
||||||
|
|
||||||
final ProcessorNode procNode = new StandardProcessorNode(processor, identifier, validationContextFactory, processScheduler, serviceProvider,
|
final ProcessorNode procNode = new StandardProcessorNode(processor, identifier, validationContextFactory, processScheduler, serviceProvider,
|
||||||
componentType, type, componentVarRegistry, reloadComponent, extensionManager, validationTrigger, extensionMissing);
|
componentType, type, componentVarRegistry, reloadComponent, extensionManager, validationTrigger, extensionMissing);
|
||||||
|
@ -450,10 +517,9 @@ public class ExtensionBuilder {
|
||||||
return procNode;
|
return procNode;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
private ReportingTaskNode createReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final boolean creationSuccessful) {
|
private ReportingTaskNode createReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final boolean creationSuccessful) {
|
||||||
final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
|
final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
|
||||||
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(serviceProvider, componentVarRegistry);
|
final ValidationContextFactory validationContextFactory = createValidationContextFactory(serviceProvider, componentVarRegistry);
|
||||||
final ReportingTaskNode taskNode;
|
final ReportingTaskNode taskNode;
|
||||||
if (creationSuccessful) {
|
if (creationSuccessful) {
|
||||||
taskNode = new StandardReportingTaskNode(reportingTask, identifier, flowController, processScheduler,
|
taskNode = new StandardReportingTaskNode(reportingTask, identifier, flowController, processScheduler,
|
||||||
|
@ -471,9 +537,13 @@ public class ExtensionBuilder {
|
||||||
return taskNode;
|
return taskNode;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private StandardValidationContextFactory createValidationContextFactory(ControllerServiceProvider serviceProvider, VariableRegistry variableRegistry) {
|
||||||
|
return new StandardValidationContextFactory(serviceProvider, variableRegistry, ruleViolationsManager, flowAnalyzer);
|
||||||
|
}
|
||||||
|
|
||||||
private ParameterProviderNode createParameterProviderNode(final LoggableComponent<ParameterProvider> parameterProvider, final boolean creationSuccessful) {
|
private ParameterProviderNode createParameterProviderNode(final LoggableComponent<ParameterProvider> parameterProvider, final boolean creationSuccessful) {
|
||||||
final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
|
final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
|
||||||
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(serviceProvider, componentVarRegistry);
|
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(serviceProvider, componentVarRegistry, ruleViolationsManager, flowAnalyzer);
|
||||||
final ParameterProviderNode parameterProviderNode;
|
final ParameterProviderNode parameterProviderNode;
|
||||||
if (creationSuccessful) {
|
if (creationSuccessful) {
|
||||||
parameterProviderNode = new StandardParameterProviderNode(parameterProvider, identifier, flowController,
|
parameterProviderNode = new StandardParameterProviderNode(parameterProvider, identifier, flowController,
|
||||||
|
@ -495,7 +565,7 @@ public class ExtensionBuilder {
|
||||||
|
|
||||||
private FlowRegistryClientNode createFlowRegistryClientNode(final LoggableComponent<FlowRegistryClient> client, final boolean creationSuccessful) {
|
private FlowRegistryClientNode createFlowRegistryClientNode(final LoggableComponent<FlowRegistryClient> client, final boolean creationSuccessful) {
|
||||||
final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
|
final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
|
||||||
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(serviceProvider, componentVarRegistry);
|
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(serviceProvider, componentVarRegistry, ruleViolationsManager, flowAnalyzer);
|
||||||
final FlowRegistryClientNode clientNode;
|
final FlowRegistryClientNode clientNode;
|
||||||
|
|
||||||
if (creationSuccessful) {
|
if (creationSuccessful) {
|
||||||
|
@ -609,7 +679,7 @@ public class ExtensionBuilder {
|
||||||
final LoggableComponent<ControllerService> proxiedLoggableComponent = new LoggableComponent<>(proxiedService, bundleCoordinate, terminationAwareLogger);
|
final LoggableComponent<ControllerService> proxiedLoggableComponent = new LoggableComponent<>(proxiedService, bundleCoordinate, terminationAwareLogger);
|
||||||
|
|
||||||
final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
|
final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
|
||||||
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(serviceProvider, componentVarRegistry);
|
final ValidationContextFactory validationContextFactory = createValidationContextFactory(serviceProvider, componentVarRegistry);
|
||||||
final ControllerServiceNode serviceNode = new StandardControllerServiceNode(originalLoggableComponent, proxiedLoggableComponent, invocationHandler,
|
final ControllerServiceNode serviceNode = new StandardControllerServiceNode(originalLoggableComponent, proxiedLoggableComponent, invocationHandler,
|
||||||
identifier, validationContextFactory, serviceProvider, componentVarRegistry, reloadComponent, extensionManager, validationTrigger);
|
identifier, validationContextFactory, serviceProvider, componentVarRegistry, reloadComponent, extensionManager, validationTrigger);
|
||||||
serviceNode.setName(rawClass.getSimpleName());
|
serviceNode.setName(rawClass.getSimpleName());
|
||||||
|
@ -696,7 +766,7 @@ public class ExtensionBuilder {
|
||||||
final ControllerServiceInvocationHandler invocationHandler = new StandardControllerServiceInvocationHandler(extensionManager, ghostService);
|
final ControllerServiceInvocationHandler invocationHandler = new StandardControllerServiceInvocationHandler(extensionManager, ghostService);
|
||||||
|
|
||||||
final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
|
final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
|
||||||
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(serviceProvider, variableRegistry);
|
final ValidationContextFactory validationContextFactory = createValidationContextFactory(serviceProvider, variableRegistry);
|
||||||
final ControllerServiceNode serviceNode = new StandardControllerServiceNode(proxiedLoggableComponent, proxiedLoggableComponent, invocationHandler, identifier,
|
final ControllerServiceNode serviceNode = new StandardControllerServiceNode(proxiedLoggableComponent, proxiedLoggableComponent, invocationHandler, identifier,
|
||||||
validationContextFactory, serviceProvider, componentType, type, componentVarRegistry, reloadComponent, extensionManager, validationTrigger, true);
|
validationContextFactory, serviceProvider, componentType, type, componentVarRegistry, reloadComponent, extensionManager, validationTrigger, true);
|
||||||
|
|
||||||
|
@ -747,6 +817,42 @@ public class ExtensionBuilder {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private LoggableComponent<FlowAnalysisRule> createLoggableFlowAnalysisRule() throws FlowAnalysisRuleInstantiationException {
|
||||||
|
try {
|
||||||
|
final LoggableComponent<FlowAnalysisRule> loggableComponent = createLoggableComponent(FlowAnalysisRule.class, new StandardLoggingContext(null));
|
||||||
|
|
||||||
|
final String taskName = loggableComponent.getComponent().getClass().getSimpleName();
|
||||||
|
final FlowAnalysisRuleInitializationContext config = new StandardFlowAnalysisInitializationContext(identifier,
|
||||||
|
loggableComponent.getLogger(), serviceProvider, kerberosConfig, nodeTypeProvider);
|
||||||
|
|
||||||
|
loggableComponent.getComponent().initialize(config);
|
||||||
|
|
||||||
|
return loggableComponent;
|
||||||
|
} catch (final Exception e) {
|
||||||
|
throw new FlowAnalysisRuleInstantiationException(type, e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private FlowAnalysisRuleNode createFlowAnalysisRuleNode(final LoggableComponent<FlowAnalysisRule> flowAnalysisRule, final boolean creationSuccessful) {
|
||||||
|
final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
|
||||||
|
final ValidationContextFactory validationContextFactory = createValidationContextFactory(serviceProvider, componentVarRegistry);
|
||||||
|
final FlowAnalysisRuleNode ruleNode;
|
||||||
|
if (creationSuccessful) {
|
||||||
|
ruleNode = new StandardFlowAnalysisRuleNode(flowAnalysisRule, identifier, flowController,
|
||||||
|
validationContextFactory, ruleViolationsManager, componentVarRegistry, reloadComponent, extensionManager, validationTrigger);
|
||||||
|
ruleNode.setName(ruleNode.getFlowAnalysisRule().getClass().getSimpleName());
|
||||||
|
} else {
|
||||||
|
final String simpleClassName = type.contains(".") ? StringUtils.substringAfterLast(type, ".") : type;
|
||||||
|
final String componentType = "(Missing) " + simpleClassName;
|
||||||
|
|
||||||
|
ruleNode = new StandardFlowAnalysisRuleNode(flowAnalysisRule, identifier, flowController, validationContextFactory, ruleViolationsManager,
|
||||||
|
componentType, type, componentVarRegistry, reloadComponent, extensionManager, validationTrigger, true);
|
||||||
|
ruleNode.setName(componentType);
|
||||||
|
}
|
||||||
|
|
||||||
|
return ruleNode;
|
||||||
|
}
|
||||||
|
|
||||||
private LoggableComponent<FlowRegistryClient> createLoggableFlowRegistryClient() throws FlowRepositoryClientInstantiationException {
|
private LoggableComponent<FlowRegistryClient> createLoggableFlowRegistryClient() throws FlowRepositoryClientInstantiationException {
|
||||||
try {
|
try {
|
||||||
final LoggableComponent<FlowRegistryClient> clientComponent = createLoggableComponent(FlowRegistryClient.class, new StandardLoggingContext(null));
|
final LoggableComponent<FlowRegistryClient> clientComponent = createLoggableComponent(FlowRegistryClient.class, new StandardLoggingContext(null));
|
||||||
|
|
|
@ -18,6 +18,8 @@ package org.apache.nifi.controller;
|
||||||
|
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.apache.nifi.admin.service.AuditService;
|
import org.apache.nifi.admin.service.AuditService;
|
||||||
|
import org.apache.nifi.flowanalysis.StandardFlowAnalyzer;
|
||||||
|
import org.apache.nifi.flowanalysis.TriggerFlowAnalysisTask;
|
||||||
import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored;
|
import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored;
|
||||||
import org.apache.nifi.annotation.notification.PrimaryNodeState;
|
import org.apache.nifi.annotation.notification.PrimaryNodeState;
|
||||||
import org.apache.nifi.authorization.Authorizer;
|
import org.apache.nifi.authorization.Authorizer;
|
||||||
|
@ -57,6 +59,9 @@ import org.apache.nifi.controller.cluster.Heartbeater;
|
||||||
import org.apache.nifi.controller.exception.CommunicationsException;
|
import org.apache.nifi.controller.exception.CommunicationsException;
|
||||||
import org.apache.nifi.controller.flow.FlowManager;
|
import org.apache.nifi.controller.flow.FlowManager;
|
||||||
import org.apache.nifi.controller.flow.StandardFlowManager;
|
import org.apache.nifi.controller.flow.StandardFlowManager;
|
||||||
|
import org.apache.nifi.controller.flowanalysis.FlowAnalysisRuleInstantiationException;
|
||||||
|
import org.apache.nifi.controller.flowanalysis.FlowAnalysisRuleProvider;
|
||||||
|
import org.apache.nifi.controller.flowanalysis.FlowAnalysisUtil;
|
||||||
import org.apache.nifi.controller.kerberos.KerberosConfig;
|
import org.apache.nifi.controller.kerberos.KerberosConfig;
|
||||||
import org.apache.nifi.controller.leader.election.LeaderElectionManager;
|
import org.apache.nifi.controller.leader.election.LeaderElectionManager;
|
||||||
import org.apache.nifi.controller.leader.election.LeaderElectionStateChangeListener;
|
import org.apache.nifi.controller.leader.election.LeaderElectionStateChangeListener;
|
||||||
|
@ -144,6 +149,7 @@ import org.apache.nifi.events.EventReporter;
|
||||||
import org.apache.nifi.flow.Bundle;
|
import org.apache.nifi.flow.Bundle;
|
||||||
import org.apache.nifi.flow.VersionedConnection;
|
import org.apache.nifi.flow.VersionedConnection;
|
||||||
import org.apache.nifi.flow.VersionedProcessGroup;
|
import org.apache.nifi.flow.VersionedProcessGroup;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRule;
|
||||||
import org.apache.nifi.flowfile.FlowFilePrioritizer;
|
import org.apache.nifi.flowfile.FlowFilePrioritizer;
|
||||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||||
import org.apache.nifi.groups.BundleUpdateStrategy;
|
import org.apache.nifi.groups.BundleUpdateStrategy;
|
||||||
|
@ -179,6 +185,7 @@ import org.apache.nifi.python.PythonProcessConfig;
|
||||||
import org.apache.nifi.registry.VariableRegistry;
|
import org.apache.nifi.registry.VariableRegistry;
|
||||||
import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper;
|
import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper;
|
||||||
import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup;
|
import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup;
|
||||||
|
import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedProcessGroup;
|
||||||
import org.apache.nifi.remote.HttpRemoteSiteListener;
|
import org.apache.nifi.remote.HttpRemoteSiteListener;
|
||||||
import org.apache.nifi.remote.RemoteGroupPort;
|
import org.apache.nifi.remote.RemoteGroupPort;
|
||||||
import org.apache.nifi.remote.RemoteResourceManager;
|
import org.apache.nifi.remote.RemoteResourceManager;
|
||||||
|
@ -206,6 +213,7 @@ import org.apache.nifi.util.FormatUtils;
|
||||||
import org.apache.nifi.util.NiFiProperties;
|
import org.apache.nifi.util.NiFiProperties;
|
||||||
import org.apache.nifi.util.ReflectionUtils;
|
import org.apache.nifi.util.ReflectionUtils;
|
||||||
import org.apache.nifi.util.concurrency.TimedLock;
|
import org.apache.nifi.util.concurrency.TimedLock;
|
||||||
|
import org.apache.nifi.validation.RuleViolationsManager;
|
||||||
import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
|
import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
|
||||||
import org.apache.nifi.web.revision.RevisionManager;
|
import org.apache.nifi.web.revision.RevisionManager;
|
||||||
import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
|
import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
|
||||||
|
@ -244,11 +252,12 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
|
import java.util.function.Supplier;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import static java.util.Objects.requireNonNull;
|
import static java.util.Objects.requireNonNull;
|
||||||
|
|
||||||
public class FlowController implements ReportingTaskProvider, Authorizable, NodeTypeProvider {
|
public class FlowController implements ReportingTaskProvider, FlowAnalysisRuleProvider, Authorizable, NodeTypeProvider {
|
||||||
private static final String STANDARD_PYTHON_BRIDGE_IMPLEMENTATION_CLASS = "org.apache.nifi.py4j.StandardPythonBridge";
|
private static final String STANDARD_PYTHON_BRIDGE_IMPLEMENTATION_CLASS = "org.apache.nifi.py4j.StandardPythonBridge";
|
||||||
|
|
||||||
// default repository implementations
|
// default repository implementations
|
||||||
|
@ -322,11 +331,13 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
||||||
private final LeaderElectionManager leaderElectionManager;
|
private final LeaderElectionManager leaderElectionManager;
|
||||||
private final ClusterCoordinator clusterCoordinator;
|
private final ClusterCoordinator clusterCoordinator;
|
||||||
private final FlowEngine validationThreadPool;
|
private final FlowEngine validationThreadPool;
|
||||||
|
private final FlowEngine flowAnalysisThreadPool;
|
||||||
private final ValidationTrigger validationTrigger;
|
private final ValidationTrigger validationTrigger;
|
||||||
private final ReloadComponent reloadComponent;
|
private final ReloadComponent reloadComponent;
|
||||||
private final ProvenanceAuthorizableFactory provenanceAuthorizableFactory;
|
private final ProvenanceAuthorizableFactory provenanceAuthorizableFactory;
|
||||||
private final UserAwareEventAccess eventAccess;
|
private final UserAwareEventAccess eventAccess;
|
||||||
private final ParameterContextManager parameterContextManager;
|
private final ParameterContextManager parameterContextManager;
|
||||||
|
private final StandardFlowAnalyzer flowAnalyzer;
|
||||||
private final StandardFlowManager flowManager;
|
private final StandardFlowManager flowManager;
|
||||||
private final RepositoryContextFactory repositoryContextFactory;
|
private final RepositoryContextFactory repositoryContextFactory;
|
||||||
private final RingBufferGarbageCollectionLog gcLog;
|
private final RingBufferGarbageCollectionLog gcLog;
|
||||||
|
@ -408,7 +419,8 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
||||||
final BulletinRepository bulletinRepo,
|
final BulletinRepository bulletinRepo,
|
||||||
final VariableRegistry variableRegistry,
|
final VariableRegistry variableRegistry,
|
||||||
final ExtensionDiscoveringManager extensionManager,
|
final ExtensionDiscoveringManager extensionManager,
|
||||||
final StatusHistoryRepository statusHistoryRepository) {
|
final StatusHistoryRepository statusHistoryRepository,
|
||||||
|
final RuleViolationsManager ruleViolationsManager) {
|
||||||
|
|
||||||
return new FlowController(
|
return new FlowController(
|
||||||
flowFileEventRepo,
|
flowFileEventRepo,
|
||||||
|
@ -425,7 +437,8 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
||||||
/* variable registry */ variableRegistry,
|
/* variable registry */ variableRegistry,
|
||||||
extensionManager,
|
extensionManager,
|
||||||
null,
|
null,
|
||||||
statusHistoryRepository);
|
statusHistoryRepository,
|
||||||
|
ruleViolationsManager);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static FlowController createClusteredInstance(
|
public static FlowController createClusteredInstance(
|
||||||
|
@ -442,7 +455,8 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
||||||
final VariableRegistry variableRegistry,
|
final VariableRegistry variableRegistry,
|
||||||
final ExtensionDiscoveringManager extensionManager,
|
final ExtensionDiscoveringManager extensionManager,
|
||||||
final RevisionManager revisionManager,
|
final RevisionManager revisionManager,
|
||||||
final StatusHistoryRepository statusHistoryRepository) {
|
final StatusHistoryRepository statusHistoryRepository,
|
||||||
|
final RuleViolationsManager ruleViolationsManager) {
|
||||||
|
|
||||||
final FlowController flowController = new FlowController(
|
final FlowController flowController = new FlowController(
|
||||||
flowFileEventRepo,
|
flowFileEventRepo,
|
||||||
|
@ -459,7 +473,8 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
||||||
variableRegistry,
|
variableRegistry,
|
||||||
extensionManager,
|
extensionManager,
|
||||||
revisionManager,
|
revisionManager,
|
||||||
statusHistoryRepository);
|
statusHistoryRepository,
|
||||||
|
ruleViolationsManager);
|
||||||
|
|
||||||
return flowController;
|
return flowController;
|
||||||
}
|
}
|
||||||
|
@ -480,7 +495,8 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
||||||
final VariableRegistry variableRegistry,
|
final VariableRegistry variableRegistry,
|
||||||
final ExtensionDiscoveringManager extensionManager,
|
final ExtensionDiscoveringManager extensionManager,
|
||||||
final RevisionManager revisionManager,
|
final RevisionManager revisionManager,
|
||||||
final StatusHistoryRepository statusHistoryRepository) {
|
final StatusHistoryRepository statusHistoryRepository,
|
||||||
|
final RuleViolationsManager ruleViolationsManager) {
|
||||||
|
|
||||||
maxTimerDrivenThreads = new AtomicInteger(10);
|
maxTimerDrivenThreads = new AtomicInteger(10);
|
||||||
|
|
||||||
|
@ -552,7 +568,21 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
||||||
|
|
||||||
parameterContextManager = new StandardParameterContextManager();
|
parameterContextManager = new StandardParameterContextManager();
|
||||||
repositoryContextFactory = new RepositoryContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceRepository, stateManagerProvider);
|
repositoryContextFactory = new RepositoryContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceRepository, stateManagerProvider);
|
||||||
flowManager = new StandardFlowManager(nifiProperties, sslContext, this, flowFileEventRepository, parameterContextManager);
|
|
||||||
|
this.flowAnalysisThreadPool = new FlowEngine(1, "Background Flow Analysis", true);
|
||||||
|
flowAnalyzer = new StandardFlowAnalyzer(
|
||||||
|
ruleViolationsManager,
|
||||||
|
this,
|
||||||
|
extensionManager
|
||||||
|
);
|
||||||
|
|
||||||
|
flowManager = new StandardFlowManager(
|
||||||
|
nifiProperties,
|
||||||
|
sslContext,
|
||||||
|
this,
|
||||||
|
flowFileEventRepository,
|
||||||
|
parameterContextManager
|
||||||
|
);
|
||||||
|
|
||||||
controllerServiceProvider = new StandardControllerServiceProvider(processScheduler, bulletinRepository, flowManager, extensionManager);
|
controllerServiceProvider = new StandardControllerServiceProvider(processScheduler, bulletinRepository, flowManager, extensionManager);
|
||||||
controllerServiceResolver = new StandardControllerServiceResolver(authorizer, flowManager, new NiFiRegistryFlowMapper(extensionManager),
|
controllerServiceResolver = new StandardControllerServiceResolver(authorizer, flowManager, new NiFiRegistryFlowMapper(extensionManager),
|
||||||
|
@ -572,7 +602,13 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
||||||
pythonBundle = PythonBundle.create(nifiProperties, pythonBridgeClassLoader);
|
pythonBundle = PythonBundle.create(nifiProperties, pythonBridgeClassLoader);
|
||||||
extensionManager.discoverPythonExtensions(pythonBundle);
|
extensionManager.discoverPythonExtensions(pythonBundle);
|
||||||
|
|
||||||
flowManager.initialize(controllerServiceProvider, pythonBridge);
|
flowManager.initialize(
|
||||||
|
controllerServiceProvider,
|
||||||
|
pythonBridge,
|
||||||
|
flowAnalyzer,
|
||||||
|
ruleViolationsManager
|
||||||
|
);
|
||||||
|
flowAnalyzer.initialize(controllerServiceProvider);
|
||||||
|
|
||||||
final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), repositoryContextFactory);
|
final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), repositoryContextFactory);
|
||||||
final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), repositoryContextFactory, this.nifiProperties);
|
final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), repositoryContextFactory, this.nifiProperties);
|
||||||
|
@ -1105,6 +1141,14 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
for (final FlowAnalysisRuleNode ruleNode : getAllFlowAnalysisRules()) {
|
||||||
|
final FlowAnalysisRule rule = ruleNode.getFlowAnalysisRule();
|
||||||
|
|
||||||
|
try (final NarCloseable nc = NarCloseable.withComponentNarLoader(extensionManager, rule.getClass(), rule.getIdentifier())) {
|
||||||
|
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, rule, ruleNode.getConfigurationContext());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
for (final ParameterProviderNode parameterProviderNode : flowManager.getAllParameterProviders()) {
|
for (final ParameterProviderNode parameterProviderNode : flowManager.getAllParameterProviders()) {
|
||||||
final ParameterProvider provider = parameterProviderNode.getParameterProvider();
|
final ParameterProvider provider = parameterProviderNode.getParameterProvider();
|
||||||
|
|
||||||
|
@ -1129,6 +1173,19 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
||||||
LOG.debug("Triggering initial validation of all components");
|
LOG.debug("Triggering initial validation of all components");
|
||||||
final long start = System.nanoTime();
|
final long start = System.nanoTime();
|
||||||
|
|
||||||
|
Supplier<VersionedProcessGroup> rootProcessGroupSupplier = () -> {
|
||||||
|
ProcessGroup rootProcessGroup = getFlowManager().getRootGroup();
|
||||||
|
|
||||||
|
NiFiRegistryFlowMapper mapper = FlowAnalysisUtil.createMapper(getExtensionManager());
|
||||||
|
|
||||||
|
InstantiatedVersionedProcessGroup versionedRootProcessGroup = mapper.mapNonVersionedProcessGroup(
|
||||||
|
rootProcessGroup,
|
||||||
|
controllerServiceProvider
|
||||||
|
);
|
||||||
|
|
||||||
|
return versionedRootProcessGroup;
|
||||||
|
};
|
||||||
|
|
||||||
final ValidationTrigger triggerIfValidating = new ValidationTrigger() {
|
final ValidationTrigger triggerIfValidating = new ValidationTrigger() {
|
||||||
@Override
|
@Override
|
||||||
public void triggerAsync(final ComponentNode component) {
|
public void triggerAsync(final ComponentNode component) {
|
||||||
|
@ -1155,11 +1212,13 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
new TriggerFlowAnalysisTask(flowAnalyzer, rootProcessGroupSupplier).run();
|
||||||
new TriggerValidationTask(flowManager, triggerIfValidating).run();
|
new TriggerValidationTask(flowManager, triggerIfValidating).run();
|
||||||
|
|
||||||
final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
|
final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
|
||||||
LOG.info("Performed initial validation of all components in {} milliseconds", millis);
|
LOG.info("Performed initial validation of all components in {} milliseconds", millis);
|
||||||
|
|
||||||
|
scheduleBackgroundFlowAnalysis(rootProcessGroupSupplier);
|
||||||
// Trigger component validation to occur every 5 seconds.
|
// Trigger component validation to occur every 5 seconds.
|
||||||
validationThreadPool.scheduleWithFixedDelay(new TriggerValidationTask(flowManager, validationTrigger), 5, 5, TimeUnit.SECONDS);
|
validationThreadPool.scheduleWithFixedDelay(new TriggerValidationTask(flowManager, validationTrigger), 5, 5, TimeUnit.SECONDS);
|
||||||
|
|
||||||
|
@ -1236,6 +1295,21 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void scheduleBackgroundFlowAnalysis(Supplier<VersionedProcessGroup> rootProcessGroupSupplier) {
|
||||||
|
try {
|
||||||
|
final long scheduleMillis = parseDurationPropertyToMillis(NiFiProperties.BACKGROUND_FLOW_ANALYSIS_SCHEDULE);
|
||||||
|
|
||||||
|
flowAnalysisThreadPool.scheduleWithFixedDelay(
|
||||||
|
new TriggerFlowAnalysisTask(flowManager.getFlowAnalyzer(), rootProcessGroupSupplier),
|
||||||
|
scheduleMillis,
|
||||||
|
scheduleMillis,
|
||||||
|
TimeUnit.MILLISECONDS
|
||||||
|
);
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.warn("Could not initialize TriggerFlowAnalysisTask.", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private void scheduleLongRunningTaskMonitor() {
|
private void scheduleLongRunningTaskMonitor() {
|
||||||
longRunningTaskMonitorThreadPool.ifPresent(flowEngine -> {
|
longRunningTaskMonitorThreadPool.ifPresent(flowEngine -> {
|
||||||
try {
|
try {
|
||||||
|
@ -1424,6 +1498,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
||||||
}
|
}
|
||||||
|
|
||||||
validationThreadPool.shutdown();
|
validationThreadPool.shutdown();
|
||||||
|
flowAnalysisThreadPool.shutdown();
|
||||||
clusterTaskExecutor.shutdownNow();
|
clusterTaskExecutor.shutdownNow();
|
||||||
|
|
||||||
if (zooKeeperStateServer != null) {
|
if (zooKeeperStateServer != null) {
|
||||||
|
@ -1627,6 +1702,11 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
||||||
return delegate.getState(taskNode);
|
return delegate.getState(taskNode);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public org.apache.nifi.flow.ScheduledState getState(final FlowAnalysisRuleNode ruleNode) {
|
||||||
|
return delegate.getState(ruleNode);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public org.apache.nifi.flow.ScheduledState getState(final ControllerServiceNode serviceNode) {
|
public org.apache.nifi.flow.ScheduledState getState(final ControllerServiceNode serviceNode) {
|
||||||
return delegate.getState(serviceNode);
|
return delegate.getState(serviceNode);
|
||||||
|
@ -3342,4 +3422,38 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
||||||
return primary;
|
return primary;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Flow Analysis
|
||||||
|
@Override
|
||||||
|
public FlowAnalysisRuleNode createFlowAnalysisRule(String type, String id, BundleCoordinate bundleCoordinate, boolean firstTimeAdded) throws FlowAnalysisRuleInstantiationException {
|
||||||
|
return flowManager.createFlowAnalysisRule(type, id, bundleCoordinate, firstTimeAdded);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FlowAnalysisRuleNode getFlowAnalysisRuleNode(String identifier) {
|
||||||
|
return flowManager.getFlowAnalysisRuleNode(identifier);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<FlowAnalysisRuleNode> getAllFlowAnalysisRules() {
|
||||||
|
return flowManager.getAllFlowAnalysisRules();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void removeFlowAnalysisRule(FlowAnalysisRuleNode flowAnalysisRule) {
|
||||||
|
flowManager.removeFlowAnalysisRule(flowAnalysisRule);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void enableFlowAnalysisRule(FlowAnalysisRuleNode flowAnalysisRule) {
|
||||||
|
flowAnalysisRule.verifyCanEnable();
|
||||||
|
flowAnalysisRule.reloadAdditionalResourcesIfNecessary();
|
||||||
|
flowAnalysisRule.enable();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void disableFlowAnalysisRule(FlowAnalysisRuleNode flowAnalysisRule) {
|
||||||
|
flowAnalysisRule.verifyCanDisable();
|
||||||
|
flowAnalysisRule.disable();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,10 +21,12 @@ import org.apache.nifi.bundle.BundleCoordinate;
|
||||||
import org.apache.nifi.components.state.StateManager;
|
import org.apache.nifi.components.state.StateManager;
|
||||||
import org.apache.nifi.controller.exception.ControllerServiceInstantiationException;
|
import org.apache.nifi.controller.exception.ControllerServiceInstantiationException;
|
||||||
import org.apache.nifi.controller.parameter.ParameterProviderInstantiationException;
|
import org.apache.nifi.controller.parameter.ParameterProviderInstantiationException;
|
||||||
|
import org.apache.nifi.controller.flowanalysis.FlowAnalysisRuleInstantiationException;
|
||||||
import org.apache.nifi.controller.flowrepository.FlowRepositoryClientInstantiationException;
|
import org.apache.nifi.controller.flowrepository.FlowRepositoryClientInstantiationException;
|
||||||
import org.apache.nifi.controller.service.ControllerServiceInvocationHandler;
|
import org.apache.nifi.controller.service.ControllerServiceInvocationHandler;
|
||||||
import org.apache.nifi.controller.service.ControllerServiceNode;
|
import org.apache.nifi.controller.service.ControllerServiceNode;
|
||||||
import org.apache.nifi.controller.service.StandardConfigurationContext;
|
import org.apache.nifi.controller.service.StandardConfigurationContext;
|
||||||
|
import org.apache.nifi.flowanalysis.FlowAnalysisRule;
|
||||||
import org.apache.nifi.logging.ComponentLog;
|
import org.apache.nifi.logging.ComponentLog;
|
||||||
import org.apache.nifi.logging.LogRepositoryFactory;
|
import org.apache.nifi.logging.LogRepositoryFactory;
|
||||||
import org.apache.nifi.nar.ExtensionManager;
|
import org.apache.nifi.nar.ExtensionManager;
|
||||||
|
@ -219,6 +221,55 @@ public class StandardReloadComponent implements ReloadComponent {
|
||||||
flowController.getValidationTrigger().triggerAsync(existingNode);
|
flowController.getValidationTrigger().triggerAsync(existingNode);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void reload(final FlowAnalysisRuleNode existingNode, final String newType, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls)
|
||||||
|
throws FlowAnalysisRuleInstantiationException {
|
||||||
|
if (existingNode == null) {
|
||||||
|
throw new IllegalStateException("Existing FlowAnalysisRuleNode cannot be null");
|
||||||
|
}
|
||||||
|
|
||||||
|
final String id = existingNode.getFlowAnalysisRule().getIdentifier();
|
||||||
|
|
||||||
|
// ghost components will have a null logger
|
||||||
|
if (existingNode.getLogger() != null) {
|
||||||
|
existingNode.getLogger().debug("Reloading component {} to type {} from bundle {}", new Object[]{id, newType, bundleCoordinate});
|
||||||
|
}
|
||||||
|
|
||||||
|
final ExtensionManager extensionManager = flowController.getExtensionManager();
|
||||||
|
|
||||||
|
// createFlowAnalysisRule will create a new instance class loader for the same id so
|
||||||
|
// save the instance class loader to use it for calling OnRemoved on the existing processor
|
||||||
|
final ClassLoader existingInstanceClassLoader = extensionManager.getInstanceClassLoader(id);
|
||||||
|
|
||||||
|
// call OnRemoved for the existing flow analysis rule using the previous instance class loader
|
||||||
|
final ConfigurationContext configurationContext = existingNode.getConfigurationContext();
|
||||||
|
try (final NarCloseable x = NarCloseable.withComponentNarLoader(existingInstanceClassLoader)) {
|
||||||
|
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, existingNode.getFlowAnalysisRule(), configurationContext);
|
||||||
|
} finally {
|
||||||
|
extensionManager.closeURLClassLoader(id, existingInstanceClassLoader);
|
||||||
|
}
|
||||||
|
|
||||||
|
// set firstTimeAdded to true so lifecycle annotations get fired, but don't register this node
|
||||||
|
// attempt the creation to make sure it works before firing the OnRemoved methods below
|
||||||
|
final String classloaderIsolationKey = existingNode.getClassLoaderIsolationKey(configurationContext);
|
||||||
|
final FlowAnalysisRuleNode newNode = flowController.getFlowManager().createFlowAnalysisRule(newType, id, bundleCoordinate, additionalUrls, true, false, classloaderIsolationKey);
|
||||||
|
|
||||||
|
// set the new flow analysis rule into the existing node
|
||||||
|
final ComponentLog componentLogger = new SimpleProcessLogger(id, existingNode.getFlowAnalysisRule(), new StandardLoggingContext(null));
|
||||||
|
final TerminationAwareLogger terminationAwareLogger = new TerminationAwareLogger(componentLogger);
|
||||||
|
LogRepositoryFactory.getRepository(id).setLogger(terminationAwareLogger);
|
||||||
|
|
||||||
|
final LoggableComponent<FlowAnalysisRule> newFlowAnalysisRule = new LoggableComponent<>(newNode.getFlowAnalysisRule(), newNode.getBundleCoordinate(), terminationAwareLogger);
|
||||||
|
existingNode.setFlowAnalysisRule(newFlowAnalysisRule);
|
||||||
|
existingNode.setExtensionMissing(newNode.isExtensionMissing());
|
||||||
|
|
||||||
|
// need to refresh the properties in case we are changing from ghost component to real component
|
||||||
|
existingNode.refreshProperties();
|
||||||
|
|
||||||
|
logger.debug("Triggering async validation of {} due to flow analysis rule reload", existingNode);
|
||||||
|
flowController.getValidationTrigger().triggerAsync(existingNode);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void reload(final ParameterProviderNode existingNode, final String newType, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls)
|
public void reload(final ParameterProviderNode existingNode, final String newType, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls)
|
||||||
throws ParameterProviderInstantiationException {
|
throws ParameterProviderInstantiationException {
|
||||||
|
|