diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnAdded.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnAdded.java index acb7a4d027..a1286ea473 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnAdded.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnAdded.java @@ -24,16 +24,25 @@ import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; /** + *

* Marker annotation a {@link org.apache.nifi.processor.Processor Processor}, * {@link org.apache.nifi.controller.ControllerService ControllerService}, or * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} * implementation can use to indicate a method * should be called whenever the component is added to the flow. This method * will be called once for the entire life of a component instance. - * + *

+ * + *

+ * Methods with this annotation are called without any arguments, as all settings + * and properties can be assumed to be the defaults. + *

+ * + *

* If any method annotated with this annotation throws a Throwable, the component * will not be added to the flow. - * + *

+ * * @author none */ @Documented diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnDisabled.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnDisabled.java index 0f78010026..b227968931 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnDisabled.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnDisabled.java @@ -23,19 +23,32 @@ import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; +import org.apache.nifi.controller.ConfigurationContext; + /** - * Marker annotation a {@link org.apache.nifi.processor.Processor Processor}, - * {@link org.apache.nifi.controller.ControllerService ControllerService} or - * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} - * can use to indicate a method should be called whenever the component is disabled. + *

+ * Marker annotation a {@link org.apache.nifi.controller.ControllerService ControllerService} + * can use to indicate a method should be called whenever the service is disabled. + *

* *

- * Methods using this annotation must take no arguments. If a method with this annotation - * throws a Throwable, a log message and bulletin will be issued for the component, but - * the component will still be disabled. + * Methods using this annotation are permitted to take zero arguments or to take a single + * argument of type {@link ConfigurationContext}. If a method with this annotation + * throws a Throwable, a log message and bulletin will be issued for the service, and the + * service will remain in a 'DISABLING' state. When this occurs, the method with this annotation + * will be called again after some period of time. This will continue until the method returns + * without throwing any Throwable. Until that time, the service will remain in a 'DISABLING' state + * and cannot be enabled again. + *

+ * + *

+ * Note that this annotation will be ignored if applied to a ReportingTask or Processor. For a Controller + * Service, enabling and disabling are considered lifecycle events, as the action makes them usable or + * unusable by other components. However, for a Processor and a Reporting + * Task, these are not lifecycle events but rather a mechanism to allow a component to be excluded when + * starting or stopping a group of components. *

* - * @author none */ @Documented @Target({ElementType.METHOD}) diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnEnabled.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnEnabled.java index 1536decb80..32aeec6e7c 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnEnabled.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnEnabled.java @@ -25,35 +25,35 @@ import java.lang.annotation.Target; /** *

- * Marker annotation a {@link org.apache.nifi.processor.Processor Processor}, - * {@link org.apache.nifi.controller.ControllerService ControllerService} or - * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} - * can use to indicate a method should be called whenever the component is enabled. - * Any method that has this annotation will be called every time a user enables the component. + * Marker annotation a {@link org.apache.nifi.controller.ControllerService ControllerService} + * can use to indicate a method should be called whenever the service is enabled. + * Any method that has this annotation will be called every time a user enables the service. * Additionally, each time that NiFi is restarted, if NiFi is configured to "auto-resume state" - * and the component is enabled (whether stopped or running), the method will be invoked. + * and the service is enabled, the method will be invoked. *

* *

- * Methods using this annotation must take either 0 arguments or a single argument. - *

- * - *

- * If using 1 argument and the component using the annotation is a Processor, that argument must - * be of type {@link org.apache.nifi.processor.ProcessContext ProcessContext}. - *

- * - *

- * If using 1 argument and the component using the annotation is a Reporting Task or Controller Service, - * that argument must be of type {@link org.apache.nifi.controller.ConfigurationContext ConfigurationContext}. + * Methods using this annotation must take either 0 arguments or a single argument of type + * {@link org.apache.nifi.controller.ConfigurationContext ConfigurationContext}. *

* *

* If a method with this annotation throws a Throwable, a log message and bulletin will be issued - * for the component, but the component will still be enabled. + * for the component. In this event, the service will remain in an 'ENABLING' state and will not be + * usable. All methods with this annotation will then be called again after a delay. The service will + * not be made available for use until all methods with this annotation have returned without throwing + * anything. + *

+ * + *

+ * Note that this annotation will be ignored if applied to a ReportingTask or Processor. For a Controller + * Service, enabling and disabling are considered lifecycle events, as the action makes them usable or + * unusable by other components. However, for a Processor and a Reporting + * Task, these are not lifecycle events but rather a mechanism to allow a component to be excluded when + * starting or stopping a group of components. *

* - * @author none + * */ @Documented @Target({ElementType.METHOD}) diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnRemoved.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnRemoved.java index 696159f0e4..71202b4317 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnRemoved.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnRemoved.java @@ -23,7 +23,11 @@ import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.processor.ProcessContext; + /** + *

* Marker annotation a {@link org.apache.nifi.processor.Processor Processor}, * {@link org.apache.nifi.controller.ControllerService ControllerService}, or * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} implementation @@ -32,7 +36,15 @@ import java.lang.annotation.Target; * component instance. If the method throw any Throwable, that Throwable will be * caught and logged but will not prevent subsequent methods with this annotation * or removal of the component from the flow. - * + *

+ * + *

+ * Methods with this annotation are permitted to take no arguments or to take a single + * argument. If using a single argument, that argument must be of type {@link ConfigurationContext} + * if the component is a ReportingTask or a ControllerService. If the component is a Processor, + * then the argument must be of type {@link ProcessContext}. + *

+ * * @author none */ @Documented diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnShutdown.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnShutdown.java index a4129e158d..3d1ce6c642 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnShutdown.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnShutdown.java @@ -23,7 +23,11 @@ import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.processor.ProcessContext; + /** + *

* Marker annotation a {@link org.apache.nifi.processor.Processor Processor}, * {@link org.apache.nifi.controller.ControllerService ControllerService}, or * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} implementation @@ -31,7 +35,14 @@ import java.lang.annotation.Target; * This will be called at most once for each component in a JVM lifetime. * It is not, however, guaranteed that this method will be called on shutdown, as * the service may be killed suddenly. - * + *

+ * + *

+ * Methods with this annotation are permitted to take either 0 or 1 argument. If an argument + * is used, it must be of type {@link ConfigurationContext} if the component is a ReportingTask + * or Controller Service, or of type {@link ProcessContext} if the component is a Processor. + *

+ * * @author none */ @Documented diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnStopped.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnStopped.java index 4715253234..fdc4fd8e09 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnStopped.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnStopped.java @@ -23,6 +23,9 @@ import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.processor.ProcessContext; + /** *

* Marker annotation a {@link org.apache.nifi.processor.Processor Processor} or @@ -47,6 +50,12 @@ import java.lang.annotation.Target; * longer scheduled to run (as opposed to after all threads have returned from the * onTrigger method), see the {@link OnUnscheduled} annotation. *

+ * + *

+ * Methods with this annotation are permitted to take either 0 or 1 argument. If an argument + * is used, it must be of type {@link ConfigurationContext} if the component is a ReportingTask + * or of type {@link ProcessContext} if the component is a Processor. + *

* * @author none */ diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnUnscheduled.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnUnscheduled.java index b1dbde1bf2..5c7e13dc2c 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnUnscheduled.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnUnscheduled.java @@ -47,8 +47,6 @@ import java.lang.annotation.Target; * If using 1 argument and the component using the annotation is a Reporting Task, that argument must * be of type {@link org.apache.nifi.controller.ConfigurationContext ConfigurationContext}. *

- * - * @author none */ @Documented @Target({ElementType.METHOD}) diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java b/nifi/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java index 82372af540..e62ff7962c 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java @@ -142,9 +142,19 @@ public final class PropertyDescriptor implements Comparable final Set validIdentifiers = context.getControllerServiceLookup().getControllerServiceIdentifiers(controllerServiceDefinition); if (validIdentifiers != null && validIdentifiers.contains(input)) { final ControllerService controllerService = context.getControllerServiceLookup().getControllerService(input); - if (!context.getControllerServiceLookup().isControllerServiceEnabled(controllerService)) { + if ( !context.isValidationRequired(controllerService) ) { return new ValidationResult.Builder() - .input(input) + .input(input) + .subject(getName()) + .valid(true) + .build(); + } + + final String serviceId = controllerService.getIdentifier(); + if (!context.getControllerServiceLookup().isControllerServiceEnabled(serviceId) && + !context.getControllerServiceLookup().isControllerServiceEnabling(serviceId)) { + return new ValidationResult.Builder() + .input(context.getControllerServiceLookup().getControllerServiceName(serviceId)) .subject(getName()) .valid(false) .explanation("Controller Service " + controllerService + " is disabled") diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java b/nifi/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java index b7b72c5141..61b68a249c 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java @@ -80,6 +80,15 @@ public interface ValidationContext { */ String getAnnotationData(); + /** + * There are times when the framework needs to consider a component valid, even if it + * references an invalid ControllerService. This method will return false + * if the component is to be considered valid even if the given Controller Service is referenced + * and is invalid. + * @param service + */ + boolean isValidationRequired(ControllerService service); + /** * Returns true if the given value contains a NiFi Expression Language expression, * false if it does not diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/AbstractControllerService.java b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/AbstractControllerService.java index c12f2f8d1f..71cdd231ea 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/AbstractControllerService.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/AbstractControllerService.java @@ -22,6 +22,7 @@ import org.apache.nifi.components.AbstractConfigurableComponent; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; import org.apache.nifi.controller.annotation.OnConfigured; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.reporting.InitializationException; @@ -30,11 +31,13 @@ public abstract class AbstractControllerService extends AbstractConfigurableComp private String identifier; private ControllerServiceLookup serviceLookup; private volatile ConfigurationContext configContext; - + private ComponentLog logger; + @Override public final void initialize(final ControllerServiceInitializationContext context) throws InitializationException { this.identifier = context.getIdentifier(); serviceLookup = context.getControllerServiceLookup(); + logger = context.getLogger(); init(context); } @@ -88,4 +91,12 @@ public abstract class AbstractControllerService extends AbstractConfigurableComp */ protected void init(final ControllerServiceInitializationContext config) throws InitializationException { } + + /** + * Returns the logger that has been provided to the component by the framework in its initialize method. + * @return + */ + protected ComponentLog getLogger() { + return logger; + } } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceInitializationContext.java b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceInitializationContext.java index b5b0412cb3..d34c6354ad 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceInitializationContext.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceInitializationContext.java @@ -16,6 +16,8 @@ */ package org.apache.nifi.controller; +import org.apache.nifi.logging.ComponentLog; + public interface ControllerServiceInitializationContext { /** @@ -33,4 +35,12 @@ public interface ControllerServiceInitializationContext { * @return */ ControllerServiceLookup getControllerServiceLookup(); + + /** + * Returns a logger that can be used to log important events in a standard way and generate + * bulletins when appropriate + * + * @return + */ + ComponentLog getLogger(); } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceLookup.java b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceLookup.java index 77b8e6296d..4b96f626a6 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceLookup.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceLookup.java @@ -41,6 +41,18 @@ public interface ControllerServiceLookup { */ boolean isControllerServiceEnabled(String serviceIdentifier); + /** + * Returns true if the Controller Service with the given + * identifier has been enabled but is still in the transitioning state, + * otherwise returns false. + * If the given identifier is not known by this ControllerServiceLookup, + * returns false. + * + * @param serviceIdentifier + * @return + */ + boolean isControllerServiceEnabling(String serviceIdentifier); + /** * Returns true if the given Controller Service is enabled, * false otherwise. If the given Controller Service is not @@ -63,4 +75,11 @@ public interface ControllerServiceLookup { */ Set getControllerServiceIdentifiers(Class serviceType) throws IllegalArgumentException; + /** + * Returns the name of the Controller service with the given identifier. If no service can be + * found with this identifier, returns {@code null}. + * @param serviceIdentifier + * @return + */ + String getControllerServiceName(String serviceIdentifier); } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/logging/ComponentLog.java b/nifi/nifi-api/src/main/java/org/apache/nifi/logging/ComponentLog.java new file mode 100644 index 0000000000..c070e23698 --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/logging/ComponentLog.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.logging; + + +/** + *

+ * The ComponentLog provides a mechanism to ensure that all NiFi components are logging and reporting + * information in a consistent way. When messages are logged to the ComponentLog, each message has the + * following characteristics: + *

+ * + *
    + *
  • + * The toString() of the component is automatically prepended to the message so that it is clear + * which component is providing the information. This is important, since a single component may have many + * different instances within the same NiFi instance. + *
  • + *
  • + * If the last value in an Object[] argument that is passed to the logger is a Throwable, then the logged message + * will include a toString() of the Throwable; in addition, if the component's logger is set to + * DEBUG level via the logback configuration, the Stacktrace will also be logged. This provides a mechanism to easily + * enable stacktraces in the logs when they are desired without filling the logs with unneeded stack traces for messages + * that end up occurring often. + *
  • + *
  • + * Any message that is logged with a Severity level that meets or exceeds the configured Bulletin Level for that component + * will also cause a Bulletin to be generated, so that the message is visible in the UI, allowing Dataflow Managers + * to understand that a problem exists and what the issue is. + *
  • + *
+ * + */ +public interface ComponentLog { + void warn(String msg, Throwable t); + + void warn(String msg, Object[] os); + + void warn(String msg, Object[] os, Throwable t); + + void warn(String msg); + + void trace(String msg, Throwable t); + + void trace(String msg, Object[] os); + + void trace(String msg); + + void trace(String msg, Object[] os, Throwable t); + + boolean isWarnEnabled(); + + boolean isTraceEnabled(); + + boolean isInfoEnabled(); + + boolean isErrorEnabled(); + + boolean isDebugEnabled(); + + void info(String msg, Throwable t); + + void info(String msg, Object[] os); + + void info(String msg); + + void info(String msg, Object[] os, Throwable t); + + String getName(); + + void error(String msg, Throwable t); + + void error(String msg, Object[] os); + + void error(String msg); + + void error(String msg, Object[] os, Throwable t); + + void debug(String msg, Throwable t); + + void debug(String msg, Object[] os); + + void debug(String msg, Object[] os, Throwable t); + + void debug(String msg); +} diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/logging/ProcessorLog.java b/nifi/nifi-api/src/main/java/org/apache/nifi/logging/ProcessorLog.java index c5fa7b15ca..0d66d8553b 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/logging/ProcessorLog.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/logging/ProcessorLog.java @@ -16,58 +16,15 @@ */ package org.apache.nifi.logging; -public interface ProcessorLog { - void warn(String msg, Throwable t); - - void warn(String msg, Object[] os); - - void warn(String msg, Object[] os, Throwable t); - - void warn(String msg); - - void trace(String msg, Throwable t); - - void trace(String msg, Object[] os); - - void trace(String msg); - - void trace(String msg, Object[] os, Throwable t); - - boolean isWarnEnabled(); - - boolean isTraceEnabled(); - - boolean isInfoEnabled(); - - boolean isErrorEnabled(); - - boolean isDebugEnabled(); - - void info(String msg, Throwable t); - - void info(String msg, Object[] os); - - void info(String msg); - - void info(String msg, Object[] os, Throwable t); - - String getName(); - - void error(String msg, Throwable t); - - void error(String msg, Object[] os); - - void error(String msg); - - void error(String msg, Object[] os, Throwable t); - - void debug(String msg, Throwable t); - - void debug(String msg, Object[] os); - - void debug(String msg, Object[] os, Throwable t); - - void debug(String msg); +/** + * The ProcessorLog is an extension of ComponentLog but provides no additional functionality. + * It exists because ProcessorLog was created first, + * but when Controller Services and Reporting Tasks began to be used more heavily loggers + * were needed for them as well. We did not want to return a ProcessorLog to a ControllerService + * or a ReportingTask, so all of the methods were moved to a higher interface named ComponentLog. + * However, we kept the ProcessorLog interface around in order to maintain backward compatibility. + */ +public interface ProcessorLog extends ComponentLog { } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/AbstractReportingTask.java b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/AbstractReportingTask.java index 5ed8f24ac4..efcf2a3637 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/AbstractReportingTask.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/AbstractReportingTask.java @@ -20,6 +20,7 @@ import java.util.concurrent.TimeUnit; import org.apache.nifi.components.AbstractConfigurableComponent; import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.ProcessorInitializationContext; public abstract class AbstractReportingTask extends AbstractConfigurableComponent implements ReportingTask { @@ -28,10 +29,12 @@ public abstract class AbstractReportingTask extends AbstractConfigurableComponen private String name; private long schedulingNanos; private ControllerServiceLookup serviceLookup; + private ComponentLog logger; @Override public final void initialize(final ReportingInitializationContext config) throws InitializationException { identifier = config.getIdentifier(); + logger = config.getLogger(); name = config.getName(); schedulingNanos = config.getSchedulingPeriod(TimeUnit.NANOSECONDS); serviceLookup = config.getControllerServiceLookup(); @@ -91,4 +94,11 @@ public abstract class AbstractReportingTask extends AbstractConfigurableComponen protected void init(final ReportingInitializationContext config) throws InitializationException { } + /** + * Returns the logger that has been provided to the component by the framework in its initialize method. + * @return + */ + protected ComponentLog getLogger() { + return logger; + } } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingInitializationContext.java b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingInitializationContext.java index a0ae88ea31..6b84589d75 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingInitializationContext.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingInitializationContext.java @@ -19,6 +19,7 @@ package org.apache.nifi.reporting; import java.util.concurrent.TimeUnit; import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.scheduling.SchedulingStrategy; /** @@ -77,4 +78,13 @@ public interface ReportingInitializationContext { * @return */ SchedulingStrategy getSchedulingStrategy(); + + + /** + * Returns a logger that can be used to log important events in a standard way and generate + * bulletins when appropriate + * + * @return + */ + ComponentLog getLogger(); } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/ClusterRequestException.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ClusterRequestException.java index 0ecea3b56b..ee5f417d1f 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/web/ClusterRequestException.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ClusterRequestException.java @@ -17,6 +17,7 @@ package org.apache.nifi.web; /** + * An general error occurred when attempting to communicate with the cluster. */ public class ClusterRequestException extends RuntimeException { diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/ComponentDetails.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ComponentDetails.java new file mode 100644 index 0000000000..0b68ed95ac --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ComponentDetails.java @@ -0,0 +1,157 @@ +/* + * 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; + +import java.util.Collection; +import java.util.Map; + +/** + * Details about a given component. Contains configuration and current validation errors. + */ +public class ComponentDetails { + + private final String id; + private final String name; + private final String type; + private final String state; + private final String annotationData; + private final Map properties; + private final Collection validationErrors; + + private ComponentDetails(final Builder builder) { + this.id = builder.id; + this.name = builder.name; + this.type = builder.type; + this.state = builder.state; + this.annotationData = builder.annotationData; + this.properties = builder.properties; + this.validationErrors = builder.validationErrors; + } + + /** + * The component id. + * + * @return + */ + public String getId() { + return id; + } + + /** + * The component name. + * + * @return + */ + public String getName() { + return name; + } + + /** + * The component type. + * + * @return + */ + public String getType() { + return type; + } + + /** + * The component state. + * + * @return + */ + public String getState() { + return state; + } + + /** + * The component's annotation data. + * + * @return + */ + public String getAnnotationData() { + return annotationData; + } + + /** + * Mapping of component properties. + * + * @return + */ + public Map getProperties() { + return properties; + } + + /** + * Current validation errors for the component. + * + * @return + */ + public Collection getValidationErrors() { + return validationErrors; + } + + public static final class Builder { + + private String id; + private String name; + private String type; + private String state; + private String annotationData; + private Map properties; + private Collection validationErrors; + + public Builder id(final String id) { + this.id = id; + return this; + } + + public Builder name(final String name) { + this.name = name; + return this; + } + + public Builder type(final String type) { + this.type = type; + return this; + } + + public Builder state(final String state) { + this.state = state; + return this; + } + + public Builder annotationData(final String annotationData) { + this.annotationData = annotationData; + return this; + } + + public Builder properties(final Map properties) { + this.properties = properties; + return this; + } + + public Builder validateErrors(final Collection validationErrors) { + this.validationErrors = validationErrors; + return this; + } + + public ComponentDetails build() { + return new ComponentDetails(this); + } + } +} diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/ConfigurationAction.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ConfigurationAction.java new file mode 100644 index 0000000000..066e77254c --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ConfigurationAction.java @@ -0,0 +1,137 @@ +/* + * 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; + +/** + * An action that represents the configuration of a component. + */ +public class ConfigurationAction { + + private final String id; + private final String name; + private final String type; + private final String field; + private final String previousValue; + private final String value; + + private ConfigurationAction(final Builder builder) { + this.id = builder.id; + this.name = builder.name; + this.type = builder.type; + this.field = builder.field; + this.previousValue = builder.previousValue; + this.value = builder.value; + } + + /** + * The id of the component being modified. + * + * @return + */ + public String getId() { + return id; + } + + /** + * The name of the component being modified. + * + * @return + */ + public String getName() { + return name; + } + + /** + * The type of the component being modified. + * + * @return + */ + public String getType() { + return type; + } + + /** + * Gets the name of the field, property, etc that has been modified. + * + * @return + */ + public String getField() { + return field; + } + + /** + * Gets the previous value. + * + * @return + */ + public String getPreviousValue() { + return previousValue; + } + + /** + * Gets the new value. + * + * @return + */ + public String getValue() { + return value; + } + + public static class Builder { + + private String id; + private String name; + private String type; + private String field; + private String previousValue; + private String value; + + public Builder id(final String id) { + this.id = id; + return this; + } + + public Builder name(final String name) { + this.name = name; + return this; + } + + public Builder type(final String type) { + this.type = type; + return this; + } + + public Builder field(final String field) { + this.field = field; + return this; + } + + public Builder previousValue(final String previousValue) { + this.previousValue = previousValue; + return this; + } + + public Builder value(final String value) { + this.value = value; + return this; + } + + public ConfigurationAction build() { + return new ConfigurationAction(this); + } + } +} diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationContext.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationContext.java new file mode 100644 index 0000000000..50f0ca30f3 --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationContext.java @@ -0,0 +1,102 @@ +/* + * 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; + +import java.util.Collection; + +import org.apache.nifi.controller.ControllerService; + +/** + * NiFi web context providing limited access to dataflow configuration for + * component custom UIs. + */ +public interface NiFiWebConfigurationContext { + + /** + * Gets the ControllerService for the specified identifier. If a + * corresponding service cannot be found, null is returned. If this NiFi is + * clustered, the only services available will be those those + * availability is NCM only. + * + * @param serviceIdentifier + * @return + */ + ControllerService getControllerService(String serviceIdentifier); + + /** + * Provides a mechanism for custom UIs to save actions to appear in NiFi + * configuration history. Note all fields within each Action must be + * populated. Null values will result in a failure to insert the audit + * record. Since the saving to these actions is separate from the actual + * configuration change, a failure to insert here will just generate a + * warning log message. The recording of these actions typically happens + * after a configuration change is applied. Since those changes have already + * been applied to the flow, we cannot revert them because of a failure to + * insert an audit record. + * + * @param requestContext + * @param actions + * @throws IllegalArgumentException When the requestContext isn't fully populated or + * isn't appropriate for the given request + */ + void saveActions(NiFiWebRequestContext requestContext, Collection actions); + + /** + * Gets the current user dn. Returns null if no user is found. + * + * @return + */ + String getCurrentUserDn(); + + /** + * Gets the current user name. Returns null if no user is found. + * + * @return + */ + String getCurrentUserName(); + + /** + * Sets the annotation data for the underlying component. + * + * @param configurationContext + * @param annotationData + * @return the configuration for the underlying component + * @throws ResourceNotFoundException if the underlying component does not exit + * @throws InvalidRevisionException if a revision other than the current + * revision is given + * @throws ClusterRequestException if the annotation data was unable to be + * set for the underlying component. This exception will only be thrown when operating + * in a cluster. + * @throws IllegalArgumentException When the requestContext isn't fully populated or + * isn't appropriate for the given request + */ + ComponentDetails setAnnotationData(NiFiWebConfigurationRequestContext configurationContext, String annotationData) throws ResourceNotFoundException, InvalidRevisionException, ClusterRequestException; + + /** + * Gets the details for the underlying component (including configuration, validation errors, and annotation data). + * + * @param requestContext + * @return the configuration for the underlying component + * @throws ResourceNotFoundException if the underlying component does not exit + * @throws ClusterRequestException if the underlying component was unable to be + * retrieved from the cluster. This exception will only be thrown when + * operating in a cluster. + * @throws IllegalArgumentException When the requestContext isn't fully populated or + * isn't appropriate for the given request + */ + ComponentDetails getComponentDetails(NiFiWebRequestContext requestContext) throws ResourceNotFoundException, ClusterRequestException; +} diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationRequestContext.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationRequestContext.java new file mode 100644 index 0000000000..791224122a --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationRequestContext.java @@ -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.web; + +/** + * Contextual details required to make a configuration request from a UI extension. + */ +public interface NiFiWebConfigurationRequestContext extends NiFiWebRequestContext { + + /** + * The revision to include in the request. + * + * @return the revision + */ + Revision getRevision(); + +} diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebContext.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebContext.java index 4c4f25d38f..01702addf8 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebContext.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebContext.java @@ -24,6 +24,7 @@ import org.apache.nifi.controller.ControllerService; * NiFi web context providing limited access to dataflow configuration for * processor custom UIs. */ +@Deprecated public interface NiFiWebContext { /** diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebContextConfig.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebContextConfig.java index 808b9d66a9..2df94e4aa4 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebContextConfig.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebContextConfig.java @@ -19,6 +19,7 @@ package org.apache.nifi.web; /** * Context configuration for methods invoked from the NiFiWebContext. */ +@Deprecated public interface NiFiWebContextConfig { /** diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebRequestContext.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebRequestContext.java new file mode 100644 index 0000000000..ac38221ae1 --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebRequestContext.java @@ -0,0 +1,58 @@ +/* + * 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; + +/** + * Contextual details required to make a request from a UI extension. + */ +public interface NiFiWebRequestContext { + + /** + * Returns the type of UI extension is making the request. + * + * @return + */ + UiExtensionType getExtensionType(); + + /** + * The request protocol scheme (http or https). When scheme is https, the + * X509Certificate can be used for subsequent remote requests. + * + * @return the protocol scheme + */ + String getScheme(); + + /** + * The id of the component. + * + * @return the ID + */ + String getId(); + + /** + * Returns the proxied entities chain. The format of the chain is as + * follows: + * + * + * <CN=original-proxied-entity><CN=first-proxy><CN=second-proxy>... + * + * + * @return the proxied entities chain or null if no chain + */ + String getProxiedEntitiesChain(); + +} diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/ProcessorConfigurationAction.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ProcessorConfigurationAction.java index 8385e4a239..ce5e069d30 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/web/ProcessorConfigurationAction.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ProcessorConfigurationAction.java @@ -19,6 +19,7 @@ package org.apache.nifi.web; /** * */ +@Deprecated public class ProcessorConfigurationAction { private final String processorId; diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/ProcessorInfo.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ProcessorInfo.java index 04810981b6..e87e73e306 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/web/ProcessorInfo.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ProcessorInfo.java @@ -22,6 +22,7 @@ import java.util.Map; /** * */ +@Deprecated public class ProcessorInfo { private final String id; diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/Revision.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/Revision.java index 1881c2f526..8a6275e1c5 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/web/Revision.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/Revision.java @@ -37,12 +37,12 @@ public class Revision implements Serializable { * the client ID */ private final String clientId; - + public Revision(Long revision, String clientId) { this.version = revision; this.clientId = clientId; } - + public String getClientId() { return clientId; } @@ -51,34 +51,6 @@ public class Revision implements Serializable { return version; } - /** - * A factory method for creating a new Revision instance whose version is - * this instance's version plus 1. - * - * @return an updated revision - */ - public Revision increment() { - final long incrementedVersion; - if (version == null) { - incrementedVersion = 0; - } else { - incrementedVersion = version + 1; - } - return new Revision(incrementedVersion, clientId); - } - - /** - * A factory method for creating a new Revision instance whose version is - * this instance's version plus 1 and whose client ID is the given client - * ID. - * - * @param clientId the client ID - * @return an updated revision - */ - public Revision increment(String clientId) { - return new Revision(increment().getVersion(), clientId); - } - @Override public boolean equals(final Object obj) { diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/UiExtensionType.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/UiExtensionType.java new file mode 100644 index 0000000000..0bbda16d66 --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/UiExtensionType.java @@ -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.web; + +/** + * Types of UI extensions. Since a UI extension could support multiple + * types of custom UIs it will need to include the type so the framework + * can appropriate understand and process the request (recording actions + * in the audit database, replicating a request throughout the cluster to + * the appropriate endpoints, etc). + */ +public enum UiExtensionType { + ContentViewer, + ProcessorConfiguration, + ControllerServiceConfiguration, + ReportingTaskConfiguration +} diff --git a/nifi/nifi-assembly/NOTICE b/nifi/nifi-assembly/NOTICE index 8d7db8d7ec..f14c662257 100644 --- a/nifi/nifi-assembly/NOTICE +++ b/nifi/nifi-assembly/NOTICE @@ -501,6 +501,54 @@ The following binary components are provided under the Apache Software License v Apache License Version 2.0 http://www.apache.org/licenses/. (c) Daniel Lemire, http://lemire.me/en/ + (ASLv2) Twitter4J + The following NOTICE information applies: + Copyright 2007 Yusuke Yamamoto + + Twitter4J includes software from JSON.org to parse JSON response from the Twitter API. You can see the license term at http://www.JSON.org/license.html + + (ASLv2) JOAuth + The following NOTICE information applies: + JOAuth + Copyright 2010-2013 Twitter, Inc + + Licensed under the Apache License, Version 2.0: http://www.apache.org/licenses/LICENSE-2.0 + + (ASLv2) Hosebird Client + The following NOTICE information applies: + Hosebird Client (hbc) + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License, Version 2.0: http://www.apache.org/licenses/LICENSE-2.0 + + (ASLv2) GeoIP2 Java API + The following NOTICE information applies: + GeoIP2 Java API + This software is Copyright (c) 2013 by MaxMind, Inc. + + This is free software, licensed under the Apache License, Version 2.0. + + (ASLv2) Google HTTP Client Library for Java + The following NOTICE information applies: + Google HTTP Client Library for Java + + This is free software, licensed under the Apache License, Version 2.0. + + (ASLv2) Amazon Web Services SDK + The following NOTICE information applies: + Copyright 2010-2014 Amazon.com, Inc. or its affiliates. All Rights Reserved. + + This product includes software developed by + Amazon Technologies, Inc (http://www.amazon.com/). + + ********************** + THIRD PARTY COMPONENTS + ********************** + This software includes third party software subject to the following copyrights: + - XML parsing and utility functions from JetS3t - Copyright 2006-2009 James Murty. + - JSON parsing and utility functions from JSON.org - Copyright 2002 JSON.org. + - PKCS#1 PEM encoded private key parsing and utility functions from oauth.googlecode.com - Copyright 1998-2010 AOL Inc. + ************************ Common Development and Distribution License 1.1 @@ -540,6 +588,14 @@ The following binary components are provided under the Common Development and Di (CDDL 1.0) JavaServer Pages(TM) API (javax.servlet.jsp:jsp-api:jar:2.1 - http://jsp.java.net) (CDDL 1.0) SR 250 Common Annotations For The JavaTM Platform (javax.annotation:jsr250-api:jar:1.0 - http://jcp.org/aboutJava/communityprocess/final/jsr250/index.html) +************************ +Creative Commons Attribution-ShareAlike 3.0 +************************ + +The following binary components are provided under the Creative Commons Attribution-ShareAlike 3.0. See project link for details. + + (CCAS 3.0) MaxMind DB (https://github.com/maxmind/MaxMind-DB) + ************************ Eclipse Public License 1.0 ************************ @@ -559,6 +615,15 @@ The following binary components are provided under the Mozilla Public License v2 (MPL 2.0) Saxon HE (net.sf.saxon:Saxon-HE:jar:9.6.0-4 - http://www.saxonica.com/) +***************** +Mozilla Public License v1.1 +***************** + +The following binary components are provided under the Mozilla Public License v1.1. See project link for details. + + (MPL 1.1) HAPI Base (ca.uhn.hapi:hapi-base:2.2 - http://hl7api.sourceforge.net/) + (MPL 1.1) HAPI Structures (ca.uhn.hapi:hapi-structures-v*:2.2 - http://hl7api.sourceforge.net/) + ***************** Public Domain ***************** diff --git a/nifi/nifi-assembly/pom.xml b/nifi/nifi-assembly/pom.xml index cae0f00a90..13ffba802d 100644 --- a/nifi/nifi-assembly/pom.xml +++ b/nifi/nifi-assembly/pom.xml @@ -1,459 +1,484 @@ - - - 4.0.0 - - org.apache.nifi - nifi - 0.1.0-incubating-SNAPSHOT - - nifi-assembly - pom - This is the assembly Apache NiFi (incubating) - - - - maven-assembly-plugin - - nifi-${project.version} - false - - - - make shared resource - - single - - package - - - src/main/assembly/dependencies.xml - - posix - - - - - - - - - ch.qos.logback - logback-classic - compile - - - org.slf4j - jcl-over-slf4j - compile - - - org.slf4j - jul-to-slf4j - compile - - - org.slf4j - log4j-over-slf4j - compile - - - org.slf4j - slf4j-api - compile - - - org.apache.nifi - nifi-api - - - org.apache.nifi - nifi-runtime - - - org.apache.nifi - nifi-bootstrap - - - org.apache.nifi - nifi-resources - resources - runtime - zip - - - org.apache.nifi - nifi-docs - resources - runtime - zip - - - org.apache.nifi - nifi-framework-nar - nar - - - org.apache.nifi - nifi-provenance-repository-nar - nar - - - org.apache.nifi - nifi-standard-services-api-nar - nar - - - org.apache.nifi - nifi-ssl-context-service-nar - nar - - - org.apache.nifi - nifi-distributed-cache-services-nar - nar - - - org.apache.nifi - nifi-standard-nar - nar - - - org.apache.nifi - nifi-jetty-bundle - nar - - - org.apache.nifi - nifi-update-attribute-nar - nar - - - org.apache.nifi - nifi-hadoop-libraries-nar - nar - - - org.apache.nifi - nifi-hadoop-nar - nar - - - org.apache.nifi - nifi-kafka-nar - nar - + + + 4.0.0 + + org.apache.nifi + nifi + 0.1.0-incubating-SNAPSHOT + + nifi-assembly + pom + This is the assembly Apache NiFi (incubating) + + + + maven-assembly-plugin + + nifi-${project.version} + false + + + + make shared resource + + single + + package + + + src/main/assembly/dependencies.xml + + posix + + + + + + + - org.apache.nifi - nifi-http-context-map-nar - nar - + ch.qos.logback + logback-classic + compile + - org.apache.nifi - nifi-kite-nar - nar - - - - - - 256 - 512 - 128 - 128 - 10m - 10 - - - ${project.version} - true - 10 sec - 500 ms - 30 sec - 10 millis + org.slf4j + jcl-over-slf4j + compile + + + org.slf4j + jul-to-slf4j + compile + + + org.slf4j + log4j-over-slf4j + compile + + + org.slf4j + slf4j-api + compile + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-runtime + + + org.apache.nifi + nifi-bootstrap + + + org.apache.nifi + nifi-resources + resources + runtime + zip + + + org.apache.nifi + nifi-docs + resources + runtime + zip + + + org.apache.nifi + nifi-framework-nar + nar + + + org.apache.nifi + nifi-provenance-repository-nar + nar + + + org.apache.nifi + nifi-standard-services-api-nar + nar + + + org.apache.nifi + nifi-ssl-context-service-nar + nar + + + org.apache.nifi + nifi-distributed-cache-services-nar + nar + + + org.apache.nifi + nifi-standard-nar + nar + + + org.apache.nifi + nifi-jetty-bundle + nar + + + org.apache.nifi + nifi-update-attribute-nar + nar + + + org.apache.nifi + nifi-hadoop-libraries-nar + nar + + + org.apache.nifi + nifi-hadoop-nar + nar + + + org.apache.nifi + nifi-kafka-nar + nar + + + org.apache.nifi + nifi-http-context-map-nar + nar + + + org.apache.nifi + nifi-kite-nar + nar + + + org.apache.nifi + nifi-social-media-nar + 0.1.0-incubating-SNAPSHOT + nar + + + org.apache.nifi + nifi-hl7-nar + 0.1.0-incubating-SNAPSHOT + nar + + + org.apache.nifi + nifi-language-translation-nar + 0.1.0-incubating-SNAPSHOT + nar + + + org.apache.nifi + nifi-geo-nar + 0.1.0-incubating-SNAPSHOT + nar + + - ./conf/flow.xml.gz - ./conf/archive/ - ./conf/reporting-tasks.xml - ./conf/controller-services.xml - ./conf/authority-providers.xml - ./conf/templates - ./database_repository + + + 256 + 512 + 128 + 128 + 10m + 10 - org.apache.nifi.controller.repository.WriteAheadFlowFileRepository - ./flowfile_repository - 256 - 2 mins - false - org.apache.nifi.controller.FileSystemSwapManager - 20000 - 5 sec - 1 - 5 sec - 4 - - org.apache.nifi.controller.repository.FileSystemRepository - 10 MB - 100 - ./content_repository - - - false - false - - - - - 30 sec - ./lib - ./work/nar/ - ./work/docs/components - - PBEWITHMD5AND256BITAES-CBC-OPENSSL - BC - ;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE + + ${project.version} + true + 10 sec + 500 ms + 30 sec + 10 millis - 9990 - - - org.apache.nifi.provenance.PersistentProvenanceRepository - ./provenance_repository - 24 hours - 1 GB - 5 mins - 100 MB - 2 - true - EventType, FlowFileUUID, Filename, ProcessorID - - 500 MB - false - 16 - - - 100000 - - - org.apache.nifi.controller.status.history.VolatileComponentStatusRepository - 288 - 5 mins - - - ./lib - - 8080 - - - ./work/jetty - 200 - - - - - - - - - - - ./conf/authorized-users.xml - 24 hours - file-provider - - - - - - - 5 sec - false - 30 sec - 45 sec - false - - - 500 ms - 3 - 1 sec + ./conf/flow.xml.gz + ./conf/archive/ + ./conf/authority-providers.xml + ./conf/templates + ./database_repository - - false - - - 2 - - - - - false - - - - 10 - 30 sec - 30 sec - 10 - 5 sec - 10 - 0 sec - - - - rpm - - false - - - - - maven-dependency-plugin - - - unpack-shared-resources - - unpack-dependencies - - generate-resources - - ${project.build.directory}/generated-resources - nifi-resources - org.apache.nifi - false - - - - unpack-docs - - unpack-dependencies - - generate-resources - - ${project.build.directory}/generated-docs - nifi-docs - org.apache.nifi - false - - - - - - org.codehaus.mojo - rpm-maven-plugin - - Apache NiFi (incubating) - Apache Nifi (incubating) is dataflow system based on the Flow-Based Programming concepts. - Apache License, Version 2.0 and others (see included LICENSE file) - http://nifi.incubator.apache.org - Utilities - /opt/nifi - - _use_internal_dependency_generator 0 - - 750 - 640 - root - root - - - - build-bin-rpm - - attached-rpm - - - bin - - nifi - - - - /opt/nifi/nifi-${project.version} - - - /opt/nifi/nifi-${project.version} - - - ./LICENSE - - - ./NOTICE - - - ../DISCLAIMER - - - ./README.md - README - - - - - /opt/nifi/nifi-${project.version}/bin - 750 - - - ${project.build.directory}/generated-resources/bin/nifi.sh - nifi.sh - true - - - - - /opt/nifi/nifi-${project.version}/conf - true - - - ${project.build.directory}/generated-resources/conf - true - - - - - /opt/nifi/nifi-${project.version}/lib - - - org.apache.nifi:nifi-bootstrap - org.apache.nifi:nifi-resources - org.apache.nifi:nifi-docs - - - - - /opt/nifi/nifi-${project.version}/lib/bootstrap - - - org.apache.nifi:nifi-bootstrap - - - - - /opt/nifi/nifi-${project.version}/docs - - - ${project.build.directory}/generated-docs - - - - - - - - - - - - + org.apache.nifi.controller.repository.WriteAheadFlowFileRepository + ./flowfile_repository + 256 + 2 mins + false + org.apache.nifi.controller.FileSystemSwapManager + 20000 + 5 sec + 1 + 5 sec + 4 + + org.apache.nifi.controller.repository.FileSystemRepository + 10 MB + 100 + ./content_repository + + + false + false + + + + + 30 sec + ./lib + ./work/nar/ + ./work/docs/components + + PBEWITHMD5AND256BITAES-CBC-OPENSSL + BC + ;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE + + 9990 + + + org.apache.nifi.provenance.PersistentProvenanceRepository + ./provenance_repository + 24 hours + 1 GB + 5 mins + 100 MB + 2 + true + EventType, FlowFileUUID, + Filename, ProcessorID + + 500 MB + false + 16 + + + 100000 + + + org.apache.nifi.controller.status.history.VolatileComponentStatusRepository + 288 + 5 mins + + + ./lib + + 8080 + + + ./work/jetty + 200 + + + + + + + + + + + ./conf/authorized-users.xml + 24 hours + file-provider + + + + + + + 5 sec + false + 30 sec + 45 sec + false + + + 500 ms + 3 + 1 sec + + + false + + + 2 + + + + + false + + + + 10 + 30 sec + 30 sec + 10 + 5 sec + 10 + 0 sec + + + + rpm + + false + + + + + maven-dependency-plugin + + + unpack-shared-resources + + unpack-dependencies + + generate-resources + + ${project.build.directory}/generated-resources + nifi-resources + org.apache.nifi + false + + + + unpack-docs + + unpack-dependencies + + generate-resources + + ${project.build.directory}/generated-docs + nifi-docs + org.apache.nifi + false + + + + + + org.codehaus.mojo + rpm-maven-plugin + + Apache NiFi (incubating) + Apache Nifi (incubating) is dataflow system based on + the Flow-Based Programming concepts. + Apache License, Version 2.0 and others (see included + LICENSE file) + http://nifi.incubator.apache.org + Utilities + /opt/nifi + + _use_internal_dependency_generator 0 + + 750 + 640 + root + root + + + + build-bin-rpm + + attached-rpm + + + bin + + nifi + + + + /opt/nifi/nifi-${project.version} + + + /opt/nifi/nifi-${project.version} + + + ./LICENSE + + + ./NOTICE + + + ../DISCLAIMER + + + ./README.md + README + + + + + /opt/nifi/nifi-${project.version}/bin + 750 + + + ${project.build.directory}/generated-resources/bin/nifi.sh + nifi.sh + true + + + + + /opt/nifi/nifi-${project.version}/conf + true + + + ${project.build.directory}/generated-resources/conf + true + + + + + /opt/nifi/nifi-${project.version}/lib + + + org.apache.nifi:nifi-bootstrap + org.apache.nifi:nifi-resources + org.apache.nifi:nifi-docs + + + + + /opt/nifi/nifi-${project.version}/lib/bootstrap + + + org.apache.nifi:nifi-bootstrap + + + + + /opt/nifi/nifi-${project.version}/docs + + + ${project.build.directory}/generated-docs + + + + + + + + + + + + diff --git a/nifi/nifi-commons/nifi-hl7-query-language/.gitignore b/nifi/nifi-commons/nifi-hl7-query-language/.gitignore new file mode 100644 index 0000000000..e91d5c41bb --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/.gitignore @@ -0,0 +1,3 @@ +/target/ +/target/ +/target/ diff --git a/nifi/nifi-commons/nifi-hl7-query-language/pom.xml b/nifi/nifi-commons/nifi-hl7-query-language/pom.xml new file mode 100644 index 0000000000..7daa400835 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/pom.xml @@ -0,0 +1,122 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-commons + 0.1.0-incubating-SNAPSHOT + + + nifi-hl7-query-language + jar + + + + maven-compiler-plugin + + 1.7 + 1.7 + + + + org.antlr + antlr3-maven-plugin + + + + antlr + + + + + + org.apache.rat + apache-rat-plugin + + + src/test/resources/hypoglycemia + src/test/resources/hyperglycemia + + + + + + + + + org.antlr + antlr-runtime + 3.5.2 + + + + + ca.uhn.hapi + hapi-base + 2.2 + + + ca.uhn.hapi + hapi-structures-v21 + 2.2 + + + ca.uhn.hapi + hapi-structures-v22 + 2.2 + + + ca.uhn.hapi + hapi-structures-v23 + 2.2 + + + ca.uhn.hapi + hapi-structures-v231 + 2.2 + + + ca.uhn.hapi + hapi-structures-v24 + 2.2 + + + ca.uhn.hapi + hapi-structures-v25 + 2.2 + + + ca.uhn.hapi + hapi-structures-v251 + 2.2 + + + ca.uhn.hapi + hapi-structures-v26 + 2.2 + + + + + junit + junit + test + + + diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryLexer.g b/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryLexer.g new file mode 100644 index 0000000000..478028b9f7 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryLexer.g @@ -0,0 +1,173 @@ +/* + * 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. + */ + +lexer grammar HL7QueryLexer; + +@header { + package org.apache.nifi.hl7.query.antlr; + import org.apache.nifi.hl7.query.exception.HL7QueryParsingException; +} + +@rulecatch { + catch(final Exception e) { + throw new HL7QueryParsingException(e); + } +} + +@members { + public void displayRecognitionError(String[] tokenNames, RecognitionException e) { + final StringBuilder sb = new StringBuilder(); + if ( e.token == null ) { + sb.append("Unrecognized token "); + } else { + sb.append("Unexpected token '").append(e.token.getText()).append("' "); + } + sb.append("at line ").append(e.line); + if ( e.approximateLineInfo ) { + sb.append(" (approximately)"); + } + sb.append(", column ").append(e.charPositionInLine); + sb.append(". Query: ").append(e.input.toString()); + + throw new HL7QueryParsingException(sb.toString()); + } + + public void recover(RecognitionException e) { + final StringBuilder sb = new StringBuilder(); + if ( e.token == null ) { + sb.append("Unrecognized token "); + } else { + sb.append("Unexpected token '").append(e.token.getText()).append("' "); + } + sb.append("at line ").append(e.line); + if ( e.approximateLineInfo ) { + sb.append(" (approximately)"); + } + sb.append(", column ").append(e.charPositionInLine); + sb.append(". Query: ").append(e.input.toString()); + + throw new HL7QueryParsingException(sb.toString()); + } +} + + +// PUNCTUATION & SPECIAL CHARACTERS +WHITESPACE : (' '|'\t'|'\n'|'\r')+ { $channel = HIDDEN; }; +COMMENT : '#' ( ~('\n') )* '\n' { $channel = HIDDEN; }; + +LPAREN : '('; +RPAREN : ')'; +LBRACE : '{'; +RBRACE : '}'; +COLON : ':'; +COMMA : ','; +DOT : '.'; +SEMICOLON : ';'; + + + +// OPERATORS +EQUALS : '='; +NOT_EQUALS : '!='; +GT : '>'; +GE : '>='; +LT : '<'; +LE : '<='; +REGEX : 'MATCHES REGEX'; +LIKE : 'LIKE'; +IS_NULL : 'IS NULL'; +NOT_NULL : 'NOT NULL'; + + +// KEYWORDS +AND : 'AND'; +OR : 'OR'; +NOT : 'NOT'; + +TRUE : 'true'; +FALSE : 'false'; + +SELECT : 'select' | 'SELECT'; +DECLARE : 'declare' | 'DECLARE'; +OPTIONAL : 'optional' | 'OPTIONAL'; +REQUIRED : 'required' | 'REQUIRED'; +AS : 'as' | 'AS'; +WHERE : 'where' | 'WHERE'; + +MESSAGE : 'MESSAGE' | 'message'; +SEGMENT : 'SEGMENT' | 'segment'; + + +SEGMENT_NAME : LETTER ALPHA_NUMERIC ALPHA_NUMERIC; + + +NUMBER : ('0'..'9')+; +fragment LETTER : 'A'..'Z'; +fragment ALPHA_NUMERIC : 'A'..'Z' | '0'..'9'; + + +// STRINGS +STRING_LITERAL +@init{StringBuilder lBuf = new StringBuilder();} + : + ( + '"' + ( + escaped=ESC {lBuf.append(getText());} | + normal = ~( '"' | '\\' | '\n' | '\r' | '\t' ) { lBuf.appendCodePoint(normal);} + )* + '"' + ) + { + setText(lBuf.toString()); + } + | + ( + '\'' + ( + escaped=ESC {lBuf.append(getText());} | + normal = ~( '\'' | '\\' | '\n' | '\r' | '\t' ) { lBuf.appendCodePoint(normal);} + )* + '\'' + ) + { + setText(lBuf.toString()); + } + ; + + +fragment +ESC + : '\\' + ( + '"' { setText("\""); } + | '\'' { setText("\'"); } + | 'r' { setText("\r"); } + | 'n' { setText("\n"); } + | 't' { setText("\t"); } + | '\\' { setText("\\\\"); } + | nextChar = ~('"' | '\'' | 'r' | 'n' | 't' | '\\') + { + StringBuilder lBuf = new StringBuilder(); lBuf.append("\\\\").appendCodePoint(nextChar); setText(lBuf.toString()); + } + ) + ; + +IDENTIFIER : ( + ~('$' | '{' | '}' | '(' | ')' | '[' | ']' | ',' | ':' | ';' | '/' | '*' | '\'' | ' ' | '\t' | '\r' | '\n' | '0'..'9' | '.') + ~('$' | '{' | '}' | '(' | ')' | '[' | ']' | ',' | ':' | ';' | '/' | '*' | '\'' | ' ' | '\t' | '\r' | '\n' | '.')* + ); diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryParser.g b/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryParser.g new file mode 100644 index 0000000000..f051872eba --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/antlr3/org/apache/nifi/hl7/query/antlr/HL7QueryParser.g @@ -0,0 +1,108 @@ +/* + * 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. + */ + +parser grammar HL7QueryParser; + +options { + output=AST; + tokenVocab=HL7QueryLexer; +} + +tokens { + QUERY; + DECLARATION; +} + +@header { + package org.apache.nifi.hl7.query.antlr; + import org.apache.nifi.hl7.query.exception.HL7QueryParsingException; +} + +@members { + public void displayRecognitionError(String[] tokenNames, RecognitionException e) { + final StringBuilder sb = new StringBuilder(); + if ( e.token == null ) { + sb.append("Unrecognized token "); + } else { + sb.append("Unexpected token '").append(e.token.getText()).append("' "); + } + sb.append("at line ").append(e.line); + if ( e.approximateLineInfo ) { + sb.append(" (approximately)"); + } + sb.append(", column ").append(e.charPositionInLine); + sb.append(". Query: ").append(e.input.toString()); + + throw new HL7QueryParsingException(sb.toString()); + } + + public void recover(final RecognitionException e) { + final StringBuilder sb = new StringBuilder(); + if ( e.token == null ) { + sb.append("Unrecognized token "); + } else { + sb.append("Unexpected token '").append(e.token.getText()).append("' "); + } + sb.append("at line ").append(e.line); + if ( e.approximateLineInfo ) { + sb.append(" (approximately)"); + } + sb.append(", column ").append(e.charPositionInLine); + sb.append(". Query: ").append(e.input.toString()); + + throw new HL7QueryParsingException(sb.toString()); + } +} + + +declareClause : DECLARE^ declaration (COMMA! declaration)*; + +requiredOrOptional : REQUIRED | OPTIONAL; +declaration : IDENTIFIER AS requiredOrOptional SEGMENT_NAME -> + ^(DECLARATION IDENTIFIER requiredOrOptional SEGMENT_NAME); + + +selectClause : SELECT^ selectableClause; +selectableClause : selectable (COMMA! selectable)*; +selectable : (MESSAGE | ref | field)^ (AS! IDENTIFIER^)?; + + +whereClause : WHERE^ conditions; + +conditions : condition ((AND^ | OR^) condition)*; + +condition : NOT^ condition | LPAREN! conditions RPAREN! | evaluation; + +evaluation : expression + ( + unaryOperator^ + | (binaryOperator^ expression) + ); + +expression : (LPAREN! expr RPAREN!) | expr; +expr : ref | field | STRING_LITERAL | NUMBER; + +unaryOperator : IS_NULL | NOT_NULL; +binaryOperator : EQUALS | NOT_EQUALS | LT | GT | LE | GE; + +ref : (SEGMENT_NAME | IDENTIFIER); +field : ref DOT^ NUMBER + (DOT^ NUMBER (DOT^ NUMBER (DOT^ NUMBER)?)?)?; + + +query : declareClause? selectClause whereClause? EOF -> + ^(QUERY declareClause? selectClause whereClause?); diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/EmptyField.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/EmptyField.java new file mode 100644 index 0000000000..be645e59c8 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/EmptyField.java @@ -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.hl7.hapi; + +import java.util.Collections; +import java.util.List; + +import org.apache.nifi.hl7.model.HL7Component; +import org.apache.nifi.hl7.model.HL7Field; + +public class EmptyField implements HL7Field { + + @Override + public String getValue() { + return null; + } + + @Override + public List getComponents() { + return Collections.emptyList(); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiField.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiField.java new file mode 100644 index 0000000000..056b6b677d --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiField.java @@ -0,0 +1,83 @@ +/* + * 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.hl7.hapi; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import org.apache.nifi.hl7.model.HL7Component; +import org.apache.nifi.hl7.model.HL7Field; + +import ca.uhn.hl7v2.model.Composite; +import ca.uhn.hl7v2.model.ExtraComponents; +import ca.uhn.hl7v2.model.Primitive; +import ca.uhn.hl7v2.model.Type; +import ca.uhn.hl7v2.model.Varies; +import ca.uhn.hl7v2.parser.EncodingCharacters; +import ca.uhn.hl7v2.parser.PipeParser; + +public class HapiField implements HL7Field, HL7Component { + private final String value; + private final List components; + + public HapiField(final Type type) { + this.value = PipeParser.encode(type, EncodingCharacters.defaultInstance()); + + final List componentList = new ArrayList<>(); + if ( type instanceof Composite ) { + final Composite composite = (Composite) type; + + for ( final Type component : composite.getComponents() ) { + componentList.add(new HapiField(component)); + } + } + + final ExtraComponents extra = type.getExtraComponents(); + if ( extra != null && extra.numComponents() > 0 ) { + final String singleFieldValue; + if ( type instanceof Primitive ) { + singleFieldValue = ((Primitive) type).getValue(); + } else { + singleFieldValue = this.value; + } + componentList.add(new SingleValueField(singleFieldValue)); + + for (int i=0; i < extra.numComponents(); i++) { + final Varies varies = extra.getComponent(i); + componentList.add(new HapiField(varies)); + } + } + + this.components = Collections.unmodifiableList(componentList); + } + + @Override + public String getValue() { + return value; + } + + @Override + public List getComponents() { + return components; + } + + @Override + public String toString() { + return value; + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiMessage.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiMessage.java new file mode 100644 index 0000000000..ddd28b2bf9 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiMessage.java @@ -0,0 +1,94 @@ +/* + * 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.hl7.hapi; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.hl7.model.HL7Message; +import org.apache.nifi.hl7.model.HL7Segment; + +import ca.uhn.hl7v2.HL7Exception; +import ca.uhn.hl7v2.model.Group; +import ca.uhn.hl7v2.model.Message; +import ca.uhn.hl7v2.model.Segment; +import ca.uhn.hl7v2.model.Structure; + +public class HapiMessage implements HL7Message { + private final Message message; + private final List allSegments; + private final Map> segmentMap; + + public HapiMessage(final Message message) throws HL7Exception { + this.message = message; + + allSegments = new ArrayList<>(); + populateSegments(message, allSegments); + + segmentMap = new HashMap<>(); + for ( final HL7Segment segment : allSegments ) { + final String segmentName = segment.getName(); + List segmentList = segmentMap.get(segmentName); + if ( segmentList == null ) { + segmentList = new ArrayList<>(); + segmentMap.put(segmentName, segmentList); + } + + segmentList.add(segment); + } + } + + private void populateSegments(final Group group, final List segments) throws HL7Exception { + for ( final String structureName : group.getNames() ) { + final Structure[] structures = group.getAll(structureName); + if ( group.isGroup(structureName) ) { + for ( final Structure structure : structures ) { + populateSegments((Group) structure, segments); + } + } else { + for ( final Structure structure : structures ) { + final Segment segment = (Segment) structure; + final HapiSegment hapiSegment = new HapiSegment(segment); + segments.add(hapiSegment); + } + } + } + } + + @Override + public List getSegments() { + return Collections.unmodifiableList(allSegments); + } + + @Override + public List getSegments(final String segmentType) { + final List segments = segmentMap.get(segmentType); + if ( segments == null ) { + return Collections.emptyList(); + } + + return Collections.unmodifiableList(segments); + } + + @Override + public String toString() { + return message.toString(); + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiSegment.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiSegment.java new file mode 100644 index 0000000000..d50afdb960 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/HapiSegment.java @@ -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.hl7.hapi; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import org.apache.nifi.hl7.model.HL7Field; +import org.apache.nifi.hl7.model.HL7Segment; + +import ca.uhn.hl7v2.HL7Exception; +import ca.uhn.hl7v2.model.Segment; +import ca.uhn.hl7v2.model.Type; + +public class HapiSegment implements HL7Segment { + private final Segment segment; + private final List fields; + + public HapiSegment(final Segment segment) throws HL7Exception { + this.segment = segment; + + final List fieldList = new ArrayList<>(); + for (int i=1; i <= segment.numFields(); i++) { + final Type[] types = segment.getField(i); + + if ( types == null || types.length == 0 ) { + fieldList.add(new EmptyField()); + continue; + } + + for ( final Type type : types ) { + fieldList.add(new HapiField(type)); + } + } + + this.fields = Collections.unmodifiableList(fieldList); + } + + + @Override + public String getName() { + return segment.getName(); + } + + @Override + public List getFields() { + return fields; + } + + @Override + public String toString() { + return segment.toString(); + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/SingleValueField.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/SingleValueField.java new file mode 100644 index 0000000000..ed99077b45 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/hapi/SingleValueField.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.hl7.hapi; + +import java.util.Collections; +import java.util.List; + +import org.apache.nifi.hl7.model.HL7Component; +import org.apache.nifi.hl7.model.HL7Field; + +public class SingleValueField implements HL7Field { + private final String value; + + public SingleValueField(final String value) { + this.value = value; + } + + @Override + public String getValue() { + return value; + } + + @Override + public List getComponents() { + return Collections.emptyList(); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/io/HL7Reader.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/io/HL7Reader.java new file mode 100644 index 0000000000..e7b31a4c4a --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/io/HL7Reader.java @@ -0,0 +1,27 @@ +/* + * 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.hl7.io; + +import java.io.IOException; + +import org.apache.nifi.hl7.model.HL7Message; + +public interface HL7Reader { + + HL7Message nextMessage() throws IOException; + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/io/exception/InvalidHL7Exception.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/io/exception/InvalidHL7Exception.java new file mode 100644 index 0000000000..669f40c6bc --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/io/exception/InvalidHL7Exception.java @@ -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.hl7.io.exception; + +import java.io.IOException; + +public class InvalidHL7Exception extends IOException { + private static final long serialVersionUID = -5675416667224562441L; + + public InvalidHL7Exception() { + super(); + } + + public InvalidHL7Exception(String message, Throwable cause) { + super(message, cause); + } + + public InvalidHL7Exception(String message) { + super(message); + } + + public InvalidHL7Exception(Throwable cause) { + super(cause); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Component.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Component.java new file mode 100644 index 0000000000..cf355041b1 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Component.java @@ -0,0 +1,24 @@ +/* + * 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.hl7.model; + +import java.util.List; + +public interface HL7Component { + String getValue(); + List getComponents(); +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/Availability.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Field.java similarity index 87% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/Availability.java rename to nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Field.java index 38df6f754e..4086e581d1 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/Availability.java +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Field.java @@ -14,11 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.controller; +package org.apache.nifi.hl7.model; -public enum Availability { - CLUSTER_MANAGER_ONLY, - NODE_ONLY, - BOTH; +public interface HL7Field extends HL7Component { } diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Message.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Message.java new file mode 100644 index 0000000000..dd9c2a9ff4 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Message.java @@ -0,0 +1,27 @@ +/* + * 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.hl7.model; + +import java.util.List; + +public interface HL7Message { + + List getSegments(); + + List getSegments(String segmentType); + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Segment.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Segment.java new file mode 100644 index 0000000000..de5aaa1d4b --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/model/HL7Segment.java @@ -0,0 +1,27 @@ +/* + * 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.hl7.model; + +import java.util.List; + +public interface HL7Segment { + + String getName(); + + List getFields(); + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/Declaration.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/Declaration.java new file mode 100644 index 0000000000..0903cc8d30 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/Declaration.java @@ -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.hl7.query; + +import org.apache.nifi.hl7.model.HL7Message; + +public interface Declaration { + + String getAlias(); + + boolean isRequired(); + + Object getDeclaredValue(HL7Message message); + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/HL7Query.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/HL7Query.java new file mode 100644 index 0000000000..a036106481 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/HL7Query.java @@ -0,0 +1,412 @@ +/* + * 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.hl7.query; + +import static org.apache.nifi.hl7.query.antlr.HL7QueryParser.*; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.antlr.runtime.ANTLRStringStream; +import org.antlr.runtime.CharStream; +import org.antlr.runtime.CommonTokenStream; +import org.antlr.runtime.tree.Tree; +import org.apache.nifi.hl7.model.HL7Message; +import org.apache.nifi.hl7.query.evaluator.BooleanEvaluator; +import org.apache.nifi.hl7.query.evaluator.Evaluator; +import org.apache.nifi.hl7.query.evaluator.IntegerEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.EqualsEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.GreaterThanEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.GreaterThanOrEqualEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.IsNullEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.LessThanEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.LessThanOrEqualEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.NotEqualsEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.NotEvaluator; +import org.apache.nifi.hl7.query.evaluator.comparison.NotNullEvaluator; +import org.apache.nifi.hl7.query.evaluator.literal.IntegerLiteralEvaluator; +import org.apache.nifi.hl7.query.evaluator.literal.StringLiteralEvaluator; +import org.apache.nifi.hl7.query.evaluator.logic.AndEvaluator; +import org.apache.nifi.hl7.query.evaluator.logic.OrEvaluator; +import org.apache.nifi.hl7.query.evaluator.message.DeclaredReferenceEvaluator; +import org.apache.nifi.hl7.query.evaluator.message.DotEvaluator; +import org.apache.nifi.hl7.query.evaluator.message.MessageEvaluator; +import org.apache.nifi.hl7.query.evaluator.message.SegmentEvaluator; +import org.apache.nifi.hl7.query.exception.HL7QueryParsingException; +import org.apache.nifi.hl7.query.result.MissedResult; +import org.apache.nifi.hl7.query.result.StandardQueryResult; + +import org.apache.nifi.hl7.query.antlr.HL7QueryLexer; +import org.apache.nifi.hl7.query.antlr.HL7QueryParser; + + +public class HL7Query { + private final Tree tree; + private final String query; + private final Set declarations = new HashSet<>(); + + private final List selections; + private final BooleanEvaluator whereEvaluator; + + private HL7Query(final Tree tree, final String query) { + this.tree = tree; + this.query = query; + + List select = null; + BooleanEvaluator where = null; + for (int i=0; i < tree.getChildCount(); i++) { + final Tree child = tree.getChild(i); + + switch (child.getType()) { + case DECLARE: + processDeclare(child); + break; + case SELECT: + select = processSelect(child); + break; + case WHERE: + where = processWhere(child); + break; + default: + throw new HL7QueryParsingException("Found unexpected clause at root level: " + tree.getText()); + } + } + + this.whereEvaluator = where; + this.selections = select; + } + + private void processDeclare(final Tree declare) { + for (int i=0; i < declare.getChildCount(); i++) { + final Tree declarationTree = declare.getChild(i); + + final String identifier = declarationTree.getChild(0).getText(); + final Tree requiredOrOptionalTree = declarationTree.getChild(1); + final boolean required = requiredOrOptionalTree.getType() == REQUIRED; + + final String segmentName = declarationTree.getChild(2).getText(); + + final Declaration declaration = new Declaration() { + @Override + public String getAlias() { + return identifier; + } + + @Override + public boolean isRequired() { + return required; + } + + @Override + public Object getDeclaredValue(final HL7Message message) { + if ( message == null ) { + return null; + } + + return message.getSegments(segmentName); + } + }; + + declarations.add(declaration); + } + } + + private List processSelect(final Tree select) { + final List selections = new ArrayList<>(); + + for (int i=0; i < select.getChildCount(); i++) { + final Tree selectable = select.getChild(i); + + final String alias = getSelectedName(selectable); + final Evaluator selectionEvaluator = buildReferenceEvaluator(selectable); + final Selection selection = new Selection(selectionEvaluator, alias); + selections.add(selection); + } + + return selections; + } + + + private String getSelectedName(final Tree selectable) { + if ( selectable.getChildCount() == 0 ) { + return selectable.getText(); + } else if (selectable.getType() == DOT ) { + return getSelectedName(selectable.getChild(0)) + "." + getSelectedName(selectable.getChild(1)); + } else { + return selectable.getChild(selectable.getChildCount() - 1).getText(); + } + } + + + private BooleanEvaluator processWhere(final Tree where) { + return buildBooleanEvaluator(where.getChild(0)); + } + + + private Evaluator buildReferenceEvaluator(final Tree tree) { + switch (tree.getType()) { + case MESSAGE: + return new MessageEvaluator(); + case SEGMENT_NAME: + return new SegmentEvaluator(new StringLiteralEvaluator(tree.getText())); + case IDENTIFIER: + return new DeclaredReferenceEvaluator(new StringLiteralEvaluator(tree.getText())); + case DOT: + final Tree firstChild = tree.getChild(0); + final Tree secondChild = tree.getChild(1); + return new DotEvaluator(buildReferenceEvaluator(firstChild), buildIntegerEvaluator(secondChild)); + case STRING_LITERAL: + return new StringLiteralEvaluator(tree.getText()); + case NUMBER: + return new IntegerLiteralEvaluator(Integer.parseInt(tree.getText())); + default: + throw new HL7QueryParsingException("Failed to build evaluator for " + tree.getText()); + } + } + + + private IntegerEvaluator buildIntegerEvaluator(final Tree tree) { + switch (tree.getType()) { + case NUMBER: + return new IntegerLiteralEvaluator(Integer.parseInt(tree.getText())); + default: + throw new HL7QueryParsingException("Failed to build Integer Evaluator for " + tree.getText()); + } + } + + + private BooleanEvaluator buildBooleanEvaluator(final Tree tree) { + // TODO: add Date comparisons + // LT/GT/GE/GE should allow for dates based on Field's Type + // BETWEEN + // DATE('2015/01/01') + // DATE('2015/01/01 12:00:00') + // DATE('24 HOURS AGO') + // DATE('YESTERDAY') + + switch (tree.getType()) { + case EQUALS: + return new EqualsEvaluator(buildReferenceEvaluator(tree.getChild(0)), buildReferenceEvaluator(tree.getChild(1))); + case NOT_EQUALS: + return new NotEqualsEvaluator(buildReferenceEvaluator(tree.getChild(0)), buildReferenceEvaluator(tree.getChild(1))); + case GT: + return new GreaterThanEvaluator(buildReferenceEvaluator(tree.getChild(0)), buildReferenceEvaluator(tree.getChild(1))); + case LT: + return new LessThanEvaluator(buildReferenceEvaluator(tree.getChild(0)), buildReferenceEvaluator(tree.getChild(1))); + case GE: + return new GreaterThanOrEqualEvaluator(buildReferenceEvaluator(tree.getChild(0)), buildReferenceEvaluator(tree.getChild(1))); + case LE: + return new LessThanOrEqualEvaluator(buildReferenceEvaluator(tree.getChild(0)), buildReferenceEvaluator(tree.getChild(1))); + case NOT: + return new NotEvaluator(buildBooleanEvaluator(tree.getChild(0))); + case AND: + return new AndEvaluator(buildBooleanEvaluator(tree.getChild(0)), buildBooleanEvaluator(tree.getChild(1))); + case OR: + return new OrEvaluator(buildBooleanEvaluator(tree.getChild(0)), buildBooleanEvaluator(tree.getChild(1))); + case IS_NULL: + return new IsNullEvaluator(buildReferenceEvaluator(tree.getChild(0))); + case NOT_NULL: + return new NotNullEvaluator(buildReferenceEvaluator(tree.getChild(0))); + default: + throw new HL7QueryParsingException("Cannot build boolean evaluator for '" + tree.getText() + "'"); + } + } + + + Tree getTree() { + return tree; + } + + public String getQuery() { + return query; + } + + @Override + public String toString() { + return "HL7Query[" + query + "]"; + } + + public static HL7Query compile(final String query) { + try { + final CommonTokenStream lexerTokenStream = createTokenStream(query); + final HL7QueryParser parser = new HL7QueryParser(lexerTokenStream); + final Tree tree = (Tree) parser.query().getTree(); + + return new HL7Query(tree, query); + } catch (final HL7QueryParsingException e) { + throw e; + } catch (final Exception e) { + throw new HL7QueryParsingException(e); + } + } + + private static CommonTokenStream createTokenStream(final String expression) throws HL7QueryParsingException { + final CharStream input = new ANTLRStringStream(expression); + final HL7QueryLexer lexer = new HL7QueryLexer(input); + return new CommonTokenStream(lexer); + } + + public List> getReturnTypes() { + final List> returnTypes = new ArrayList<>(); + + for ( final Selection selection : selections ) { + returnTypes.add( selection.getEvaluator().getType() ); + } + + return returnTypes; + } + + @SuppressWarnings("unchecked") + public QueryResult evaluate(final HL7Message message) { + + int totalIterations = 1; + final LinkedHashMap> possibleValueMap = new LinkedHashMap<>(); + for ( final Declaration declaration : declarations ) { + final Object value = declaration.getDeclaredValue(message); + if ( value == null && declaration.isRequired() ) { + return new MissedResult(selections); + } + + final List possibleValues; + if ( value instanceof List ) { + possibleValues = (List) value; + } else if ( value instanceof Collection ) { + possibleValues = new ArrayList((Collection) value); + } else { + possibleValues = new ArrayList<>(1); + possibleValues.add(value); + } + + if ( possibleValues.isEmpty() ) { + return new MissedResult(selections); + } + + possibleValueMap.put(declaration.getAlias(), possibleValues); + totalIterations *= possibleValues.size(); + } + + final Set> resultSet = new HashSet<>(); + for (int i=0; i < totalIterations; i++) { + final Map aliasValues = assignAliases(possibleValueMap, i); + + aliasValues.put(Evaluator.MESSAGE_KEY, message); + if (whereEvaluator == null || Boolean.TRUE.equals(whereEvaluator.evaluate(aliasValues))) { + final Map resultMap = new HashMap<>(); + + for ( final Selection selection : selections ) { + final Object value = selection.getEvaluator().evaluate(aliasValues); + resultMap.put(selection.getName(), value); + } + + resultSet.add(resultMap); + } + } + +// for ( final Declaration declaration : declarations ) { +// final Object value = declaration.getDeclaredValue(message); +// if ( value == null && declaration.isRequired() ) { +// return new MissedResult(selections); +// } +// objectMap.put(declaration.getAlias(), value); +// } +// +// if (whereEvaluator == null || Boolean.TRUE.equals(whereEvaluator.evaluate(objectMap))) { +// for ( final Selection selection : selections ) { +// final Object value = selection.getEvaluator().evaluate(objectMap); +// resultMap.put(selection.getName(), value); +// } +// } else { +// return new MissedResult(selections); +// } + + return new StandardQueryResult(selections, resultSet); + } + + + // assigns one of the possible values to each alias, based on which iteration this is. + // require LinkedHashMap just to be very clear and explicit that the order of the Map MUST be guaranteed + // between multiple invocations of this method. + // package protected for testing visibility +// static Map assignAliases(final LinkedHashMap> possibleValues, final int iteration) { +// final Map aliasMap = new HashMap<>(); +// +// int aliasIndex = possibleValues.size() - 1; +// for ( final Map.Entry> entry : possibleValues.entrySet() ) { +// final String alias = entry.getKey(); +// final List validValues = entry.getValue(); +// +// final int validValueIndex; +// if (aliasIndex > 0) { +// validValueIndex = iteration / aliasIndex; +// } else { +// validValueIndex = iteration; +// } +// +// final Object obj = validValues.get(validValueIndex % validValues.size()); +// aliasMap.put(alias, obj); +// +// aliasIndex--; +// } +// +// return aliasMap; +// } +// + + static Map assignAliases(final LinkedHashMap> possibleValues, final int iteration) { + final Map aliasMap = new HashMap<>(); + + int divisor = 1; + for ( final Map.Entry> entry : possibleValues.entrySet() ) { + final String alias = entry.getKey(); + final List validValues = entry.getValue(); + + final int idx = (iteration / divisor) % validValues.size(); + final Object obj = validValues.get(idx); + aliasMap.put(alias, obj); + + divisor *= validValues.size(); + } + + return aliasMap; + } + + public String toTreeString() { + final StringBuilder sb = new StringBuilder(); + toTreeString(tree, sb, 0); + return sb.toString(); + } + + private void toTreeString(final Tree tree, final StringBuilder sb, final int indentLevel) { + final String nodeName = tree.getText(); + for (int i=0; i < indentLevel; i++) { + sb.append(" "); + } + sb.append(nodeName); + sb.append("\n"); + + for (int i=0; i < tree.getChildCount(); i++) { + final Tree child = tree.getChild(i); + toTreeString(child, sb, indentLevel + 2); + } + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/QueryResult.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/QueryResult.java new file mode 100644 index 0000000000..b198bc7ffb --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/QueryResult.java @@ -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.hl7.query; + +import java.util.List; + +public interface QueryResult { + boolean isMatch(); + + List getLabels(); + + int getHitCount(); + + ResultHit nextHit(); +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/ResultHit.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/ResultHit.java new file mode 100644 index 0000000000..ee97e5d94c --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/ResultHit.java @@ -0,0 +1,25 @@ +/* + * 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.hl7.query; + +import java.util.Map; + +public interface ResultHit { + Object getValue(String label); + + Map getSelectedValues(); +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/Selection.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/Selection.java new file mode 100644 index 0000000000..36a181f539 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/Selection.java @@ -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.hl7.query; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public class Selection { + private final Evaluator evaluator; + private final String name; + + public Selection(final Evaluator evaluator, final String name) { + this.evaluator = evaluator; + this.name = name; + } + + public String getName() { + return name; + } + + public Evaluator getEvaluator() { + return evaluator; + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/BooleanEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/BooleanEvaluator.java new file mode 100644 index 0000000000..fdd807ed3e --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/BooleanEvaluator.java @@ -0,0 +1,24 @@ +/* + * 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.hl7.query.evaluator; + +public abstract class BooleanEvaluator implements Evaluator { + + public Class getType() { + return Boolean.class; + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/Evaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/Evaluator.java new file mode 100644 index 0000000000..d86c30e06d --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/Evaluator.java @@ -0,0 +1,27 @@ +/* + * 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.hl7.query.evaluator; + +import java.util.Map; + +public interface Evaluator { + public static final String MESSAGE_KEY = "message"; + + T evaluate(Map objectMap); + + Class getType(); +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/IntegerEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/IntegerEvaluator.java new file mode 100644 index 0000000000..6afa9ed46c --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/IntegerEvaluator.java @@ -0,0 +1,26 @@ +/* + * 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.hl7.query.evaluator; + + +public abstract class IntegerEvaluator implements Evaluator { + + public Class getType() { + return Integer.class; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/StringEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/StringEvaluator.java new file mode 100644 index 0000000000..5f73649318 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/StringEvaluator.java @@ -0,0 +1,25 @@ +/* + * 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.hl7.query.evaluator; + +public abstract class StringEvaluator implements Evaluator { + + public Class getType() { + return String.class; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/AbstractComparisonEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/AbstractComparisonEvaluator.java new file mode 100644 index 0000000000..a7fa1b7a29 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/AbstractComparisonEvaluator.java @@ -0,0 +1,106 @@ +/* + * 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.hl7.query.evaluator.comparison; + +import java.util.Collection; +import java.util.Map; + +import org.apache.nifi.hl7.model.HL7Field; +import org.apache.nifi.hl7.query.evaluator.BooleanEvaluator; +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public abstract class AbstractComparisonEvaluator extends BooleanEvaluator { + private final Evaluator lhs; + private final Evaluator rhs; + + public AbstractComparisonEvaluator(final Evaluator lhs, final Evaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + public final Boolean evaluate(final Map objectMap) { + final Object lhsValue = lhs.evaluate(objectMap); + if ( lhsValue == null ) { + return false; + } + + final Object rhsValue = rhs.evaluate(objectMap); + if ( rhsValue == null ) { + return false; + } + + return compareRaw(lhsValue, rhsValue); + } + + + private Boolean compareRaw(Object lhsValue, Object rhsValue) { + if ( lhsValue == null || rhsValue == null ) { + return false; + } + + if ( lhsValue instanceof HL7Field ) { + lhsValue = ((HL7Field) lhsValue).getValue(); + } + + if ( rhsValue instanceof HL7Field ) { + rhsValue = ((HL7Field) rhsValue).getValue(); + } + + if ( lhsValue == null || rhsValue == null ) { + return false; + } + + // both are collections, and compare(lhsValue, rhsValue) is false. + // this would be the case, for instance, if we compared field 1 of one segment to + // a field in another segment, and both fields had components. + if ( lhsValue instanceof Collection && rhsValue instanceof Collection ) { + return false; + } + + // if one side is a collection but the other is not, check if any element in that + // collection compares to the other element in a way that satisfies the condition. + // this would happen, for instance, if we check Segment1.Field5 = 'X' and field 5 repeats + // with a value "A~B~C~X~Y~Z"; in this case we do want to consider Field 5 = X as true. + if ( lhsValue instanceof Collection ) { + for ( final Object lhsObject : (Collection) lhsValue ) { + if ( compareRaw(lhsObject, rhsValue) ) { + return true; + } + } + + return false; + } + + if ( rhsValue instanceof Collection ) { + for ( final Object rhsObject : (Collection) rhsValue ) { + if ( compareRaw(rhsObject, lhsValue) ) { + return true; + } + } + + return false; + } + + if ( lhsValue != null && rhsValue != null && compare(lhsValue, rhsValue) ) { + return true; + } + + return false; + } + + protected abstract boolean compare(Object lhs, Object rhs); +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/AbstractNumericComparison.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/AbstractNumericComparison.java new file mode 100644 index 0000000000..2529c4914b --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/AbstractNumericComparison.java @@ -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.hl7.query.evaluator.comparison; + +import java.util.regex.Pattern; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public abstract class AbstractNumericComparison extends AbstractComparisonEvaluator { + private static final Pattern NUMERIC_PATTERN = Pattern.compile("\\d+(\\.\\d+)?"); + + public AbstractNumericComparison(final Evaluator lhs, final Evaluator rhs) { + super(lhs, rhs); + } + + @Override + protected final boolean compare(final Object lhs, final Object rhs) { + final Double lhsDouble = toDouble(lhs); + if ( lhsDouble == null ) { + return false; + } + + final Double rhsDouble = toDouble(rhs); + if ( rhsDouble == null ) { + return false; + } + + return compareNumbers(lhsDouble, rhsDouble); + } + + private Double toDouble(final Object value) { + if ( value == null ) { + return null; + } + + if ( value instanceof Double ) { + return (Double) value; + } + if ( value instanceof Number ) { + return ((Number) value).doubleValue(); + } + + if ( value instanceof String ) { + if ( NUMERIC_PATTERN.matcher((String) value).matches() ) { + return Double.parseDouble((String) value); + } + } + + return null; + } + + protected abstract boolean compareNumbers(final Double lhs, final Double rhs); +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/EqualsEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/EqualsEvaluator.java new file mode 100644 index 0000000000..7ee2f8707b --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/EqualsEvaluator.java @@ -0,0 +1,32 @@ +/* + * 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.hl7.query.evaluator.comparison; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public class EqualsEvaluator extends AbstractComparisonEvaluator { + + public EqualsEvaluator(final Evaluator lhs, final Evaluator rhs) { + super(lhs, rhs); + } + + @Override + protected boolean compare(final Object lhs, final Object rhs) { + return lhs != null && rhs != null && ((lhs == rhs) || (lhs.equals(rhs)) || lhs.toString().equals(rhs.toString())); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/GreaterThanEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/GreaterThanEvaluator.java new file mode 100644 index 0000000000..bf8596ecc9 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/GreaterThanEvaluator.java @@ -0,0 +1,34 @@ +/* + * 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.hl7.query.evaluator.comparison; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; + + +public class GreaterThanEvaluator extends AbstractNumericComparison { + + public GreaterThanEvaluator(final Evaluator lhs, final Evaluator rhs) { + super(lhs, rhs); + } + + @Override + protected boolean compareNumbers(final Double lhs, final Double rhs) { + return lhs > rhs; + } + + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/GreaterThanOrEqualEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/GreaterThanOrEqualEvaluator.java new file mode 100644 index 0000000000..69115a3855 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/GreaterThanOrEqualEvaluator.java @@ -0,0 +1,34 @@ +/* + * 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.hl7.query.evaluator.comparison; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; + + +public class GreaterThanOrEqualEvaluator extends AbstractNumericComparison { + + public GreaterThanOrEqualEvaluator(final Evaluator lhs, final Evaluator rhs) { + super(lhs, rhs); + } + + @Override + protected boolean compareNumbers(final Double lhs, final Double rhs) { + return lhs >= rhs; + } + + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/IsNullEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/IsNullEvaluator.java new file mode 100644 index 0000000000..69d481e289 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/IsNullEvaluator.java @@ -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.hl7.query.evaluator.comparison; + +import java.util.Collection; +import java.util.Map; + +import org.apache.nifi.hl7.model.HL7Component; +import org.apache.nifi.hl7.query.evaluator.BooleanEvaluator; +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public class IsNullEvaluator extends BooleanEvaluator { + private final Evaluator subjectEvaluator; + + public IsNullEvaluator(final Evaluator subjectEvaluator) { + this.subjectEvaluator = subjectEvaluator; + } + + @Override + public Boolean evaluate(final Map objectMap) { + Object subjectValue = subjectEvaluator.evaluate(objectMap); + if ( subjectValue == null ) { + return true; + } + + return isNull(subjectValue); + } + + private boolean isNull(Object subjectValue) { + if ( subjectValue == null ) { + return true; + } + + if ( subjectValue instanceof HL7Component ) { + subjectValue = ((HL7Component) subjectValue).getValue(); + } + + if ( subjectValue instanceof Collection ) { + final Collection collection = (Collection) subjectValue; + if ( collection.isEmpty() ) { + return true; + } + + for ( final Object obj : collection ) { + if ( !isNull(obj) ) { + return false; + } + } + + return true; + } + + return subjectValue == null; + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/LessThanEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/LessThanEvaluator.java new file mode 100644 index 0000000000..891d5e4c3b --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/LessThanEvaluator.java @@ -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.hl7.query.evaluator.comparison; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public class LessThanEvaluator extends AbstractNumericComparison { + public LessThanEvaluator(final Evaluator lhs, final Evaluator rhs) { + super(lhs, rhs); + } + + @Override + protected boolean compareNumbers(final Double lhs, final Double rhs) { + return lhs < rhs; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/LessThanOrEqualEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/LessThanOrEqualEvaluator.java new file mode 100644 index 0000000000..c6fb0974ed --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/LessThanOrEqualEvaluator.java @@ -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.hl7.query.evaluator.comparison; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public class LessThanOrEqualEvaluator extends AbstractNumericComparison { + public LessThanOrEqualEvaluator(final Evaluator lhs, final Evaluator rhs) { + super(lhs, rhs); + } + + @Override + protected boolean compareNumbers(final Double lhs, final Double rhs) { + return lhs <= rhs; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotEqualsEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotEqualsEvaluator.java new file mode 100644 index 0000000000..b7c1ce28a0 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotEqualsEvaluator.java @@ -0,0 +1,32 @@ +/* + * 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.hl7.query.evaluator.comparison; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public class NotEqualsEvaluator extends AbstractComparisonEvaluator { + + public NotEqualsEvaluator(final Evaluator lhs, final Evaluator rhs) { + super(lhs, rhs); + } + + @Override + protected boolean compare(final Object lhs, final Object rhs) { + return lhs != null && rhs != null && lhs != rhs && !lhs.equals(rhs) && !lhs.toString().equals(rhs.toString()); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotEvaluator.java new file mode 100644 index 0000000000..58888d9aea --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotEvaluator.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.hl7.query.evaluator.comparison; + +import java.util.Map; + +import org.apache.nifi.hl7.query.evaluator.BooleanEvaluator; + +public class NotEvaluator extends BooleanEvaluator { + private final BooleanEvaluator subjectEvaluator; + + public NotEvaluator(final BooleanEvaluator subjectEvaluator) { + this.subjectEvaluator = subjectEvaluator; + } + + @Override + public Boolean evaluate(final Map objectMap) { + final Boolean subjectValue = subjectEvaluator.evaluate(objectMap); + return (subjectValue == null || Boolean.TRUE.equals(subjectValue)); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotNullEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotNullEvaluator.java new file mode 100644 index 0000000000..a764fef61c --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/comparison/NotNullEvaluator.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.hl7.query.evaluator.comparison; + +import java.util.Collection; +import java.util.Map; + +import org.apache.nifi.hl7.model.HL7Component; +import org.apache.nifi.hl7.query.evaluator.BooleanEvaluator; +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public class NotNullEvaluator extends BooleanEvaluator { + private final Evaluator subjectEvaluator; + + public NotNullEvaluator(final Evaluator subjectEvaluator) { + this.subjectEvaluator = subjectEvaluator; + } + + @Override + public Boolean evaluate(final Map objectMap) { + Object subjectValue = subjectEvaluator.evaluate(objectMap); + if ( subjectValue == null ) { + return false; + } + + return isNotNull(subjectValue); + } + + private boolean isNotNull(Object subjectValue) { + if ( subjectValue instanceof HL7Component ) { + subjectValue = ((HL7Component) subjectValue).getValue(); + } + + if ( subjectValue instanceof Collection ) { + final Collection collection = (Collection) subjectValue; + if ( collection.isEmpty() ) { + return false; + } + + for ( final Object obj : collection ) { + if ( isNotNull(obj) ) { + return true; + } + } + + return false; + } + + return subjectValue != null; + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/literal/IntegerLiteralEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/literal/IntegerLiteralEvaluator.java new file mode 100644 index 0000000000..c6ff6e479d --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/literal/IntegerLiteralEvaluator.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.hl7.query.evaluator.literal; + +import java.util.Map; + +import org.apache.nifi.hl7.query.evaluator.IntegerEvaluator; + +public class IntegerLiteralEvaluator extends IntegerEvaluator { + private final Integer value; + + public IntegerLiteralEvaluator(final Integer value) { + this.value = value; + } + + + @Override + public Integer evaluate(final Map objectMap) { + return value; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/literal/StringLiteralEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/literal/StringLiteralEvaluator.java new file mode 100644 index 0000000000..3b296117c9 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/literal/StringLiteralEvaluator.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.hl7.query.evaluator.literal; + +import java.util.Map; + +import org.apache.nifi.hl7.query.evaluator.StringEvaluator; + +public class StringLiteralEvaluator extends StringEvaluator { + private final String value; + + public StringLiteralEvaluator(final String value) { + this.value = value; + } + + @Override + public String evaluate(final Map objectMap) { + return value; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/logic/AndEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/logic/AndEvaluator.java new file mode 100644 index 0000000000..21f596e3dc --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/logic/AndEvaluator.java @@ -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.hl7.query.evaluator.logic; + +import java.util.Map; + +import org.apache.nifi.hl7.query.evaluator.BooleanEvaluator; + +public class AndEvaluator extends BooleanEvaluator { + private final BooleanEvaluator lhs; + private final BooleanEvaluator rhs; + + public AndEvaluator(final BooleanEvaluator lhs, final BooleanEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Boolean evaluate(final Map objectMap) { + final Boolean lhsValue = lhs.evaluate(objectMap); + if ( lhsValue == null || Boolean.FALSE.equals(lhsValue) ) { + return false; + } + + final Boolean rhsValue = rhs.evaluate(objectMap); + return (rhsValue != null && Boolean.TRUE.equals(rhsValue)); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/logic/OrEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/logic/OrEvaluator.java new file mode 100644 index 0000000000..d090946da0 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/logic/OrEvaluator.java @@ -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.hl7.query.evaluator.logic; + +import java.util.Map; + +import org.apache.nifi.hl7.query.evaluator.BooleanEvaluator; + +public class OrEvaluator extends BooleanEvaluator { + private final BooleanEvaluator lhs; + private final BooleanEvaluator rhs; + + public OrEvaluator(final BooleanEvaluator lhs, final BooleanEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Boolean evaluate(final Map objectMap) { + final Boolean lhsValue = lhs.evaluate(objectMap); + if ( lhsValue != null && Boolean.TRUE.equals(lhsValue) ) { + return true; + } + + final Boolean rhsValue = rhs.evaluate(objectMap); + return (rhsValue != null && Boolean.TRUE.equals(rhsValue)); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/DeclaredReferenceEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/DeclaredReferenceEvaluator.java new file mode 100644 index 0000000000..6afb8d7ec1 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/DeclaredReferenceEvaluator.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.hl7.query.evaluator.message; + +import java.util.Map; + +import org.apache.nifi.hl7.query.evaluator.Evaluator; +import org.apache.nifi.hl7.query.evaluator.StringEvaluator; + +public class DeclaredReferenceEvaluator implements Evaluator { + private final StringEvaluator referenceNameEvaluator; + + public DeclaredReferenceEvaluator(final StringEvaluator referenceNameEvaluator) { + this.referenceNameEvaluator = referenceNameEvaluator; + } + + @Override + public Object evaluate(final Map objectMap) { + final String referenceName = referenceNameEvaluator.evaluate(objectMap); + return objectMap.get(referenceName); + } + + @Override + public Class getType() { + return Object.class; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/DotEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/DotEvaluator.java new file mode 100644 index 0000000000..c5fbf41c54 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/DotEvaluator.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.hl7.query.evaluator.message; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.hl7.model.HL7Component; +import org.apache.nifi.hl7.model.HL7Message; +import org.apache.nifi.hl7.model.HL7Segment; +import org.apache.nifi.hl7.query.evaluator.Evaluator; +import org.apache.nifi.hl7.query.evaluator.IntegerEvaluator; + +public class DotEvaluator implements Evaluator { + private final Evaluator lhs; + private final IntegerEvaluator rhs; + + public DotEvaluator(final Evaluator lhs, final IntegerEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Object evaluate(final Map objectMap) { + final Object lhsValue = this.lhs.evaluate(objectMap); + final Integer rhsValue = this.rhs.evaluate(objectMap); + + if ( lhsValue == null || rhsValue == null ) { + return null; + } + + final List results = new ArrayList<>(); + if ( lhsValue instanceof Collection ) { + final Collection lhsCollection = (Collection) lhsValue; + for ( final Object obj : lhsCollection ) { + final Object val = getValue(obj, rhsValue); + results.add(val); + } + } else { + final Object val = getValue(lhsValue, rhsValue); + return val; + } + + return results; + } + + private Object getValue(final Object lhsValue, final int rhsValue) { + final List list; + if ( lhsValue instanceof HL7Message ) { + list = ((HL7Message) lhsValue).getSegments(); + } else if ( lhsValue instanceof HL7Segment ) { + list = ((HL7Segment) lhsValue).getFields(); + } else if ( lhsValue instanceof HL7Component ) { + list = ((HL7Component) lhsValue).getComponents(); + } else { + return null; + } + + if ( rhsValue > list.size() ) { + return null; + } + + // convert from 0-based to 1-based + return list.get(rhsValue - 1); + } + + @Override + public Class getType() { + return Object.class; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/FieldEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/FieldEvaluator.java new file mode 100644 index 0000000000..869c2d0783 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/FieldEvaluator.java @@ -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.hl7.query.evaluator.message; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.hl7.model.HL7Field; +import org.apache.nifi.hl7.model.HL7Segment; +import org.apache.nifi.hl7.query.evaluator.Evaluator; +import org.apache.nifi.hl7.query.evaluator.IntegerEvaluator; + +@SuppressWarnings("rawtypes") +public class FieldEvaluator implements Evaluator { + private final SegmentEvaluator segmentEvaluator; + private final IntegerEvaluator indexEvaluator; + + public FieldEvaluator(final SegmentEvaluator segmentEvaluator, final IntegerEvaluator indexEvaluator) { + this.segmentEvaluator = segmentEvaluator; + this.indexEvaluator = indexEvaluator; + } + + public List evaluate(final Map objectMap) { + final List segments = segmentEvaluator.evaluate(objectMap); + if ( segments == null ) { + return Collections.emptyList(); + } + + final Integer index = indexEvaluator.evaluate(objectMap); + if ( index == null ) { + return Collections.emptyList(); + } + + final List fields = new ArrayList<>(); + for ( final HL7Segment segment : segments ) { + final List segmentFields = segment.getFields(); + if ( segmentFields.size() <= index ) { + continue; + } + + fields.add(segmentFields.get(index)); + } + + return fields; + } + + public Class getType() { + return List.class; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/MessageEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/MessageEvaluator.java new file mode 100644 index 0000000000..5e08961954 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/MessageEvaluator.java @@ -0,0 +1,34 @@ +/* + * 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.hl7.query.evaluator.message; + +import java.util.Map; + +import org.apache.nifi.hl7.model.HL7Message; +import org.apache.nifi.hl7.query.evaluator.Evaluator; + +public class MessageEvaluator implements Evaluator { + + public HL7Message evaluate(final Map objectMap) { + return (HL7Message) objectMap.get(Evaluator.MESSAGE_KEY); + } + + public Class getType() { + return HL7Message.class; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/SegmentEvaluator.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/SegmentEvaluator.java new file mode 100644 index 0000000000..1b9782d0e6 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/evaluator/message/SegmentEvaluator.java @@ -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.hl7.query.evaluator.message; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.hl7.model.HL7Message; +import org.apache.nifi.hl7.model.HL7Segment; +import org.apache.nifi.hl7.query.evaluator.Evaluator; +import org.apache.nifi.hl7.query.evaluator.StringEvaluator; + +@SuppressWarnings("rawtypes") +public class SegmentEvaluator implements Evaluator { + private final StringEvaluator segmentTypeEvaluator; + + public SegmentEvaluator(final StringEvaluator segmentTypeEvaluator) { + this.segmentTypeEvaluator = segmentTypeEvaluator; + } + + public List evaluate(final Map objectMap) { + final String segmentType = segmentTypeEvaluator.evaluate(objectMap); + if ( segmentType == null ) { + return Collections.emptyList(); + } + + final HL7Message message = (HL7Message) objectMap.get(Evaluator.MESSAGE_KEY); + final List segments = message.getSegments(segmentType); + return (segments == null) ? Collections.emptyList() : segments; + } + + public Class getType() { + return List.class; + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/exception/HL7QueryParsingException.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/exception/HL7QueryParsingException.java new file mode 100644 index 0000000000..998f3bc87c --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/exception/HL7QueryParsingException.java @@ -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.hl7.query.exception; + +public class HL7QueryParsingException extends RuntimeException { + private static final long serialVersionUID = 1L; + + public HL7QueryParsingException() { + super(); + } + + public HL7QueryParsingException(final Throwable cause) { + super(cause); + } + + public HL7QueryParsingException(final String message) { + super(message); + } + + public HL7QueryParsingException(final String message, final Throwable cause) { + super(message, cause); + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/MissedResult.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/MissedResult.java new file mode 100644 index 0000000000..a6b36c8621 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/MissedResult.java @@ -0,0 +1,56 @@ +/* + * 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.hl7.query.result; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.nifi.hl7.query.QueryResult; +import org.apache.nifi.hl7.query.ResultHit; +import org.apache.nifi.hl7.query.Selection; + +public class MissedResult implements QueryResult { + private final List selections; + + public MissedResult(final List selections) { + this.selections = selections; + } + + @Override + public List getLabels() { + final List labels = new ArrayList<>(); + for ( final Selection selection : selections ) { + labels.add(selection.getName()); + } + return labels; + } + + @Override + public boolean isMatch() { + return false; + } + + @Override + public ResultHit nextHit() { + return null; + } + + @Override + public int getHitCount() { + return 0; + } +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/StandardQueryResult.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/StandardQueryResult.java new file mode 100644 index 0000000000..fbc16cab4c --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/StandardQueryResult.java @@ -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.hl7.query.result; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.nifi.hl7.query.QueryResult; +import org.apache.nifi.hl7.query.ResultHit; +import org.apache.nifi.hl7.query.Selection; + +public class StandardQueryResult implements QueryResult { + private final List selections; + private final Set> hits; + private final Iterator> hitIterator; + + public StandardQueryResult(final List selections, final Set> hits) { + this.selections = selections; + this.hits = hits; + + hitIterator = hits.iterator(); + } + + @Override + public boolean isMatch() { + return !hits.isEmpty(); + } + + @Override + public List getLabels() { + final List labels = new ArrayList<>(); + for ( final Selection selection : selections ) { + labels.add(selection.getName()); + } + return labels; + } + + @Override + public int getHitCount() { + return hits.size(); + } + + @Override + public ResultHit nextHit() { + if ( hitIterator.hasNext() ) { + return new StandardResultHit(hitIterator.next()); + } else { + return null; + } + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/StandardResultHit.java b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/StandardResultHit.java new file mode 100644 index 0000000000..944e998173 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/main/java/org/apache/nifi/hl7/query/result/StandardResultHit.java @@ -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.hl7.query.result; + +import java.util.Collections; +import java.util.Map; + +import org.apache.nifi.hl7.query.ResultHit; + +public class StandardResultHit implements ResultHit { + private final Map values; + + public StandardResultHit(final Map values) { + this.values = values; + } + + @Override + public Object getValue(final String label) { + return values.get(label); + } + + @Override + public Map getSelectedValues() { + return Collections.unmodifiableMap(values); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/java/org/apache/nifi/hl7/query/TestHL7Query.java b/nifi/nifi-commons/nifi-hl7-query-language/src/test/java/org/apache/nifi/hl7/query/TestHL7Query.java new file mode 100644 index 0000000000..596440c380 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/test/java/org/apache/nifi/hl7/query/TestHL7Query.java @@ -0,0 +1,310 @@ +/* + * 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.hl7.query; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.hl7.hapi.HapiMessage; +import org.apache.nifi.hl7.model.HL7Field; +import org.apache.nifi.hl7.model.HL7Message; +import org.junit.Test; + +import ca.uhn.hl7v2.DefaultHapiContext; +import ca.uhn.hl7v2.HL7Exception; +import ca.uhn.hl7v2.HapiContext; +import ca.uhn.hl7v2.model.Message; +import ca.uhn.hl7v2.parser.PipeParser; +import ca.uhn.hl7v2.validation.impl.ValidationContextFactory; + +@SuppressWarnings("resource") +public class TestHL7Query { + + @Test + public void testAssignAliases() { + final LinkedHashMap> possibleValueMap = new LinkedHashMap<>(); + + final List valuesA = new ArrayList<>(); + valuesA.add("a"); + valuesA.add("b"); + valuesA.add("c"); + + final List valuesB = new ArrayList<>(); + valuesB.add("d"); + + final List valuesC = new ArrayList<>(); + valuesC.add("e"); + valuesC.add("f"); + + final List valuesD = new ArrayList<>(); + valuesD.add("g"); + valuesD.add("h"); + + possibleValueMap.put("A", valuesA); + possibleValueMap.put("B", valuesB); + possibleValueMap.put("C", valuesC); + possibleValueMap.put("D", valuesD); + + for (int i=0; i < valuesA.size() * valuesB.size() * valuesC.size() * valuesD.size(); i++) { + System.out.println(i + " : " + HL7Query.assignAliases(possibleValueMap, i)); + } + + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 0), "a", "d", "e", "g"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 1), "b", "d", "e", "g"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 2), "c", "d", "e", "g"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 3), "a", "d", "f", "g"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 4), "b", "d", "f", "g"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 5), "c", "d", "f", "g"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 6), "a", "d", "e", "h"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 7), "b", "d", "e", "h"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 8), "c", "d", "e", "h"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 9), "a", "d", "f", "h"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 10), "b", "d", "f", "h"); + verifyAssignments(HL7Query.assignAliases(possibleValueMap, 11), "c", "d", "f", "h"); + } + + private void verifyAssignments(final Map map, final String a, final String b, final String c, final String d) { + assertEquals(a, map.get("A")); + assertEquals(b, map.get("B")); + assertEquals(c, map.get("C")); + assertEquals(d, map.get("D")); + } + + @Test + public void testSelectMessage() throws HL7Exception, IOException { + final HL7Query query = HL7Query.compile("SELECT MESSAGE"); + final HL7Message msg = createMessage(new File("src/test/resources/hypoglycemia")); + final QueryResult result = query.evaluate(msg); + assertTrue(result.isMatch()); + final List labels = result.getLabels(); + assertEquals(1, labels.size()); + assertEquals("MESSAGE", labels.get(0)); + + assertEquals(1, result.getHitCount()); + assertEquals(msg, result.nextHit().getValue("MESSAGE")); + } + + @Test + @SuppressWarnings({ "unchecked", "rawtypes" }) + public void testSelectField() throws HL7Exception, IOException { + final HL7Query query = HL7Query.compile("SELECT PID.5"); + final HL7Message msg = createMessage(new File("src/test/resources/hypoglycemia")); + final QueryResult result = query.evaluate(msg); + assertTrue(result.isMatch()); + final List labels = result.getLabels(); + assertEquals(1, labels.size()); + assertEquals(1, result.getHitCount()); + + final Object names = result.nextHit().getValue("PID.5"); + assertTrue(names instanceof List); + final List nameList = (List) names; + assertEquals(1, nameList.size()); + final HL7Field nameField = (HL7Field) nameList.get(0); + assertEquals("SMITH^JOHN", nameField.getValue()); + } + + @Test + public void testSelectAbnormalTestResult() throws HL7Exception, IOException { + final String query = "DECLARE result AS REQUIRED OBX SELECT result WHERE result.7 != 'N' AND result.1 = 1"; + + final HL7Query hl7Query = HL7Query.compile(query); + final QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + } + + + @Test + public void testFieldEqualsString() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.7 = 'L'"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.7 = 'H'"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + } + + @Test + public void testLessThan() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 < 600"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 < 59"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + } + + @Test + public void testCompareTwoFields() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 < result.6.2"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE NOT(result.4 > result.6.3)"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + } + + @Test + public void testLessThanOrEqual() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 <= 59"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 <= 600"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 <= 58"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + } + + @Test + public void testGreaterThanOrEqual() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 >= 59"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 >= 6"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 >= 580"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + } + + @Test + public void testGreaterThan() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 > 58"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 > 6"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.4 > 580"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + } + + + @Test + public void testDistinctValuesReturned() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result1 AS REQUIRED OBX, result2 AS REQUIRED OBX SELECT MESSAGE WHERE result1.7 = 'L' OR result2.7 != 'H'"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + assertEquals(1, result.getHitCount()); + } + + @Test + public void testAndWithParens() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.7 = 'L' AND result.3.1 = 'GLU'"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.7 = 'L' AND result.3.1 = 'GLU'"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hyperglycemia"))); + assertFalse( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.7 = 'H' AND result.3.1 = 'GLU'"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.7 = 'H' AND result.3.1 = 'GLU'"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hyperglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE (result.7 = 'H') AND (result.3.1 = 'GLU')"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hyperglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE ((result.7 = 'H') AND (result.3.1 = 'GLU'))"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hyperglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE (( ((result.7 = 'H')) AND ( ((result.3.1 = 'GLU')) )))"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hyperglycemia"))); + assertTrue( result.isMatch() ); + + } + + + @Test + public void testIsNull() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.999 IS NULL"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.1 IS NULL"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + + hl7Query = HL7Query.compile("SELECT MESSAGE WHERE ZZZ IS NULL"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("SELECT MESSAGE WHERE OBX IS NULL"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + } + + + @Test + public void testNotNull() throws HL7Exception, IOException { + HL7Query hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.999 NOT NULL"); + QueryResult result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + + hl7Query = HL7Query.compile("DECLARE result AS REQUIRED OBX SELECT MESSAGE WHERE result.1 NOT NULL"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + + hl7Query = HL7Query.compile("SELECT MESSAGE WHERE ZZZ NOT NULL"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertFalse( result.isMatch() ); + + hl7Query = HL7Query.compile("SELECT MESSAGE WHERE OBX NOT NULL"); + result = hl7Query.evaluate(createMessage(new File("src/test/resources/hypoglycemia"))); + assertTrue( result.isMatch() ); + } + + private HL7Message createMessage(final File file) throws HL7Exception, IOException { + final byte[] bytes = Files.readAllBytes(file.toPath()); + final String msgText = new String(bytes, "UTF-8"); + + final HapiContext hapiContext = new DefaultHapiContext(); + hapiContext.setValidationContext(ValidationContextFactory.noValidation()); + + final PipeParser parser = hapiContext.getPipeParser(); + final Message message = parser.parse(msgText); + return new HapiMessage(message); + } + +} diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hyperglycemia b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hyperglycemia new file mode 100644 index 0000000000..cae413b7d1 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hyperglycemia @@ -0,0 +1,5 @@ +MSH|^~\&|XXXXXX||HealthOrg01||||ORU^R01|Q1111111111111111111|P|2.3| +PID|||000000001||SMITH^JOHN||19700101|M||||||||||999999999999|123456789| +PD1||||1234567890^LAST^FIRST^M^^^^^NPI| +OBR|1|341856649^HNAM_ORDERID|000000000000000000|648088^Basic Metabolic Panel|||20150101000100|||||||||1620^Johnson^John^R||||||20150101000100|||M|||||||||||20150101000100| +OBX|1|NM|GLU^Glucose Lvl|159|mg/dL|65-99^65^99|H|||F|||20150101000100| \ No newline at end of file diff --git a/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hypoglycemia b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hypoglycemia new file mode 100644 index 0000000000..764ffcc623 --- /dev/null +++ b/nifi/nifi-commons/nifi-hl7-query-language/src/test/resources/hypoglycemia @@ -0,0 +1,5 @@ +MSH|^~\&|XXXXXX||HealthOrg01||||ORU^R01|Q1111111111111111111|P|2.3| +PID|||000000001||SMITH^JOHN||19700101|M||||||||||999999999999|123456789| +PD1||||1234567890^LAST^FIRST^M^^^^^NPI| +OBR|1|341856649^HNAM_ORDERID|000000000000000000|648088^Basic Metabolic Panel|||20150101000100|||||||||1620^Johnson^John^R||||||20150101000100|||M|||||||||||20150101000100| +OBX|1|NM|GLU^Glucose Lvl|59|mg/dL|65-99^65^99|L|||F|||20150101000100| \ No newline at end of file diff --git a/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java b/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java index a22e7bb249..d1621eda43 100644 --- a/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java +++ b/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java @@ -236,7 +236,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } try { @@ -254,7 +254,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } try { @@ -271,7 +271,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } if (input == null) { @@ -289,7 +289,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } if (input == null) { @@ -319,7 +319,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } try { @@ -347,7 +347,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } final ValidationResult vr = DATA_SIZE_VALIDATOR.validate(subject, input, context); @@ -372,7 +372,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } final boolean matches = pattern.matcher(input).matches(); @@ -457,7 +457,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } String reason = null; @@ -503,7 +503,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } if (input == null) { @@ -628,7 +628,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } final ControllerService svc = context.getControllerServiceLookup().getControllerService(input); diff --git a/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java index 87a82d40eb..3b427a77ae 100644 --- a/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java +++ b/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java @@ -42,8 +42,6 @@ public class NiFiProperties extends Properties { public static final String PROPERTIES_FILE_PATH = "nifi.properties.file.path"; public static final String FLOW_CONFIGURATION_FILE = "nifi.flow.configuration.file"; public static final String FLOW_CONFIGURATION_ARCHIVE_FILE = "nifi.flow.configuration.archive.file"; - public static final String TASK_CONFIGURATION_FILE = "nifi.reporting.task.configuration.file"; - public static final String SERVICE_CONFIGURATION_FILE = "nifi.controller.service.configuration.file"; public static final String AUTHORITY_PROVIDER_CONFIGURATION_FILE = "nifi.authority.provider.configuration.file"; public static final String REPOSITORY_DATABASE_DIRECTORY = "nifi.database.directory"; public static final String RESTORE_DIRECTORY = "nifi.restore.directory"; diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java index e80f3284f5..1b5412cd01 100644 --- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java @@ -88,6 +88,7 @@ import org.apache.nifi.web.api.dto.ControllerDTO; import org.apache.nifi.web.api.dto.PortDTO; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.slf4j.helpers.MessageFormatter; public class EndpointConnectionPool { public static final long PEER_REFRESH_PERIOD = 60000L; @@ -202,6 +203,28 @@ public class EndpointConnectionPool { }, 5, 5, TimeUnit.SECONDS); } + void warn(final String msg, final Object... args) { + logger.warn(msg, args); + if ( eventReporter != null ) { + eventReporter.reportEvent(Severity.WARNING, "Site-to-Site", MessageFormatter.arrayFormat(msg, args).getMessage()); + } + } + + void warn(final String msg, final Throwable t) { + logger.warn(msg, t); + + if ( eventReporter != null ) { + eventReporter.reportEvent(Severity.WARNING, "Site-to-Site", msg + ": " + t.toString()); + } + } + + void error(final String msg, final Object... args) { + logger.error(msg, args); + if ( eventReporter != null ) { + eventReporter.reportEvent(Severity.ERROR, "Site-to-Site", MessageFormatter.arrayFormat(msg, args).getMessage()); + } + } + private String getPortIdentifier(final TransferDirection transferDirection) throws IOException { if ( remoteDestination.getIdentifier() != null ) { return remoteDestination.getIdentifier(); @@ -271,6 +294,7 @@ public class EndpointConnectionPool { logger.debug("{} No Connection available for Port {}; creating new Connection", this, portId); protocol = new SocketClientProtocol(); protocol.setDestination(new IdEnrichedRemoteDestination(remoteDestination, portId)); + protocol.setEventReporter(eventReporter); final long penalizationMillis = remoteDestination.getYieldPeriod(TimeUnit.MILLISECONDS); try { @@ -312,8 +336,15 @@ public class EndpointConnectionPool { // handle error cases if ( protocol.isDestinationFull() ) { - logger.warn("{} {} indicates that port's destination is full; penalizing peer", this, peer); + logger.warn("{} {} indicates that port {}'s destination is full; penalizing peer", + this, peer, config.getPortName() == null ? config.getPortIdentifier() : config.getPortName()); + penalize(peer, penalizationMillis); + try { + peer.close(); + } catch (final IOException ioe) { + } + continue; } else if ( protocol.isPortInvalid() ) { penalize(peer, penalizationMillis); @@ -336,7 +367,7 @@ public class EndpointConnectionPool { cleanup(protocol, peer); final String message = String.format("%s failed to communicate with %s due to %s", this, peer == null ? clusterUrl : peer, e.toString()); - logger.error(message); + error(message); if ( logger.isDebugEnabled() ) { logger.error("", e); } @@ -359,6 +390,15 @@ public class EndpointConnectionPool { } } } while ( connection == null || codec == null || commsSession == null || protocol == null ); + } catch (final Throwable t) { + if ( commsSession != null ) { + try { + commsSession.close(); + } catch (final IOException ioe) { + } + } + + throw t; } finally { if ( !addBack.isEmpty() ) { connectionQueue.addAll(addBack); @@ -449,7 +489,7 @@ public class EndpointConnectionPool { peerList = createPeerStatusList(direction); } catch (final Exception e) { final String message = String.format("%s Failed to update list of peers due to %s", this, e.toString()); - logger.warn(message); + warn(message); if ( logger.isDebugEnabled() ) { logger.warn("", e); } @@ -489,7 +529,7 @@ public class EndpointConnectionPool { } private boolean isPenalized(final PeerStatus peerStatus) { - final Long expirationEnd = peerTimeoutExpirations.get(peerStatus); + final Long expirationEnd = peerTimeoutExpirations.get(peerStatus.getPeerDescription()); return (expirationEnd == null ? false : expirationEnd > System.currentTimeMillis() ); } @@ -573,7 +613,7 @@ public class EndpointConnectionPool { clientProtocol.shutdown(peer); } catch (final IOException e) { final String message = String.format("%s Failed to shutdown protocol when updating list of peers due to %s", this, e.toString()); - logger.warn(message); + warn(message); if (logger.isDebugEnabled()) { logger.warn("", e); } @@ -583,7 +623,7 @@ public class EndpointConnectionPool { peer.close(); } catch (final IOException e) { final String message = String.format("%s Failed to close resources when updating list of peers due to %s", this, e.toString()); - logger.warn(message); + warn(message); if (logger.isDebugEnabled()) { logger.warn("", e); } @@ -608,7 +648,8 @@ public class EndpointConnectionPool { } } catch (final IOException e) { - logger.error("Failed to persist list of Peers due to {}; if restarted and peer's NCM is down, may be unable to transfer data until communications with NCM are restored", e.toString(), e); + error("Failed to persist list of Peers due to {}; if restarted and peer's NCM is down, may be unable to transfer data until communications with NCM are restored", e.toString()); + logger.error("", e); } } @@ -700,7 +741,7 @@ public class EndpointConnectionPool { final int flowFileCount = nodeInfo.getTotalFlowFiles(); // don't allow any node to get more than 80% of the data final double percentageOfFlowFiles = Math.min(0.8D, ((double) flowFileCount / (double) totalFlowFileCount)); - final double relativeWeighting = (direction == TransferDirection.RECEIVE) ? (1 - percentageOfFlowFiles) : percentageOfFlowFiles; + final double relativeWeighting = (direction == TransferDirection.SEND) ? (1 - percentageOfFlowFiles) : percentageOfFlowFiles; final int entries = Math.max(1, (int) (numDestinations * relativeWeighting)); entryCountMap.put(nodeInfo, Math.max(1, entries)); @@ -804,7 +845,7 @@ public class EndpointConnectionPool { peerStatusCache = new PeerStatusCache(statuses); logger.info("{} Successfully refreshed Peer Status; remote instance consists of {} peers", this, statuses.size()); } catch (Exception e) { - logger.warn("{} Unable to refresh Remote Group's peers due to {}", this, e); + warn("{} Unable to refresh Remote Group's peers due to {}", this, e); if (logger.isDebugEnabled()) { logger.warn("", e); } diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java index bd9319f8c1..4aab3f7f8e 100644 --- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/SocketClient.java @@ -84,6 +84,7 @@ public class SocketClient implements SiteToSiteClient { logger.debug("Unable to resolve port [{}] to an identifier", portName); } else { logger.debug("Resolved port [{}] to identifier [{}]", portName, portId); + this.portIdentifier = portId; } return portId; @@ -130,8 +131,14 @@ public class SocketClient implements SiteToSiteClient { return null; } - final Transaction transaction = connectionState.getSocketClientProtocol().startTransaction( + final Transaction transaction; + try { + transaction = connectionState.getSocketClientProtocol().startTransaction( connectionState.getPeer(), connectionState.getCodec(), direction); + } catch (final Throwable t) { + pool.terminate(connectionState); + throw new IOException("Unable to create Transaction to communicate with " + connectionState.getPeer(), t); + } // Wrap the transaction in a new one that will return the EndpointConnectionState back to the pool whenever // the transaction is either completed or canceled. diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java index c3275eab00..83c5305054 100644 --- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java @@ -27,6 +27,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; +import org.apache.nifi.events.EventReporter; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.processor.ProcessContext; @@ -75,6 +76,7 @@ public class SocketClientProtocol implements ClientProtocol { private int batchCount; private long batchSize; private long batchMillis; + private EventReporter eventReporter; private static final long BATCH_SEND_NANOS = TimeUnit.SECONDS.toNanos(5L); // send batches of up to 5 seconds @@ -93,6 +95,10 @@ public class SocketClientProtocol implements ClientProtocol { this.batchMillis = millis; } + public void setEventReporter(final EventReporter eventReporter) { + this.eventReporter = eventReporter; + } + public void setDestination(final RemoteDestination destination) { this.destination = destination; this.useCompression = destination.isUseCompression(); @@ -272,7 +278,7 @@ public class SocketClientProtocol implements ClientProtocol { } return new SocketClientTransaction(versionNegotiator.getVersion(), destination.getIdentifier(), peer, codec, - direction, useCompression, (int) destination.getYieldPeriod(TimeUnit.MILLISECONDS)); + direction, useCompression, (int) destination.getYieldPeriod(TimeUnit.MILLISECONDS), eventReporter); } diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java index a1ce07e450..e69104f201 100644 --- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientTransaction.java @@ -27,6 +27,7 @@ import java.util.zip.CRC32; import java.util.zip.CheckedInputStream; import java.util.zip.CheckedOutputStream; +import org.apache.nifi.events.EventReporter; import org.apache.nifi.remote.Communicant; import org.apache.nifi.remote.Peer; import org.apache.nifi.remote.Transaction; @@ -39,6 +40,7 @@ import org.apache.nifi.remote.io.CompressionOutputStream; import org.apache.nifi.remote.protocol.DataPacket; import org.apache.nifi.remote.protocol.RequestType; import org.apache.nifi.remote.util.StandardDataPacket; +import org.apache.nifi.reporting.Severity; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,6 +58,7 @@ public class SocketClientTransaction implements Transaction { private final Peer peer; private final int penaltyMillis; private final String destinationId; + private final EventReporter eventReporter; private boolean dataAvailable = false; private int transfers = 0; @@ -63,7 +66,7 @@ public class SocketClientTransaction implements Transaction { private TransactionState state; SocketClientTransaction(final int protocolVersion, final String destinationId, final Peer peer, final FlowFileCodec codec, - final TransferDirection direction, final boolean useCompression, final int penaltyMillis) throws IOException { + final TransferDirection direction, final boolean useCompression, final int penaltyMillis, final EventReporter eventReporter) throws IOException { this.protocolVersion = protocolVersion; this.destinationId = destinationId; this.peer = peer; @@ -74,6 +77,7 @@ public class SocketClientTransaction implements Transaction { this.compress = useCompression; this.state = TransactionState.TRANSACTION_STARTED; this.penaltyMillis = penaltyMillis; + this.eventReporter = eventReporter; initialize(); } @@ -116,11 +120,11 @@ public class SocketClientTransaction implements Transaction { try { try { if ( state != TransactionState.DATA_EXCHANGED && state != TransactionState.TRANSACTION_STARTED) { - throw new IllegalStateException("Cannot receive data because Transaction State is " + state); + throw new IllegalStateException("Cannot receive data from " + peer + " because Transaction State is " + state); } if ( direction == TransferDirection.SEND ) { - throw new IllegalStateException("Attempting to receive data but started a SEND Transaction"); + throw new IllegalStateException("Attempting to receive data from " + peer + " but started a SEND Transaction"); } // if we already know there's no data, just return null @@ -142,7 +146,7 @@ public class SocketClientTransaction implements Transaction { this.dataAvailable = false; break; default: - throw new ProtocolException("Got unexpected response when asking for data: " + dataAvailableCode); + throw new ProtocolException("Got unexpected response from " + peer + " when asking for data: " + dataAvailableCode); } } @@ -184,11 +188,11 @@ public class SocketClientTransaction implements Transaction { try { try { if ( state != TransactionState.DATA_EXCHANGED && state != TransactionState.TRANSACTION_STARTED) { - throw new IllegalStateException("Cannot send data because Transaction State is " + state); + throw new IllegalStateException("Cannot send data to " + peer + " because Transaction State is " + state); } if ( direction == TransferDirection.RECEIVE ) { - throw new IllegalStateException("Attempting to send data but started a RECEIVE Transaction"); + throw new IllegalStateException("Attempting to send data to " + peer + " but started a RECEIVE Transaction"); } if ( transfers > 0 ) { @@ -242,7 +246,7 @@ public class SocketClientTransaction implements Transaction { try { try { if ( state != TransactionState.TRANSACTION_CONFIRMED ) { - throw new IllegalStateException("Cannot complete transaction because state is " + state + + throw new IllegalStateException("Cannot complete transaction with " + peer + " because state is " + state + "; Transaction can only be completed when state is " + TransactionState.TRANSACTION_CONFIRMED); } @@ -272,7 +276,7 @@ public class SocketClientTransaction implements Transaction { peer.penalize(destinationId, penaltyMillis); backoff = true; } else if ( transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED ) { - throw new ProtocolException("After sending data, expected TRANSACTION_FINISHED response but got " + transactionResponse); + throw new ProtocolException("After sending data to " + peer + ", expected TRANSACTION_FINISHED response but got " + transactionResponse); } state = TransactionState.TRANSACTION_COMPLETED; @@ -324,7 +328,10 @@ public class SocketClientTransaction implements Transaction { try { confirmTransactionResponse = Response.read(dis); } catch (final IOException ioe) { - logger.error("Failed to receive response code from {} when expected confirmation of transaction", peer); + logger.error("Failed to receive response code from {} when expecting confirmation of transaction", peer); + if ( eventReporter != null ) { + eventReporter.reportEvent(Severity.ERROR, "Site-to-Site", "Failed to receive response code from " + peer + " when expecting confirmation of transaction"); + } throw ioe; } diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java index cb7af08e84..c5cca78614 100644 --- a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java @@ -39,7 +39,7 @@ public class TestEndpointConnectionStatePool { collection.add(new NodeInformation("ShouldGetMedium", 5, 5555, true, 4096)); clusterNodeInfo.setNodeInformation(collection); - final List destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND); + final List destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.RECEIVE); for ( final PeerStatus peerStatus : destinations ) { System.out.println(peerStatus.getPeerDescription()); } @@ -53,7 +53,7 @@ public class TestEndpointConnectionStatePool { collection.add(new NodeInformation("ShouldGetLots", 2, 2222, true, 50000)); clusterNodeInfo.setNodeInformation(collection); - final List destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND); + final List destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.RECEIVE); for ( final PeerStatus peerStatus : destinations ) { System.out.println(peerStatus.getPeerDescription()); } diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/EscapeUtils.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/EscapeUtils.java new file mode 100644 index 0000000000..9d48d3dc57 --- /dev/null +++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/EscapeUtils.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.util; + +public class EscapeUtils { + + /** + * Escapes the specified html by replacing &, <, >, ", ', / + * with their corresponding html entity. If html is null, null is returned. + * + * @param html + * @return + */ + public static String escapeHtml(String html) { + if (html == null) { + return null; + } + + html = html.replace("&", "&"); + html = html.replace("<", "<"); + html = html.replace(">", ">"); + html = html.replace("\"", """); + html = html.replace("'", "'"); + html = html.replace("/", "/"); + + return html; + } +} diff --git a/nifi/nifi-commons/pom.xml b/nifi/nifi-commons/pom.xml index 53580549bc..1d2ce46418 100644 --- a/nifi/nifi-commons/pom.xml +++ b/nifi/nifi-commons/pom.xml @@ -36,5 +36,6 @@ nifi-processor-utilities nifi-write-ahead-log nifi-site-to-site-client + nifi-hl7-query-language diff --git a/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc index 96dd423619..d33b4de975 100644 --- a/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc +++ b/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc @@ -229,7 +229,7 @@ When the DFM makes changes to the dataflow, the NCM communicates those changes t *Dealing with Disconnected Nodes* + -A DFM may manually disconnect a node from the cluster. But if a node becomes disconnected for any other reason (such as due to lack of heartbeat), the NCM will show a bulletin on the User Interface, and the DFM will not be able to make any changes to the dataflow until the issue of the disconnected node is resolved. The DFM or the Administrator will need to troubleshoot the issue with the node and resolve it before any new changes may be made to the dataflow. However, it is worth noting that just because a node is disconnected does not mean that it is definitely down; it just means that the NCM cannot communicate with the node. +A DFM may manually disconnect a node from the cluster. But if a node becomes disconnected for any other reason (such as due to lack of heartbeat), the NCM will show a bulletin on the User Interface, and the DFM will not be able to make any changes to the dataflow until the issue of the disconnected node is resolved. The DFM or the Administrator will need to troubleshoot the issue with the node and resolve it before any new changes may be made to the dataflow. However, it is worth noting that just because a node is disconnected does not mean that it is not working; it just means that the NCM cannot communicate with the node. *Basic Cluster Setup* + @@ -242,7 +242,7 @@ This section describes the setup for a simple two-node, non-secure, unicast clus Administrators may install each instance on a separate server; however, it is also perfectly fine to install the NCM and one of the nodes on the same server, as the NCM is very lightweight. Just keep in mind that the ports assigned to each instance must not collide if the NCM and one of the nodes share the same server. -For each instance, the clustering properties in the _nifi.properties_ file will need to be updated. All the clustering properties are described in the <> section of this guide; however, in this section, we will focus on the minimum properties that must be set for simple cluster. +For each instance, certain properties in the _nifi.properties_ file will need to be updated. In particular, the Web and Clustering properties should be evaluated for your situation and adjusted accordingly. All the properties are described in the <> section of this guide; however, in this section, we will focus on the minimum properties that must be set for a simple cluster. For all three instances, the Cluster Common Properties can be left with the default settings. Note, however, that if you change these settings, they must be set the same on every instance in the cluster (NCM and nodes). @@ -255,18 +255,20 @@ For the NCM, the minimum properties to configure are as follows: For Node 1, the minimum properties to configure are as follows: -* Under the Web Properties, set either the http or https port that you want Node 1 to run on. If the NCM is running on the same server, choose a different web port for Node 1. +* Under the Web Properties, set either the http or https port that you want Node 1 to run on. If the NCM is running on the same server, choose a different web port for Node 1. Also, consider whether you need to set the http or https host property. * Under Cluster Node Properties, set the following: ** nifi.cluster.is.node - Set this to _true_. +** nifi.cluster.node.address - Set this to the fully qualified hostname of the node. If left blank, it defaults to "localhost". ** nifi.cluster.node.protocol.port - Set this to an open port that is higher than 1024 (anything lower requires root). If Node 1 and the NCM are on the same server, make sure this port is different from the nifi.cluster.protocol.manager.port. ** nifi.cluster.node.unicast.manager.address - Set this to the NCM's fully qualified hostname. ** nifi.cluster.node.unicast.manager.protocol.port - Set this to exactly the same port that was set on the NCM for the property nifi.cluster.manager.protocol.port. For Node 2, the minimum properties to configure are as follows: -* Under the Web Properties, set either the http or https port that you want Node 2 to run on. +* Under the Web Properties, set either the http or https port that you want Node 2 to run on. Also, consider whether you need to set the http or https host property. * Under the Cluster Node Properties, set the following: ** nifi.cluster.is.node - Set this to _true_. +** nifi.cluster.node.address - Set this to the fully qualified hostname of the node. If left blank, it defaults to "localhost". ** nifi.cluster.node.protocol.port - Set this to an open port that is higher than 1024 (anything lower requires root). ** nifi.cluster.node.unicast.manager.address - Set this to the NCM's fully qualified hostname. ** nifi.cluster.node.unicast.manager.protocol.port - Set this to exactly the same port that was set on the NCM for the property nifi.cluster.manager.protocol.port. @@ -275,6 +277,17 @@ Now, it is possible to start up the cluster. Technically, it does not matter whi image:ncm.png["NCM User Interface", width=940] +*Troubleshooting* + +If you encounter issues and your cluster does not work as described, investigate the nifi.app log and nifi.user log on both the NCM and the nodes. If needed, you can change the logging level to DEBUG by editing the conf/logback.xml file. Specifically, set the level="DEBUG" in the following line (instead of "INFO"): + +---- + + + +---- + [[system_properties]] diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceInitializationContext.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceInitializationContext.java index 86624aeaaa..fd3c2de562 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceInitializationContext.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceInitializationContext.java @@ -19,13 +19,20 @@ package org.apache.nifi.util; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerServiceInitializationContext; import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.logging.ComponentLog; public class MockControllerServiceInitializationContext extends MockControllerServiceLookup implements ControllerServiceInitializationContext, ControllerServiceLookup { private final String identifier; + private final ComponentLog logger; public MockControllerServiceInitializationContext(final ControllerService controllerService, final String identifier) { + this(controllerService, identifier, new MockProcessorLog(identifier, controllerService)); + } + + public MockControllerServiceInitializationContext(final ControllerService controllerService, final String identifier, final ComponentLog logger) { this.identifier = identifier; + this.logger = logger; addControllerService(controllerService, identifier); } @@ -33,9 +40,19 @@ public class MockControllerServiceInitializationContext extends MockControllerSe public String getIdentifier() { return identifier; } + + @Override + public String getControllerServiceName(final String serviceIdentifier) { + return null; + } @Override public ControllerServiceLookup getControllerServiceLookup() { return this; } + + @Override + public ComponentLog getLogger() { + return logger; + } } diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java index 8298a39617..2734440497 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java @@ -76,6 +76,11 @@ public abstract class MockControllerServiceLookup implements ControllerServiceLo return isControllerServiceEnabled(service.getIdentifier()); } + @Override + public boolean isControllerServiceEnabling(final String serviceIdentifier) { + return false; + } + @Override public Set getControllerServiceIdentifiers(final Class serviceType) { final Set ids = new HashSet<>(); @@ -86,4 +91,10 @@ public abstract class MockControllerServiceLookup implements ControllerServiceLo } return ids; } + + @Override + public String getControllerServiceName(String serviceIdentifier) { + final ControllerServiceConfiguration status = controllerServiceMap.get(serviceIdentifier); + return status == null ? null : serviceIdentifier; + } } diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorInitializationContext.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorInitializationContext.java index f49a6c5ee9..0aa2749a57 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorInitializationContext.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorInitializationContext.java @@ -62,6 +62,11 @@ public class MockProcessorInitializationContext implements ProcessorInitializati return this; } + @Override + public String getControllerServiceName(String serviceIdentifier) { + return context.getControllerServiceName(serviceIdentifier); + } + @Override public boolean isControllerServiceEnabled(String serviceIdentifier) { return context.isControllerServiceEnabled(serviceIdentifier); @@ -71,4 +76,9 @@ public class MockProcessorInitializationContext implements ProcessorInitializati public boolean isControllerServiceEnabled(ControllerService service) { return context.isControllerServiceEnabled(service); } + + @Override + public boolean isControllerServiceEnabling(String serviceIdentifier) { + return context.isControllerServiceEnabling(serviceIdentifier); + } } diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorLog.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorLog.java index f8489f88ea..5505e88fec 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorLog.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorLog.java @@ -17,28 +17,26 @@ package org.apache.nifi.util; import org.apache.nifi.logging.ProcessorLog; -import org.apache.nifi.processor.Processor; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class MockProcessorLog implements ProcessorLog { private final Logger logger; - private final Processor processor; + private final Object component; - public MockProcessorLog(final String processorId, final Processor processor) { - this.logger = LoggerFactory.getLogger(processor.getClass()); - this.processor = processor; + public MockProcessorLog(final String componentId, final Object component) { + this.logger = LoggerFactory.getLogger(component.getClass()); + this.component = component; } private Object[] addProcessor(final Object[] originalArgs) { - return prependToArgs(originalArgs, processor); + return prependToArgs(originalArgs, component); } private Object[] addProcessorAndThrowable(final Object[] os, final Throwable t) { final Object[] modifiedArgs = new Object[os.length + 2]; - modifiedArgs[0] = processor.toString(); + modifiedArgs[0] = component.toString(); for (int i = 0; i < os.length; i++) { modifiedArgs[i + 1] = os[i]; } @@ -75,7 +73,7 @@ public class MockProcessorLog implements ProcessorLog { */ @Override public void warn(final String msg, final Throwable t) { - warn("{} " + msg, new Object[]{processor}, t); + warn("{} " + msg, new Object[]{component}, t); } /** @@ -118,7 +116,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void warn(String msg) { msg = "{} " + msg; - logger.warn(msg, processor); + logger.warn(msg, component); } /** @@ -129,7 +127,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void trace(String msg, Throwable t) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.trace(msg, os, t); } @@ -152,7 +150,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void trace(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.trace(msg, os); } @@ -224,7 +222,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void info(String msg, Throwable t) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.info(msg, os); if (logger.isDebugEnabled()) { @@ -252,7 +250,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void info(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.info(msg, os); } @@ -291,7 +289,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void error(String msg, Throwable t) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.error(msg, os, t); if (logger.isDebugEnabled()) { @@ -322,7 +320,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void error(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.error(msg, os); } @@ -352,7 +350,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void debug(String msg, Throwable t) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.debug(msg, os, t); } @@ -394,7 +392,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void debug(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.debug(msg, os); } diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockReportingInitializationContext.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockReportingInitializationContext.java index a874536d67..7cabef22a7 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockReportingInitializationContext.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockReportingInitializationContext.java @@ -22,6 +22,7 @@ import java.util.concurrent.TimeUnit; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.reporting.ReportingInitializationContext; import org.apache.nifi.scheduling.SchedulingStrategy; @@ -30,10 +31,12 @@ public class MockReportingInitializationContext extends MockControllerServiceLoo private final String identifier; private final String name; private final Map properties = new HashMap<>(); + private final ComponentLog logger; - public MockReportingInitializationContext(final String identifier, final String name) { + public MockReportingInitializationContext(final String identifier, final String name, final ComponentLog logger) { this.identifier = identifier; this.name = name; + this.logger = logger; } @Override @@ -78,4 +81,9 @@ public class MockReportingInitializationContext extends MockControllerServiceLoo public SchedulingStrategy getSchedulingStrategy() { return SchedulingStrategy.TIMER_DRIVEN; } + + @Override + public ComponentLog getLogger() { + return logger; + } } diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java index c00386ef5d..c9b1cdaf30 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java @@ -103,6 +103,21 @@ public class MockValidationContext implements ValidationContext, ControllerServi } @Override + public String getControllerServiceName(final String serviceIdentifier) { + final ControllerServiceConfiguration configuration = context.getConfiguration(serviceIdentifier); + return configuration == null ? null : serviceIdentifier; + } + + @Override + public boolean isValidationRequired(final ControllerService service) { + return true; + } + + @Override + public boolean isControllerServiceEnabling(String serviceIdentifier) { + return context.isControllerServiceEnabling(serviceIdentifier); + } + public boolean isExpressionLanguagePresent(final String value) { if ( value == null ) { return false; diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java index 8d691dd879..d66ed8165d 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java @@ -59,6 +59,7 @@ import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.ProcessSessionFactory; import org.apache.nifi.processor.Processor; import org.apache.nifi.processor.QueueSize; @@ -512,13 +513,15 @@ public class StandardProcessorTestRunner implements TestRunner { @Override public void addControllerService(final String identifier, final ControllerService service, final Map properties) throws InitializationException { + // hold off on failing due to deprecated annotation for now... will introduce later. // for ( final Method method : service.getClass().getMethods() ) { // if ( method.isAnnotationPresent(org.apache.nifi.controller.annotation.OnConfigured.class) ) { // Assert.fail("Controller Service " + service + " is using deprecated Annotation " + org.apache.nifi.controller.annotation.OnConfigured.class + " for method " + method); // } // } - final MockControllerServiceInitializationContext initContext = new MockControllerServiceInitializationContext(requireNonNull(service), requireNonNull(identifier)); + final ComponentLog logger = new MockProcessorLog(identifier, service); + final MockControllerServiceInitializationContext initContext = new MockControllerServiceInitializationContext(requireNonNull(service), requireNonNull(identifier), logger); service.initialize(initContext); final Map resolvedProps = new HashMap<>(); diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-nar/pom.xml new file mode 100644 index 0000000000..3e6f450181 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-nar/pom.xml @@ -0,0 +1,36 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-aws-bundle + 0.1.0-incubating-SNAPSHOT + + + nifi-aws-nar + nar + + + + org.apache.nifi + nifi-aws-processors + 0.1.0-incubating-SNAPSHOT + + + + diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml new file mode 100644 index 0000000000..227077344e --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml @@ -0,0 +1,58 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-aws-bundle + 0.1.0-incubating-SNAPSHOT + + + nifi-aws-processors + jar + + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-processor-utils + + + com.amazonaws + aws-java-sdk + + + + org.apache.nifi + nifi-mock + test + + + org.slf4j + slf4j-simple + test + + + junit + junit + test + + + diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java new file mode 100644 index 0000000000..11c6a9d69f --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java @@ -0,0 +1,195 @@ +/* + * 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.processors.aws; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import com.amazonaws.AmazonWebServiceClient; +import com.amazonaws.ClientConfiguration; +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.auth.AnonymousAWSCredentials; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.auth.PropertiesCredentials; +import com.amazonaws.regions.Region; +import com.amazonaws.regions.Regions; + +public abstract class AbstractAWSProcessor extends AbstractProcessor { + + public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success") + .description("FlowFiles are routed to success after being successfully copied to Amazon S3").build(); + public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure") + .description("FlowFiles are routed to failure if unable to be copied to Amazon S3").build(); + + public static final Set relationships = Collections.unmodifiableSet( + new HashSet<>(Arrays.asList(REL_SUCCESS, REL_FAILURE))); + + public static final PropertyDescriptor CREDENTAILS_FILE = new PropertyDescriptor.Builder() + .name("Credentials File") + .expressionLanguageSupported(false) + .required(false) + .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR) + .build(); + public static final PropertyDescriptor ACCESS_KEY = new PropertyDescriptor.Builder() + .name("Access Key") + .expressionLanguageSupported(false) + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .sensitive(true) + .build(); + public static final PropertyDescriptor SECRET_KEY = new PropertyDescriptor.Builder() + .name("Secret Key") + .expressionLanguageSupported(false) + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .sensitive(true) + .build(); + public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder() + .name("Region") + .required(true) + .allowableValues(getAvailableRegions()) + .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue()) + .build(); + + public static final PropertyDescriptor TIMEOUT = new PropertyDescriptor.Builder() + .name("Communications Timeout") + .required(true) + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .defaultValue("30 secs") + .build(); + + + private volatile ClientType client; + + + private static AllowableValue createAllowableValue(final Regions regions) { + return new AllowableValue(regions.getName(), regions.getName(), regions.getName()); + } + + private static AllowableValue[] getAvailableRegions() { + final List values = new ArrayList<>(); + for ( final Regions regions : Regions.values() ) { + values.add(createAllowableValue(regions)); + } + + return (AllowableValue[]) values.toArray(new AllowableValue[values.size()]); + } + + @Override + public Set getRelationships() { + return relationships; + } + + @Override + protected Collection customValidate(final ValidationContext validationContext) { + final List problems = new ArrayList<>(super.customValidate(validationContext)); + + final boolean accessKeySet = validationContext.getProperty(ACCESS_KEY).isSet(); + final boolean secretKeySet = validationContext.getProperty(SECRET_KEY).isSet(); + if ( (accessKeySet && !secretKeySet) || (secretKeySet && !accessKeySet) ) { + problems.add(new ValidationResult.Builder().input("Access Key").valid(false).explanation("If setting Secret Key or Access Key, must set both").build()); + } + + final boolean credentialsFileSet = validationContext.getProperty(CREDENTAILS_FILE).isSet(); + if ( (secretKeySet || accessKeySet) && credentialsFileSet ) { + problems.add(new ValidationResult.Builder().input("Access Key").valid(false).explanation("Cannot set both Credentials File and Secret Key/Access Key").build()); + } + + return problems; + } + + + protected ClientConfiguration createConfiguration(final ProcessContext context) { + final ClientConfiguration config = new ClientConfiguration(); + config.setMaxConnections(context.getMaxConcurrentTasks()); + config.setMaxErrorRetry(0); + config.setUserAgent("NiFi"); + + final int commsTimeout = context.getProperty(TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue(); + config.setConnectionTimeout(commsTimeout); + config.setSocketTimeout(commsTimeout); + + return config; + } + + + @OnScheduled + public void onScheduled(final ProcessContext context) { + final ClientType awsClient = createClient(context, getCredentials(context), createConfiguration(context)); + this.client = awsClient; + + // if the processor supports REGION, get the configured region. + if ( getSupportedPropertyDescriptors().contains(REGION) ) { + final String region = context.getProperty(REGION).getValue(); + if ( region != null ) { + client.setRegion(Region.getRegion(Regions.fromName(region))); + } + } + } + + protected abstract ClientType createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config); + + protected ClientType getClient() { + return client; + } + + protected AWSCredentials getCredentials(final ProcessContext context) { + final String accessKey = context.getProperty(ACCESS_KEY).getValue(); + final String secretKey = context.getProperty(SECRET_KEY).getValue(); + + final String credentialsFile = context.getProperty(CREDENTAILS_FILE).getValue(); + + if ( credentialsFile != null ) { + try { + return new PropertiesCredentials(new File(credentialsFile)); + } catch (final IOException ioe) { + throw new ProcessException("Could not read Credentials File", ioe); + } + } + + if ( accessKey != null && secretKey != null ) { + return new BasicAWSCredentials(accessKey, secretKey); + } + + return new AnonymousAWSCredentials(); + } + + + protected boolean isEmpty(final String value) { + return value == null || value.trim().equals(""); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/AbstractS3Processor.java b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/AbstractS3Processor.java new file mode 100644 index 0000000000..624015b6a5 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/AbstractS3Processor.java @@ -0,0 +1,171 @@ +/* + * 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.processors.aws.s3; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.processors.aws.AbstractAWSProcessor; + +import com.amazonaws.ClientConfiguration; +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.services.s3.AmazonS3Client; +import com.amazonaws.services.s3.model.AccessControlList; +import com.amazonaws.services.s3.model.CanonicalGrantee; +import com.amazonaws.services.s3.model.EmailAddressGrantee; +import com.amazonaws.services.s3.model.Grantee; +import com.amazonaws.services.s3.model.Owner; +import com.amazonaws.services.s3.model.Permission; + +public abstract class AbstractS3Processor extends AbstractAWSProcessor { + + public static final PropertyDescriptor FULL_CONTROL_USER_LIST = new PropertyDescriptor.Builder() + .name("FullControl User List") + .required(false) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .description("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have Full Control for an object") + .defaultValue("${s3.permissions.full.users}") + .build(); + public static final PropertyDescriptor READ_USER_LIST = new PropertyDescriptor.Builder() + .name("Read Permission User List") + .required(false) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .description("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have Read Access for an object") + .defaultValue("${s3.permissions.read.users}") + .build(); + public static final PropertyDescriptor WRITE_USER_LIST = new PropertyDescriptor.Builder() + .name("Write Permission User List") + .required(false) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .description("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have Write Access for an object") + .defaultValue("${s3.permissions.write.users}") + .build(); + public static final PropertyDescriptor READ_ACL_LIST = new PropertyDescriptor.Builder() + .name("Read ACL User List") + .required(false) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .description("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have permissions to read the Access Control List for an object") + .defaultValue("${s3.permissions.readacl.users}") + .build(); + public static final PropertyDescriptor WRITE_ACL_LIST = new PropertyDescriptor.Builder() + .name("Write ACL User List") + .required(false) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .description("A comma-separated list of Amazon User ID's or E-mail addresses that specifies who should have permissions to change the Access Control List for an object") + .defaultValue("${s3.permissions.writeacl.users}") + .build(); + public static final PropertyDescriptor OWNER = new PropertyDescriptor.Builder() + .name("Owner") + .required(false) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .description("The Amazon ID to use for the object's owner") + .defaultValue("${s3.owner}") + .build(); + public static final PropertyDescriptor BUCKET = new PropertyDescriptor.Builder() + .name("Bucket") + .expressionLanguageSupported(true) + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor KEY = new PropertyDescriptor.Builder() + .name("Object Key") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("${filename}") + .build(); + + + @Override + protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) { + return new AmazonS3Client(credentials, config); + } + + + protected Grantee createGrantee(final String value) { + if ( isEmpty(value) ) { + return null; + } + + if ( value.contains("@") ) { + return new EmailAddressGrantee(value); + } else { + return new CanonicalGrantee(value); + } + } + + protected final List createGrantees(final String value) { + if ( isEmpty(value) ) { + return Collections.emptyList(); + } + + final List grantees = new ArrayList<>(); + final String[] vals = value.split(","); + for ( final String val : vals ) { + final String identifier = val.trim(); + final Grantee grantee = createGrantee(identifier); + if ( grantee != null ) { + grantees.add(grantee); + } + } + return grantees; + } + + protected final AccessControlList createACL(final ProcessContext context, final FlowFile flowFile) { + final AccessControlList acl = new AccessControlList(); + + final String ownerId = context.getProperty(OWNER).evaluateAttributeExpressions(flowFile).getValue(); + if ( !isEmpty(ownerId) ) { + final Owner owner = new Owner(); + owner.setId(ownerId); + acl.setOwner(owner); + } + + for ( final Grantee grantee : createGrantees(context.getProperty(FULL_CONTROL_USER_LIST).evaluateAttributeExpressions(flowFile).getValue())) { + acl.grantPermission(grantee, Permission.FullControl); + } + + for ( final Grantee grantee : createGrantees(context.getProperty(READ_USER_LIST).evaluateAttributeExpressions(flowFile).getValue())) { + acl.grantPermission(grantee, Permission.Read); + } + + for ( final Grantee grantee : createGrantees(context.getProperty(WRITE_USER_LIST).evaluateAttributeExpressions(flowFile).getValue())) { + acl.grantPermission(grantee, Permission.Write); + } + + for ( final Grantee grantee : createGrantees(context.getProperty(READ_ACL_LIST).evaluateAttributeExpressions(flowFile).getValue())) { + acl.grantPermission(grantee, Permission.ReadAcp); + } + + for ( final Grantee grantee : createGrantees(context.getProperty(WRITE_ACL_LIST).evaluateAttributeExpressions(flowFile).getValue())) { + acl.grantPermission(grantee, Permission.WriteAcp); + } + + return acl; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/FetchS3Object.java b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/FetchS3Object.java new file mode 100644 index 0000000000..63c834630c --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/FetchS3Object.java @@ -0,0 +1,157 @@ +/* + * 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.processors.aws.s3; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.util.StandardValidators; + +import com.amazonaws.AmazonClientException; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.model.GetObjectRequest; +import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.S3Object; + + +@SupportsBatching +@SeeAlso({PutS3Object.class}) +@Tags({"Amazon", "S3", "AWS", "Get", "Fetch"}) +@CapabilityDescription("Retrieves the contents of an S3 Object and writes it to the content of a FlowFile") +@WritesAttributes({ + @WritesAttribute(attribute="s3.bucket", description="The name of the S3 bucket"), + @WritesAttribute(attribute="path", description="The path of the file"), + @WritesAttribute(attribute="absolute.path", description="The path of the file"), + @WritesAttribute(attribute="filename", description="The name of the file"), + @WritesAttribute(attribute="hash.value", description="The MD5 sum of the file"), + @WritesAttribute(attribute="hash.algorithm", description="MD5"), + @WritesAttribute(attribute="mime.type", description="If S3 provides the content type/MIME type, this attribute will hold that file"), + @WritesAttribute(attribute="s3.etag", description="The ETag that can be used to see if the file has changed"), + @WritesAttribute(attribute="s3.expirationTime", description="If the file has an expiration date, this attribute will be set, containing the milliseconds since epoch in UTC time"), + @WritesAttribute(attribute="s3.expirationTimeRuleId", description="The ID of the rule that dictates this object's expiration time"), + @WritesAttribute(attribute="s3.version", description="The version of the S3 object"), +}) +public class FetchS3Object extends AbstractS3Processor { + + public static final PropertyDescriptor VERSION_ID = new PropertyDescriptor.Builder() + .name("Version") + .description("The Version of the Object to download") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .required(false) + .build(); + + public static final List properties = Collections.unmodifiableList( + Arrays.asList(BUCKET, KEY, REGION, ACCESS_KEY, SECRET_KEY, CREDENTAILS_FILE, TIMEOUT, VERSION_ID) ); + + @Override + protected List getSupportedPropertyDescriptors() { + return properties; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) { + FlowFile flowFile = session.get(); + if ( flowFile == null ) { + return; + } + + final long startNanos = System.nanoTime(); + final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions(flowFile).getValue(); + final String key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue(); + final String versionId = context.getProperty(VERSION_ID).evaluateAttributeExpressions(flowFile).getValue(); + + final AmazonS3 client = getClient(); + final GetObjectRequest request; + if ( versionId == null ) { + request = new GetObjectRequest(bucket, key); + } else { + request = new GetObjectRequest(bucket, key, versionId); + } + + final Map attributes = new HashMap<>(); + try (final S3Object s3Object = client.getObject(request)) { + flowFile = session.importFrom(s3Object.getObjectContent(), flowFile); + attributes.put("s3.bucket", s3Object.getBucketName()); + + final ObjectMetadata metadata = s3Object.getObjectMetadata(); + if ( metadata.getContentDisposition() != null ) { + final String fullyQualified = metadata.getContentDisposition(); + final int lastSlash = fullyQualified.lastIndexOf("/"); + if ( lastSlash > -1 && lastSlash < fullyQualified.length() - 1 ) { + attributes.put(CoreAttributes.PATH.key(), fullyQualified.substring(0, lastSlash)); + attributes.put(CoreAttributes.ABSOLUTE_PATH.key(), fullyQualified); + attributes.put(CoreAttributes.FILENAME.key(), fullyQualified.substring(lastSlash + 1)); + } else { + attributes.put(CoreAttributes.FILENAME.key(), metadata.getContentDisposition()); + } + } + if (metadata.getContentMD5() != null ) { + attributes.put("hash.value", metadata.getContentMD5()); + attributes.put("hash.algorithm", "MD5"); + } + if ( metadata.getContentType() != null ) { + attributes.put(CoreAttributes.MIME_TYPE.key(), metadata.getContentType()); + } + if ( metadata.getETag() != null ) { + attributes.put("s3.etag", metadata.getETag()); + } + if ( metadata.getExpirationTime() != null ) { + attributes.put("s3.expirationTime", String.valueOf(metadata.getExpirationTime().getTime())); + } + if ( metadata.getExpirationTimeRuleId() != null ) { + attributes.put("s3.expirationTimeRuleId", metadata.getExpirationTimeRuleId()); + } + if ( metadata.getUserMetadata() != null ) { + attributes.putAll(metadata.getUserMetadata()); + } + if ( metadata.getVersionId() != null ) { + attributes.put("s3.version", metadata.getVersionId()); + } + } catch (final IOException | AmazonClientException ioe) { + getLogger().error("Failed to retrieve S3 Object for {}; routing to failure", new Object[] {flowFile, ioe}); + session.transfer(flowFile, REL_FAILURE); + return; + } + + if ( !attributes.isEmpty() ) { + flowFile = session.putAllAttributes(flowFile, attributes); + } + + session.transfer(flowFile, REL_SUCCESS); + final long transferMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos); + getLogger().info("Successfully retrieved S3 Object for {} in {} millis; routing to success", new Object[] {flowFile, transferMillis}); + session.getProvenanceReporter().receive(flowFile, "http://" + bucket + ".amazonaws.com/" + key, transferMillis); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java new file mode 100644 index 0000000000..9a4fc5b995 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/PutS3Object.java @@ -0,0 +1,181 @@ +/* + * 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.processors.aws.s3; + +import java.io.BufferedInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Arrays; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.annotation.behavior.DynamicProperty; +import org.apache.nifi.annotation.behavior.ReadsAttribute; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.io.InputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; + +import com.amazonaws.AmazonClientException; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.model.AccessControlList; +import com.amazonaws.services.s3.model.ObjectMetadata; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.PutObjectResult; +import com.amazonaws.services.s3.model.StorageClass; + +@SupportsBatching +@SeeAlso({FetchS3Object.class}) +@Tags({"Amazon", "S3", "AWS", "Archive", "Put"}) +@CapabilityDescription("Puts FlowFiles to an Amazon S3 Bucket") +@DynamicProperty(name="The name of a User-Defined Metadata field to add to the S3 Object", + value="The value of a User-Defined Metadata field to add to the S3 Object", + description="Allows user-defined metadata to be added to the S3 object as key/value pairs", + supportsExpressionLanguage=true) +@ReadsAttribute(attribute="filename", description="Uses the FlowFile's filename as the filename for the S3 object") +@WritesAttributes({ + @WritesAttribute(attribute="s3.version", description="The version of the S3 Object that was put to S3"), + @WritesAttribute(attribute="s3.etag", description="The ETag of the S3 Object"), + @WritesAttribute(attribute="s3.expiration", description="A human-readable form of the expiration date of the S3 object, if one is set") +}) +public class PutS3Object extends AbstractS3Processor { + public static final PropertyDescriptor EXPIRATION_RULE_ID = new PropertyDescriptor.Builder() + .name("Expiration Time Rule") + .required(false) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final PropertyDescriptor STORAGE_CLASS = new PropertyDescriptor.Builder() + .name("Storage Class") + .required(true) + .allowableValues(StorageClass.Standard.name(), StorageClass.ReducedRedundancy.name()) + .defaultValue(StorageClass.Standard.name()) + .build(); + + public static final List properties = Collections.unmodifiableList( + Arrays.asList(KEY, BUCKET, ACCESS_KEY, SECRET_KEY, CREDENTAILS_FILE, STORAGE_CLASS, REGION, TIMEOUT, EXPIRATION_RULE_ID, + FULL_CONTROL_USER_LIST, READ_USER_LIST, WRITE_USER_LIST, READ_ACL_LIST, WRITE_ACL_LIST, OWNER) ); + + + @Override + protected List getSupportedPropertyDescriptors() { + return properties; + } + + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .name(propertyDescriptorName) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .dynamic(true) + .build(); + } + + public void onTrigger(final ProcessContext context, final ProcessSession session) { + FlowFile flowFile = session.get(); + if ( flowFile == null ) { + return; + } + + final long startNanos = System.nanoTime(); + + final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions(flowFile).getValue(); + final String key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue(); + + final AmazonS3 s3 = getClient(); + final FlowFile ff = flowFile; + final Map attributes = new HashMap<>(); + try { + session.read(flowFile, new InputStreamCallback() { + @Override + public void process(final InputStream rawIn) throws IOException { + try (final InputStream in = new BufferedInputStream(rawIn)) { + final ObjectMetadata objectMetadata = new ObjectMetadata(); + objectMetadata.setContentDisposition(ff.getAttribute(CoreAttributes.FILENAME.key())); + objectMetadata.setContentLength(ff.getSize()); + + final String expirationRule = context.getProperty(EXPIRATION_RULE_ID).evaluateAttributeExpressions(ff).getValue(); + if ( expirationRule != null ) { + objectMetadata.setExpirationTimeRuleId(expirationRule); + } + + final Map userMetadata = new HashMap<>(); + for ( final Map.Entry entry : context.getProperties().entrySet() ) { + if ( entry.getKey().isDynamic() ) { + final String value = context.getProperty(entry.getKey()).evaluateAttributeExpressions(ff).getValue(); + userMetadata.put(entry.getKey().getName(), value); + } + } + + if ( !userMetadata.isEmpty() ) { + objectMetadata.setUserMetadata(userMetadata); + } + + final PutObjectRequest request = new PutObjectRequest(bucket, key, in, objectMetadata); + request.setStorageClass(StorageClass.valueOf(context.getProperty(STORAGE_CLASS).getValue())); + final AccessControlList acl = createACL(context, ff); + if ( acl != null ) { + request.setAccessControlList(acl); + } + + final PutObjectResult result = s3.putObject(request); + if ( result.getVersionId() != null ) { + attributes.put("s3.version", result.getVersionId()); + } + + attributes.put("s3.etag", result.getETag()); + + final Date expiration = result.getExpirationTime(); + if ( expiration != null ) { + attributes.put("s3.expiration", expiration.toString()); + } + } + } + }); + + if ( !attributes.isEmpty() ) { + flowFile = session.putAllAttributes(flowFile, attributes); + } + session.transfer(flowFile, REL_SUCCESS); + + final String url = "http://" + bucket + ".s3.amazonaws.com/" + key; + final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos); + session.getProvenanceReporter().send(flowFile, url, millis); + + getLogger().info("Successfully put {} to Amazon S3 in {} milliseconds", new Object[] {ff, millis}); + } catch (final ProcessException | AmazonClientException pe) { + getLogger().error("Failed to put {} to Amazon S3 due to {}", new Object[] {flowFile, pe}); + session.transfer(flowFile, REL_FAILURE); + } + } +} \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sns/AbstractSNSProcessor.java b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sns/AbstractSNSProcessor.java new file mode 100644 index 0000000000..5447169034 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sns/AbstractSNSProcessor.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.aws.sns; + +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.processors.aws.AbstractAWSProcessor; + +import com.amazonaws.ClientConfiguration; +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.services.sns.AmazonSNSClient; + +public abstract class AbstractSNSProcessor extends AbstractAWSProcessor { + + protected static final AllowableValue ARN_TYPE_TOPIC = + new AllowableValue("Topic ARN", "Topic ARN", "The ARN is the name of a topic"); + protected static final AllowableValue ARN_TYPE_TARGET = + new AllowableValue("Target ARN", "Target ARN", "The ARN is the name of a particular Target, used to notify a specific subscriber"); + + public static final PropertyDescriptor ARN = new PropertyDescriptor.Builder() + .name("Amazon Resource Name (ARN)") + .description("The name of the resource to which notifications should be published") + .expressionLanguageSupported(true) + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final PropertyDescriptor ARN_TYPE = new PropertyDescriptor.Builder() + .name("ARN Type") + .description("The type of Amazon Resource Name that is being used.") + .expressionLanguageSupported(false) + .required(true) + .allowableValues(ARN_TYPE_TOPIC, ARN_TYPE_TARGET) + .defaultValue(ARN_TYPE_TOPIC.getValue()) + .build(); + + + + @Override + protected AmazonSNSClient createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) { + return new AmazonSNSClient(credentials, config); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sns/PutSNS.java b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sns/PutSNS.java new file mode 100644 index 0000000000..1de3251ce8 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sns/PutSNS.java @@ -0,0 +1,155 @@ +/* + * 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.processors.aws.sns; + +import java.io.ByteArrayOutputStream; +import java.nio.charset.Charset; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.processors.aws.sqs.GetSQS; +import org.apache.nifi.processors.aws.sqs.PutSQS; + +import com.amazonaws.services.sns.AmazonSNSClient; +import com.amazonaws.services.sns.model.MessageAttributeValue; +import com.amazonaws.services.sns.model.PublishRequest; + +@SupportsBatching +@SeeAlso({GetSQS.class, PutSQS.class}) +@Tags({"amazon", "aws", "sns", "topic", "put", "publish", "pubsub"}) +@CapabilityDescription("Sends the content of a FlowFile as a notification to the Amazon Simple Notification Service") +public class PutSNS extends AbstractSNSProcessor { + + public static final PropertyDescriptor CHARACTER_ENCODING = new PropertyDescriptor.Builder() + .name("Character Set") + .description("The character set in which the FlowFile's content is encoded") + .defaultValue("UTF-8") + .expressionLanguageSupported(true) + .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR) + .required(true) + .build(); + public static final PropertyDescriptor USE_JSON_STRUCTURE = new PropertyDescriptor.Builder() + .name("Use JSON Structure") + .description("If true, the contents of the FlowFile must be JSON with a top-level element named 'default'. Additional elements can be used to send different messages to different protocols. See the Amazon SNS Documentation for more information.") + .defaultValue("false") + .allowableValues("true", "false") + .required(true) + .build(); + public static final PropertyDescriptor SUBJECT = new PropertyDescriptor.Builder() + .name("E-mail Subject") + .description("The optional subject to use for any subscribers that are subscribed via E-mail") + .expressionLanguageSupported(true) + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final List properties = Collections.unmodifiableList( + Arrays.asList(ARN, ARN_TYPE, SUBJECT, REGION, ACCESS_KEY, SECRET_KEY, CREDENTAILS_FILE, TIMEOUT, + USE_JSON_STRUCTURE, CHARACTER_ENCODING) ); + + public static final int MAX_SIZE = 256 * 1024; + + @Override + protected List getSupportedPropertyDescriptors() { + return properties; + } + + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .name(propertyDescriptorName) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .required(false) + .dynamic(true) + .build(); + } + + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) { + FlowFile flowFile = session.get(); + if ( flowFile == null ) { + return; + } + + if ( flowFile.getSize() > MAX_SIZE ) { + getLogger().error("Cannot publish {} to SNS because its size exceeds Amazon SNS's limit of 256KB; routing to failure", new Object[] {flowFile}); + session.transfer(flowFile, REL_FAILURE); + return; + } + + final Charset charset = Charset.forName(context.getProperty(CHARACTER_ENCODING).evaluateAttributeExpressions(flowFile).getValue()); + + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + session.exportTo(flowFile, baos); + final String message = new String(baos.toByteArray(), charset); + + final AmazonSNSClient client = getClient(); + final PublishRequest request = new PublishRequest(); + request.setMessage(message); + + if ( context.getProperty(USE_JSON_STRUCTURE).asBoolean() ) { + request.setMessageStructure("json"); + } + + final String arn = context.getProperty(ARN).evaluateAttributeExpressions(flowFile).getValue(); + final String arnType = context.getProperty(ARN_TYPE).getValue(); + if ( arnType.equalsIgnoreCase(ARN_TYPE_TOPIC.getValue()) ) { + request.setTopicArn(arn); + } else { + request.setTargetArn(arn); + } + + final String subject = context.getProperty(SUBJECT).evaluateAttributeExpressions(flowFile).getValue(); + if ( subject != null ) { + request.setSubject(subject); + } + + for ( final Map.Entry entry : context.getProperties().entrySet() ) { + if ( entry.getKey().isDynamic() && !isEmpty(entry.getValue()) ) { + final MessageAttributeValue value = new MessageAttributeValue(); + value.setStringValue(context.getProperty(entry.getKey()).evaluateAttributeExpressions(flowFile).getValue()); + value.setDataType("String"); + request.addMessageAttributesEntry(entry.getKey().getName(), value); + } + } + + try { + client.publish(request); + session.transfer(flowFile, REL_SUCCESS); + session.getProvenanceReporter().send(flowFile, arn); + getLogger().info("Successfully published notification for {}", new Object[] {flowFile}); + } catch (final Exception e) { + getLogger().error("Failed to publish Amazon SNS message for {} due to {}", new Object[] {flowFile, e}); + session.transfer(flowFile, REL_FAILURE); + return; + } + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/AbstractSQSProcessor.java b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/AbstractSQSProcessor.java new file mode 100644 index 0000000000..2ef749f6d9 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/AbstractSQSProcessor.java @@ -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.processors.aws.sqs; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.processors.aws.AbstractAWSProcessor; + +import com.amazonaws.ClientConfiguration; +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.services.sqs.AmazonSQSClient; + +public abstract class AbstractSQSProcessor extends AbstractAWSProcessor { + + public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder() + .name("Batch Size") + .description("The maximum number of messages to send in a single network request") + .required(true) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("25") + .build(); + + public static final PropertyDescriptor QUEUE_URL = new PropertyDescriptor.Builder() + .name("Queue URL") + .description("The URL of the queue to act upon") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .required(true) + .build(); + + @Override + protected AmazonSQSClient createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) { + return new AmazonSQSClient(credentials, config); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/DeleteSQS.java b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/DeleteSQS.java new file mode 100644 index 0000000000..2416044af1 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/DeleteSQS.java @@ -0,0 +1,95 @@ +/* + * 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.processors.aws.sqs; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.util.StandardValidators; + +import com.amazonaws.services.sqs.AmazonSQSClient; +import com.amazonaws.services.sqs.model.DeleteMessageBatchRequest; +import com.amazonaws.services.sqs.model.DeleteMessageBatchRequestEntry; + +@SupportsBatching +@SeeAlso({GetSQS.class, PutSQS.class}) +@Tags({"Amazon", "AWS", "SQS", "Queue", "Delete"}) +@CapabilityDescription("Deletes a message from an Amazon Simple Queuing Service Queue") +public class DeleteSQS extends AbstractSQSProcessor { + public static final PropertyDescriptor RECEIPT_HANDLE = new PropertyDescriptor.Builder() + .name("Receipt Handle") + .description("The identifier that specifies the receipt of the message") + .expressionLanguageSupported(true) + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .defaultValue("${sqs.receipt.handle}") + .build(); + + public static final List properties = Collections.unmodifiableList( + Arrays.asList(ACCESS_KEY, SECRET_KEY, REGION, QUEUE_URL, TIMEOUT) ); + + @Override + protected List getSupportedPropertyDescriptors() { + return properties; + } + + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) { + List flowFiles = session.get(1); + if ( flowFiles.isEmpty() ) { + return; + } + + final FlowFile firstFlowFile = flowFiles.get(0); + final String queueUrl = context.getProperty(QUEUE_URL).evaluateAttributeExpressions(firstFlowFile).getValue(); + + final AmazonSQSClient client = getClient(); + final DeleteMessageBatchRequest request = new DeleteMessageBatchRequest(); + request.setQueueUrl(queueUrl); + + final List entries = new ArrayList<>(flowFiles.size()); + + for ( final FlowFile flowFile : flowFiles ) { + final DeleteMessageBatchRequestEntry entry = new DeleteMessageBatchRequestEntry(); + entry.setReceiptHandle(context.getProperty(RECEIPT_HANDLE).evaluateAttributeExpressions(flowFile).getValue()); + entries.add(entry); + } + + request.setEntries(entries); + + try { + client.deleteMessageBatch(request); + getLogger().info("Successfully deleted {} objects from SQS", new Object[] {flowFiles.size()}); + session.transfer(flowFiles, REL_SUCCESS); + } catch (final Exception e) { + getLogger().error("Failed to delete {} objects from SQS due to {}", new Object[] {flowFiles.size(), e}); + session.transfer(flowFiles, REL_FAILURE); + } + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/GetSQS.java b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/GetSQS.java new file mode 100644 index 0000000000..6c0257bf45 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/GetSQS.java @@ -0,0 +1,204 @@ +/* + * 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.processors.aws.sqs; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.io.OutputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; + +import com.amazonaws.services.sqs.AmazonSQSClient; +import com.amazonaws.services.sqs.model.DeleteMessageBatchRequest; +import com.amazonaws.services.sqs.model.DeleteMessageBatchRequestEntry; +import com.amazonaws.services.sqs.model.Message; +import com.amazonaws.services.sqs.model.MessageAttributeValue; +import com.amazonaws.services.sqs.model.ReceiveMessageRequest; +import com.amazonaws.services.sqs.model.ReceiveMessageResult; + +@SupportsBatching +@Tags({ "Amazon", "AWS", "SQS", "Queue", "Get", "Fetch", "Poll"}) +@SeeAlso({PutSQS.class, DeleteSQS.class}) +@CapabilityDescription("Fetches messages from an Amazon Simple Queuing Service Queue") +@WritesAttributes({ + @WritesAttribute(attribute="hash.value", description="The MD5 sum of the message"), + @WritesAttribute(attribute="hash.algorithm", description="MD5"), + @WritesAttribute(attribute="sqs.message.id", description="The unique identifier of the SQS message"), + @WritesAttribute(attribute="sqs.receipt.handle", description="The SQS Receipt Handle that is to be used to delete the message from the queue") +}) +public class GetSQS extends AbstractSQSProcessor { + public static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder() + .name("Character Set") + .description("The Character Set that should be used to encode the textual content of the SQS message") + .required(true) + .defaultValue("UTF-8") + .allowableValues(Charset.availableCharsets().keySet().toArray(new String[0])) + .build(); + + public static final PropertyDescriptor AUTO_DELETE = new PropertyDescriptor.Builder() + .name("Auto Delete Messages") + .description("Specifies whether the messages should be automatically deleted by the processors once they have been received.") + .required(true) + .allowableValues("true", "false") + .defaultValue("true") + .build(); + + public static final PropertyDescriptor VISIBILITY_TIMEOUT = new PropertyDescriptor.Builder() + .name("Visibility Timeout") + .description("The amount of time after a message is received but not deleted that the message is hidden from other consumers") + .expressionLanguageSupported(false) + .required(true) + .defaultValue("15 mins") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .build(); + + public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder() + .name("Batch Size") + .description("The maximum number of messages to send in a single network request") + .required(true) + .addValidator(StandardValidators.createLongValidator(1L, 10L, true)) + .defaultValue("10") + .build(); + + + public static final PropertyDescriptor STATIC_QUEUE_URL = new PropertyDescriptor.Builder() + .fromPropertyDescriptor(QUEUE_URL) + .expressionLanguageSupported(false) + .build(); + + public static final List properties = Collections.unmodifiableList( + Arrays.asList(STATIC_QUEUE_URL, AUTO_DELETE, ACCESS_KEY, SECRET_KEY, CREDENTAILS_FILE, REGION, BATCH_SIZE, TIMEOUT, CHARSET, VISIBILITY_TIMEOUT) ); + + @Override + protected List getSupportedPropertyDescriptors() { + return properties; + } + + @Override + public Set getRelationships() { + return Collections.singleton(REL_SUCCESS); + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) { + final String queueUrl = context.getProperty(STATIC_QUEUE_URL).getValue(); + + final AmazonSQSClient client = getClient(); + + final ReceiveMessageRequest request = new ReceiveMessageRequest(); + request.setAttributeNames(Collections.singleton("All")); + request.setMaxNumberOfMessages(context.getProperty(BATCH_SIZE).asInteger()); + request.setVisibilityTimeout(context.getProperty(VISIBILITY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue()); + request.setQueueUrl(queueUrl); + + final Charset charset = Charset.forName(context.getProperty(CHARSET).getValue()); + + final ReceiveMessageResult result; + try { + result = client.receiveMessage(request); + } catch (final Exception e) { + getLogger().error("Failed to receive messages from Amazon SQS due to {}", new Object[] {e}); + context.yield(); + return; + } + + final List messages = result.getMessages(); + if ( messages.isEmpty() ) { + context.yield(); + return; + } + + final boolean autoDelete = context.getProperty(AUTO_DELETE).asBoolean(); + + for ( final Message message : messages ) { + FlowFile flowFile = session.create(); + + final Map attributes = new HashMap<>(); + for ( final Map.Entry entry : message.getAttributes().entrySet() ) { + attributes.put("sqs." + entry.getKey(), entry.getValue()); + } + + for ( final Map.Entry entry : message.getMessageAttributes().entrySet() ) { + attributes.put("sqs." + entry.getKey(), entry.getValue().getStringValue()); + } + + attributes.put("hash.value", message.getMD5OfBody()); + attributes.put("hash.algorithm", "md5"); + attributes.put("sqs.message.id", message.getMessageId()); + attributes.put("sqs.receipt.handle", message.getReceiptHandle()); + + flowFile = session.putAllAttributes(flowFile, attributes); + flowFile = session.write(flowFile, new OutputStreamCallback() { + @Override + public void process(final OutputStream out) throws IOException { + out.write(message.getBody().getBytes(charset)); + } + }); + + session.transfer(flowFile, REL_SUCCESS); + session.getProvenanceReporter().receive(flowFile, queueUrl); + + getLogger().info("Successfully received {} from Amazon SQS", new Object[] {flowFile}); + } + + if ( autoDelete ) { + // If we want to auto-delete messages, we must fist commit the session to ensure that the data + // is persisted in NiFi's repositories. + session.commit(); + + final DeleteMessageBatchRequest deleteRequest = new DeleteMessageBatchRequest(); + deleteRequest.setQueueUrl(queueUrl); + final List deleteRequestEntries = new ArrayList<>(); + for ( final Message message : messages ) { + final DeleteMessageBatchRequestEntry entry = new DeleteMessageBatchRequestEntry(); + entry.setId(message.getMessageId()); + entry.setReceiptHandle(message.getReceiptHandle()); + deleteRequestEntries.add(entry); + } + + deleteRequest.setEntries(deleteRequestEntries); + + try { + client.deleteMessageBatch(deleteRequest); + } catch (final Exception e) { + getLogger().error("Received {} messages from Amazon SQS but failed to delete the messages; these messages may be duplicated. Reason for deletion failure: {}", new Object[] {messages.size(), e}); + } + } + + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/PutSQS.java b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/PutSQS.java new file mode 100644 index 0000000000..81268fe2cd --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/sqs/PutSQS.java @@ -0,0 +1,147 @@ +/* + * 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.processors.aws.sqs; + +import java.io.ByteArrayOutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.annotation.behavior.DynamicProperty; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.util.StandardValidators; + +import com.amazonaws.services.sqs.AmazonSQSClient; +import com.amazonaws.services.sqs.model.MessageAttributeValue; +import com.amazonaws.services.sqs.model.SendMessageBatchRequest; +import com.amazonaws.services.sqs.model.SendMessageBatchRequestEntry; + + +@SupportsBatching +@Tags({"Amazon", "AWS", "SQS", "Queue", "Put", "Publish"}) +@SeeAlso({GetSQS.class, DeleteSQS.class}) +@CapabilityDescription("Publishes a message to an Amazon Simple Queuing Service Queue") +@DynamicProperty(name="The name of a Message Attribute to add to the message", value="The value of the Message Attribute", + description="Allows the user to add key/value pairs as Message Attributes by adding a property whose name will become the name of " + + "the Message Attribute and value will become the value of the Message Attribute", supportsExpressionLanguage=true) +public class PutSQS extends AbstractSQSProcessor { + + public static final PropertyDescriptor DELAY = new PropertyDescriptor.Builder() + .name("Delay") + .description("The amount of time to delay the message before it becomes available to consumers") + .required(true) + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .defaultValue("0 secs") + .build(); + + public static final List properties = Collections.unmodifiableList( + Arrays.asList(QUEUE_URL, ACCESS_KEY, SECRET_KEY, CREDENTAILS_FILE, REGION, DELAY, TIMEOUT) ); + + private volatile List userDefinedProperties = Collections.emptyList(); + + @Override + protected List getSupportedPropertyDescriptors() { + return properties; + } + + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .name(propertyDescriptorName) + .expressionLanguageSupported(true) + .required(false) + .dynamic(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + } + + @OnScheduled + public void setup(final ProcessContext context) { + userDefinedProperties = new ArrayList<>(); + for ( final PropertyDescriptor descriptor : context.getProperties().keySet() ) { + if ( descriptor.isDynamic() ) { + userDefinedProperties.add(descriptor); + } + } + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) { + FlowFile flowFile = session.get(); + if ( flowFile == null ) { + return; + } + + final long startNanos = System.nanoTime(); + final AmazonSQSClient client = getClient(); + final SendMessageBatchRequest request = new SendMessageBatchRequest(); + final String queueUrl = context.getProperty(QUEUE_URL).evaluateAttributeExpressions(flowFile).getValue(); + request.setQueueUrl(queueUrl); + + final Set entries = new HashSet<>(); + + final SendMessageBatchRequestEntry entry = new SendMessageBatchRequestEntry(); + entry.setId(flowFile.getAttribute("uuid")); + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + session.exportTo(flowFile, baos); + final String flowFileContent = baos.toString(); + entry.setMessageBody(flowFileContent); + + final Map messageAttributes = new HashMap<>(); + + for ( final PropertyDescriptor descriptor : userDefinedProperties ) { + final MessageAttributeValue mav = new MessageAttributeValue(); + mav.setDataType("String"); + mav.setStringValue(context.getProperty(descriptor).evaluateAttributeExpressions(flowFile).getValue()); + messageAttributes.put(descriptor.getName(), mav); + } + + entry.setMessageAttributes(messageAttributes); + entry.setDelaySeconds(context.getProperty(DELAY).asTimePeriod(TimeUnit.SECONDS).intValue()); + entries.add(entry); + + request.setEntries(entries); + + try { + client.sendMessageBatch(request); + } catch (final Exception e) { + getLogger().error("Failed to send messages to Amazon SQS due to {}; routing to failure", new Object[] {e}); + session.transfer(flowFile, REL_FAILURE); + return; + } + + getLogger().info("Successfully published message to Amazon SQS for {}", new Object[] {flowFile}); + session.transfer(flowFile, REL_SUCCESS); + final long transmissionMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos); + session.getProvenanceReporter().send(flowFile, queueUrl, transmissionMillis); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor new file mode 100644 index 0000000000..4f2405c91d --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -0,0 +1,20 @@ +# 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. +org.apache.nifi.processors.aws.s3.FetchS3Object +org.apache.nifi.processors.aws.s3.PutS3Object +org.apache.nifi.processors.aws.sns.PutSNS +org.apache.nifi.processors.aws.sqs.GetSQS +org.apache.nifi.processors.aws.sqs.PutSQS +org.apache.nifi.processors.aws.sqs.DeleteSQS diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestFetchS3Object.java b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestFetchS3Object.java new file mode 100644 index 0000000000..40f951538b --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestFetchS3Object.java @@ -0,0 +1,44 @@ +package org.apache.nifi.processors.aws.s3; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Ignore; +import org.junit.Test; + +@Ignore("For local testing only - interacts with S3 so the credentials file must be configured and all necessary buckets created") +public class TestFetchS3Object { + private final String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties"; + + @Test + public void testGet() throws IOException { + final TestRunner runner = TestRunners.newTestRunner(new FetchS3Object()); + runner.setProperty(FetchS3Object.BUCKET, "anonymous-test-bucket-00000000"); + runner.setProperty(FetchS3Object.CREDENTAILS_FILE, CREDENTIALS_FILE); + runner.setProperty(FetchS3Object.KEY, "folder/1.txt"); + + final Map attrs = new HashMap<>(); + attrs.put("start", "0"); + + runner.enqueue(new byte[0], attrs); + runner.run(1); + + runner.assertAllFlowFilesTransferred(FetchS3Object.REL_SUCCESS, 1); + final List ffs = runner.getFlowFilesForRelationship(FetchS3Object.REL_SUCCESS); + final MockFlowFile out = ffs.iterator().next(); + + final byte[] expectedBytes = Files.readAllBytes(Paths.get("src/test/resources/hello.txt")); + out.assertContentEquals(new String(expectedBytes)); + for ( final Map.Entry entry : out.getAttributes().entrySet() ) { + System.out.println(entry.getKey() + " : " + entry.getValue()); + } + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestPutS3Object.java b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestPutS3Object.java new file mode 100644 index 0000000000..0a019f3328 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/s3/TestPutS3Object.java @@ -0,0 +1,87 @@ +package org.apache.nifi.processors.aws.s3; + +import java.io.IOException; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.Map; + +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; + +import com.amazonaws.services.s3.model.StorageClass; + +@Ignore("For local testing only - interacts with S3 so the credentials file must be configured and all necessary buckets created") +public class TestPutS3Object { + + private final String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties"; + + @Test + public void testSimplePut() throws IOException { + final TestRunner runner = TestRunners.newTestRunner(new PutS3Object()); + runner.setProperty(PutS3Object.CREDENTAILS_FILE, CREDENTIALS_FILE); + runner.setProperty(PutS3Object.BUCKET, "test-bucket-00000000-0000-0000-0000-123456789012"); + runner.setProperty(PutS3Object.EXPIRATION_RULE_ID, "Expire Quickly"); + Assert.assertTrue( runner.setProperty("x-custom-prop", "hello").isValid() ); + + for (int i=0; i < 3; i++) { + final Map attrs = new HashMap<>(); + attrs.put("filename", String.valueOf(i) + ".txt"); + runner.enqueue(Paths.get("src/test/resources/hello.txt"), attrs); + } + runner.run(3); + + runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 3); + } + + @Test + public void testPutInFolder() throws IOException { + final TestRunner runner = TestRunners.newTestRunner(new PutS3Object()); + runner.setProperty(PutS3Object.BUCKET, "test-bucket-00000000-0000-0000-0000-123456789012"); + runner.setProperty(PutS3Object.CREDENTAILS_FILE, CREDENTIALS_FILE); + runner.setProperty(PutS3Object.EXPIRATION_RULE_ID, "Expire Quickly"); + Assert.assertTrue( runner.setProperty("x-custom-prop", "hello").isValid() ); + + final Map attrs = new HashMap<>(); + attrs.put("filename", "folder/1.txt"); + runner.enqueue(Paths.get("src/test/resources/hello.txt"), attrs); + runner.run(); + + runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1); + } + + + @Test + public void testStorageClass() throws IOException { + final TestRunner runner = TestRunners.newTestRunner(new PutS3Object()); + runner.setProperty(PutS3Object.BUCKET, "test-bucket-00000000-0000-0000-0000-123456789012"); + runner.setProperty(PutS3Object.CREDENTAILS_FILE, CREDENTIALS_FILE); + runner.setProperty(PutS3Object.STORAGE_CLASS, StorageClass.ReducedRedundancy.name()); + Assert.assertTrue( runner.setProperty("x-custom-prop", "hello").isValid() ); + + final Map attrs = new HashMap<>(); + attrs.put("filename", "folder/2.txt"); + runner.enqueue(Paths.get("src/test/resources/hello.txt"), attrs); + runner.run(); + + runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1); + } + + @Test + public void testPermissions() throws IOException { + final TestRunner runner = TestRunners.newTestRunner(new PutS3Object()); + runner.setProperty(PutS3Object.BUCKET, "test-bucket-00000000-0000-0000-0000-123456789012"); + runner.setProperty(PutS3Object.CREDENTAILS_FILE, CREDENTIALS_FILE); + runner.setProperty(PutS3Object.FULL_CONTROL_USER_LIST, "28545acd76c35c7e91f8409b95fd1aa0c0914bfa1ac60975d9f48bc3c5e090b5"); + + final Map attrs = new HashMap<>(); + attrs.put("filename", "folder/4.txt"); + runner.enqueue(Paths.get("src/test/resources/hello.txt"), attrs); + runner.run(); + + runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sns/TestPutSNS.java b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sns/TestPutSNS.java new file mode 100644 index 0000000000..b505622ca0 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sns/TestPutSNS.java @@ -0,0 +1,34 @@ +package org.apache.nifi.processors.aws.sns; + +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.Map; + +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Ignore; +import org.junit.Test; + +@Ignore("For local testing only - interacts with S3 so the credentials file must be configured and all necessary buckets created") +public class TestPutSNS { + private final String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties"; + + @Test + public void testPublish() throws IOException { + final TestRunner runner = TestRunners.newTestRunner(new PutSNS()); + runner.setProperty(PutSNS.CREDENTAILS_FILE, CREDENTIALS_FILE); + runner.setProperty(PutSNS.ARN, "arn:aws:sns:us-west-2:100515378163:test-topic-1"); + assertTrue( runner.setProperty("DynamicProperty", "hello!").isValid() ); + + final Map attrs = new HashMap<>(); + attrs.put("filename", "1.txt"); + runner.enqueue(Paths.get("src/test/resources/hello.txt"), attrs); + runner.run(); + + runner.assertAllFlowFilesTransferred(PutSNS.REL_SUCCESS, 1); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sqs/TestGetSQS.java b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sqs/TestGetSQS.java new file mode 100644 index 0000000000..de4a5d9cc2 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sqs/TestGetSQS.java @@ -0,0 +1,32 @@ +package org.apache.nifi.processors.aws.sqs; + +import java.util.List; + +import org.apache.nifi.processors.aws.sns.PutSNS; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Ignore; +import org.junit.Test; + +@Ignore("For local testing only - interacts with S3 so the credentials file must be configured and all necessary buckets created") +public class TestGetSQS { + private final String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties"; + + @Test + public void testSimpleGet() { + final TestRunner runner = TestRunners.newTestRunner(new GetSQS()); + runner.setProperty(PutSNS.CREDENTAILS_FILE, CREDENTIALS_FILE); + runner.setProperty(GetSQS.TIMEOUT, "30 secs"); + runner.setProperty(GetSQS.QUEUE_URL, "https://sqs.us-west-2.amazonaws.com/100515378163/test-queue-000000000"); + + runner.run(1); + + final List flowFiles = runner.getFlowFilesForRelationship(GetSQS.REL_SUCCESS); + for ( final MockFlowFile mff : flowFiles ) { + System.out.println(mff.getAttributes()); + System.out.println(new String(mff.toByteArray())); + } + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sqs/TestPutSQS.java b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sqs/TestPutSQS.java new file mode 100644 index 0000000000..a90a4ce339 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/sqs/TestPutSQS.java @@ -0,0 +1,35 @@ +package org.apache.nifi.processors.aws.sqs; + +import java.io.IOException; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.Map; + +import org.apache.nifi.processors.aws.sns.PutSNS; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; + +@Ignore("For local testing only - interacts with S3 so the credentials file must be configured and all necessary buckets created") +public class TestPutSQS { + private final String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties"; + + @Test + public void testSimplePut() throws IOException { + final TestRunner runner = TestRunners.newTestRunner(new PutSQS()); + runner.setProperty(PutSNS.CREDENTAILS_FILE, CREDENTIALS_FILE); + runner.setProperty(PutSQS.TIMEOUT, "30 secs"); + runner.setProperty(PutSQS.QUEUE_URL, "https://sqs.us-west-2.amazonaws.com/100515378163/test-queue-000000000"); + Assert.assertTrue( runner.setProperty("x-custom-prop", "hello").isValid() ); + + final Map attrs = new HashMap<>(); + attrs.put("filename", "1.txt"); + runner.enqueue(Paths.get("src/test/resources/hello.txt"), attrs); + runner.run(1); + + runner.assertAllFlowFilesTransferred(PutSQS.REL_SUCCESS, 1); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/resources/hello.txt b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/resources/hello.txt new file mode 100644 index 0000000000..ee13cb732d --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/resources/hello.txt @@ -0,0 +1 @@ +Hello, World!! \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-aws-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-aws-bundle/pom.xml new file mode 100644 index 0000000000..117d7dd1c4 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-aws-bundle/pom.xml @@ -0,0 +1,43 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-nar-bundles + 0.1.0-incubating-SNAPSHOT + + + nifi-aws-bundle + pom + + + nifi-aws-processors + nifi-aws-nar + + + + + + com.amazonaws + aws-java-sdk + 1.9.24 + + + + + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java index 5d6d2226d8..5d5d498ee5 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java @@ -48,12 +48,12 @@ public interface ActionDAO { /** * Finds the previous values for the specified property in the specified - * processor. Returns empty list if there are none. + * component. Returns empty list if there are none. * - * @param processorId + * @param componentId * @return */ - Map> getPreviousValues(String processorId); + Map> getPreviousValues(String componentId); /** * Finds the specified action. diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java index 4b89655c47..34623f4248 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java @@ -32,7 +32,7 @@ import org.apache.nifi.action.Action; import org.apache.nifi.action.Component; import org.apache.nifi.action.Operation; import org.apache.nifi.action.component.details.ComponentDetails; -import org.apache.nifi.action.component.details.ProcessorDetails; +import org.apache.nifi.action.component.details.ExtensionDetails; import org.apache.nifi.action.component.details.RemoteProcessGroupDetails; import org.apache.nifi.action.details.ActionDetails; import org.apache.nifi.action.details.ConfigureDetails; @@ -70,7 +70,7 @@ public class StandardActionDAO implements ActionDAO { // ----------------- // component details // ----------------- - private static final String INSERT_PROCESSOR_DETAILS = "INSERT INTO PROCESSOR_DETAILS (" + private static final String INSERT_EXTENSION_DETAILS = "INSERT INTO PROCESSOR_DETAILS (" + "ACTION_ID, TYPE" + ") VALUES (" + "?, " @@ -145,7 +145,7 @@ public class StandardActionDAO implements ActionDAO { // ----------------- // component details // ----------------- - private static final String SELECT_PROCESSOR_DETAILS_FOR_ACTION = "SELECT * FROM PROCESSOR_DETAILS WHERE ACTION_ID = ?"; + private static final String SELECT_EXTENSION_DETAILS_FOR_ACTION = "SELECT * FROM PROCESSOR_DETAILS WHERE ACTION_ID = ?"; private static final String SELECT_REMOTE_PROCESS_GROUP_DETAILS_FOR_ACTION = "SELECT * FROM REMOTE_PROCESS_GROUP_DETAILS WHERE ACTION_ID = ?"; @@ -179,8 +179,8 @@ public class StandardActionDAO implements ActionDAO { + "ORDER BY A.ACTION_TIMESTAMP DESC " + "LIMIT 4"; - private Connection connection; - private Map columnMap; + private final Connection connection; + private final Map columnMap; public StandardActionDAO(Connection connection) { this.connection = connection; @@ -233,8 +233,8 @@ public class StandardActionDAO implements ActionDAO { // determine the type of component ComponentDetails componentDetails = action.getComponentDetails(); - if (componentDetails instanceof ProcessorDetails) { - createProcessorDetails(action.getId(), (ProcessorDetails) componentDetails); + if (componentDetails instanceof ExtensionDetails) { + createExtensionDetails(action.getId(), (ExtensionDetails) componentDetails); } else if (componentDetails instanceof RemoteProcessGroupDetails) { createRemoteProcessGroupDetails(action.getId(), (RemoteProcessGroupDetails) componentDetails); } @@ -260,26 +260,26 @@ public class StandardActionDAO implements ActionDAO { } /** - * Persists the processor details. + * Persists the extension details. * * @param actionId - * @param processorDetails + * @param extensionDetails * @throws DataAccessException */ - private void createProcessorDetails(int actionId, ProcessorDetails processorDetails) throws DataAccessException { + private void createExtensionDetails(int actionId, ExtensionDetails extensionDetails) throws DataAccessException { PreparedStatement statement = null; try { - // obtain a statement to insert to the processor action table - statement = connection.prepareStatement(INSERT_PROCESSOR_DETAILS); + // obtain a statement to insert to the extension action table + statement = connection.prepareStatement(INSERT_EXTENSION_DETAILS); statement.setInt(1, actionId); - statement.setString(2, StringUtils.left(processorDetails.getType(), 1000)); + statement.setString(2, StringUtils.left(extensionDetails.getType(), 1000)); // insert the action int updateCount = statement.executeUpdate(); // ensure the operation completed successfully if (updateCount != 1) { - throw new DataAccessException("Unable to insert processor details."); + throw new DataAccessException("Unable to insert extension details."); } } catch (SQLException sqle) { throw new DataAccessException(sqle); @@ -601,8 +601,8 @@ public class StandardActionDAO implements ActionDAO { // get the component details if appropriate ComponentDetails componentDetails = null; - if (Component.Processor.equals(component)) { - componentDetails = getProcessorDetails(actionId); + if (Component.Processor.equals(component) || Component.ControllerService.equals(component) || Component.ReportingTask.equals(component)) { + componentDetails = getExtensionDetails(actionId); } else if (Component.RemoteProcessGroup.equals(component)) { componentDetails = getRemoteProcessGroupDetails(actionId); } @@ -675,8 +675,8 @@ public class StandardActionDAO implements ActionDAO { // get the component details if appropriate ComponentDetails componentDetails = null; - if (Component.Processor.equals(component)) { - componentDetails = getProcessorDetails(actionId); + if (Component.Processor.equals(component) || Component.ControllerService.equals(component) || Component.ReportingTask.equals(component)) { + componentDetails = getExtensionDetails(actionId); } else if (Component.RemoteProcessGroup.equals(component)) { componentDetails = getRemoteProcessGroupDetails(actionId); } @@ -713,19 +713,19 @@ public class StandardActionDAO implements ActionDAO { } /** - * Loads the specified processor details. + * Loads the specified extension details. * * @param actionId * @return * @throws DataAccessException */ - private ProcessorDetails getProcessorDetails(Integer actionId) throws DataAccessException { - ProcessorDetails processorDetails = null; + private ExtensionDetails getExtensionDetails(Integer actionId) throws DataAccessException { + ExtensionDetails extensionDetails = null; PreparedStatement statement = null; ResultSet rs = null; try { // create the statement - statement = connection.prepareStatement(SELECT_PROCESSOR_DETAILS_FOR_ACTION); + statement = connection.prepareStatement(SELECT_EXTENSION_DETAILS_FOR_ACTION); statement.setInt(1, actionId); // execute the query @@ -733,8 +733,8 @@ public class StandardActionDAO implements ActionDAO { // ensure results if (rs.next()) { - processorDetails = new ProcessorDetails(); - processorDetails.setType(rs.getString("TYPE")); + extensionDetails = new ExtensionDetails(); + extensionDetails.setType(rs.getString("TYPE")); } } catch (SQLException sqle) { throw new DataAccessException(sqle); @@ -743,7 +743,7 @@ public class StandardActionDAO implements ActionDAO { RepositoryUtils.closeQuietly(statement); } - return processorDetails; + return extensionDetails; } /** @@ -931,7 +931,7 @@ public class StandardActionDAO implements ActionDAO { } @Override - public Map> getPreviousValues(String processorId) { + public Map> getPreviousValues(String componentId) { Map> previousValues = new LinkedHashMap<>(); PreparedStatement statement = null; @@ -939,7 +939,7 @@ public class StandardActionDAO implements ActionDAO { try { // create the statement statement = connection.prepareStatement(SELECT_PREVIOUSLY_CONFIGURED_FIELDS); - statement.setString(1, processorId); + statement.setString(1, componentId); // execute the query rs = statement.executeQuery(); @@ -947,7 +947,7 @@ public class StandardActionDAO implements ActionDAO { // ensure results while (rs.next()) { final String property = rs.getString("NAME"); - previousValues.put(property, getPreviousValuesForProperty(processorId, property)); + previousValues.put(property, getPreviousValuesForProperty(componentId, property)); } } catch (SQLException sqle) { throw new DataAccessException(sqle); @@ -959,7 +959,7 @@ public class StandardActionDAO implements ActionDAO { return previousValues; } - private List getPreviousValuesForProperty(final String processorId, final String property) { + private List getPreviousValuesForProperty(final String componentId, final String property) { List previousValues = new ArrayList<>(); PreparedStatement statement = null; @@ -967,7 +967,7 @@ public class StandardActionDAO implements ActionDAO { try { // create the statement statement = connection.prepareStatement(SELECT_PREVIOUS_VALUES); - statement.setString(1, processorId); + statement.setString(1, componentId); statement.setString(2, property); // execute the query diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/AuditService.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/AuditService.java index 0843bd804a..7ca4e0618a 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/AuditService.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/AuditService.java @@ -40,12 +40,12 @@ public interface AuditService { /** * Finds the previous values for the specified property in the specified - * processor. Returns null if there are none. + * component. Returns null if there are none. * - * @param processorId + * @param componentId * @return */ - Map> getPreviousValues(String processorId); + Map> getPreviousValues(String componentId); /** * Get the actions within the given date range. diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java index 5ce663eb39..569439baa9 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java @@ -28,16 +28,16 @@ import org.apache.nifi.history.PreviousValue; */ public class GetPreviousValues implements AdministrationAction>> { - private final String processorId; + private final String componentId; - public GetPreviousValues(String processorId) { - this.processorId = processorId; + public GetPreviousValues(String componentId) { + this.componentId = componentId; } @Override public Map> execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) { ActionDAO actionDao = daoFactory.getActionDAO(); - return actionDao.getPreviousValues(processorId); + return actionDao.getPreviousValues(componentId); } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java index 127f1df813..721e6b27ec 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java @@ -81,7 +81,7 @@ public class StandardAuditService implements AuditService { } @Override - public Map> getPreviousValues(String processorId) { + public Map> getPreviousValues(String componentId) { Transaction transaction = null; Map> previousValues = null; @@ -91,7 +91,7 @@ public class StandardAuditService implements AuditService { transaction = transactionBuilder.start(); // seed the accounts - GetPreviousValues getActions = new GetPreviousValues(processorId); + GetPreviousValues getActions = new GetPreviousValues(componentId); previousValues = transaction.execute(getActions); // commit the transaction diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentHistoryDTO.java similarity index 77% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentHistoryDTO.java index 2741116a54..3bdbe2814e 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentHistoryDTO.java @@ -20,29 +20,29 @@ import java.util.Map; import javax.xml.bind.annotation.XmlType; /** - * History of a processor's properties. + * History of a component's properties. */ -@XmlType(name = "processorHistory") -public class ProcessorHistoryDTO { +@XmlType(name = "componentHistory") +public class ComponentHistoryDTO { - private String processorId; + private String componentId; private Map propertyHistory; /** - * The processor id. + * The component id. * * @return */ - public String getProcessorId() { - return processorId; + public String getComponentId() { + return componentId; } - public void setProcessorId(String processorId) { - this.processorId = processorId; + public void setComponentId(String componentId) { + this.componentId = componentId; } /** - * The history for this processors properties. + * The history for this components properties. * * @return */ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.java index b916025fd1..190cb47fe2 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.java @@ -16,7 +16,10 @@ */ package org.apache.nifi.web.api.dto; +import java.util.Date; import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; /** * Details for the controller configuration. @@ -32,6 +35,7 @@ public class ControllerConfigurationDTO { private Long autoRefreshIntervalSeconds; private Boolean siteToSiteSecure; + private Date currentTime; private Integer timeOffset; private String contentViewerUrl; @@ -117,6 +121,20 @@ public class ControllerConfigurationDTO { this.siteToSiteSecure = siteToSiteSecure; } + /** + * The current time on the server. + * + * @return + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getCurrentTime() { + return currentTime; + } + + public void setCurrentTime(Date currentTime) { + this.currentTime = currentTime; + } + /** * The time offset of the server. * diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceDTO.java new file mode 100644 index 0000000000..75d18a2caa --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceDTO.java @@ -0,0 +1,190 @@ +/* + * 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 java.util.Collection; +import java.util.Map; +import java.util.Set; +import javax.xml.bind.annotation.XmlType; + +/** + * A Controller Service that can be shared by other components + */ +@XmlType(name = "controllerService") +public class ControllerServiceDTO extends NiFiComponentDTO { + + private String name; + private String type; + private String comments; + private String availability; + private String state; + + private Map properties; + private Map descriptors; + + private String customUiUrl; + private String annotationData; + + private Set referencingComponents; + + private Collection validationErrors; + + /** + * The controller service name. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The controller service type. + * + * @return + */ + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + + /** + * The comment for the Controller Service + * @return + */ + public String getComments() { + return comments; + } + + public void setComments(String comments) { + this.comments = comments; + } + + /** + * Where this service is available. Possible values are NCM, NODE. + * + * @return + */ + public String getAvailability() { + return availability; + } + + public void setAvailability(String availability) { + this.availability = availability; + } + + /** + * The state of this controller service. Possible values are ENABLED, ENABLING, DISABLED, DISABLING. + * @return + */ + public String getState() { + return state; + } + + public void setState(String state) { + this.state = state; + } + + /** + * The controller service properties. + * + * @return + */ + public Map getProperties() { + return properties; + } + + public void setProperties(Map properties) { + this.properties = properties; + } + + /** + * The descriptors for the controller service properties. + * + * @return + */ + public Map getDescriptors() { + return descriptors; + } + + public void setDescriptors(Map descriptors) { + this.descriptors = descriptors; + } + + /** + * Returns the URL for this controller services custom configuration UI + * if applicable. Null otherwise. + * + * @return + */ + public String getCustomUiUrl() { + return customUiUrl; + } + + public void setCustomUiUrl(String customUiUrl) { + this.customUiUrl = customUiUrl; + } + + /** + * The annotation data for this controller service. + * + * @return + */ + public String getAnnotationData() { + return annotationData; + } + + public void setAnnotationData(String annotationData) { + this.annotationData = annotationData; + } + + /** + * All components referencing this controller service. + * + * @return + */ + public Set getReferencingComponents() { + return referencingComponents; + } + + public void setReferencingComponents(Set referencingComponents) { + this.referencingComponents = referencingComponents; + } + + /** + * Gets the validation errors from this controller service. These validation errors + * represent the problems with the controller service that must be resolved before it + * can be enabled. + * + * @return The validation errors + */ + public Collection getValidationErrors() { + return validationErrors; + } + + public void setValidationErrors(Collection validationErrors) { + this.validationErrors = validationErrors; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceReferencingComponentDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceReferencingComponentDTO.java new file mode 100644 index 0000000000..7fc57ffc0b --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceReferencingComponentDTO.java @@ -0,0 +1,207 @@ +/* + * 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 java.util.Collection; +import java.util.Map; +import java.util.Set; +import javax.xml.bind.annotation.XmlType; + +/** + * A component referencing a controller service. This can either be another + * controller service or a processor. Depending on the type of component + * different properties may be set. + */ +@XmlType(name = "controllerServiceReferencingComponent") +public class ControllerServiceReferencingComponentDTO { + private String groupId; + private String id; + private String name; + private String type; + private String state; + + private Map properties; + private Map descriptors; + + private Collection validationErrors; + + private String referenceType; + private Integer activeThreadCount; + + private Boolean referenceCycle; + private Set referencingComponents; + + /** + * Group id for this component referencing a controller service. If this + * component is another service, this field is blank. + * + * @return + */ + public String getGroupId() { + return groupId; + } + + public void setGroupId(String groupId) { + this.groupId = groupId; + } + + /** + * The id for this component referencing a controller service. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The name for this component referencing a controller service. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The type for this component referencing a controller service. + * + * @return + */ + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + /** + * The state of the processor referencing a controller service. If this + * component is another service, this field is blank. + * + * @return + */ + public String getState() { + return state; + } + + public void setState(String state) { + this.state = state; + } + + /** + * The type of reference this is (Processor, ControllerService, or ReportingTask). + * @return + */ + public String getReferenceType() { + return referenceType; + } + + public void setReferenceType(String referenceType) { + this.referenceType = referenceType; + } + + /** + * The component properties. + * + * @return + */ + public Map getProperties() { + return properties; + } + + public void setProperties(Map properties) { + this.properties = properties; + } + + /** + * The descriptors for the components properties. + * + * @return + */ + public Map getDescriptors() { + return descriptors; + } + + public void setDescriptors(Map descriptors) { + this.descriptors = descriptors; + } + + /** + * Any validation error associated with this component. + * + * @return + */ + public Collection getValidationErrors() { + return validationErrors; + } + + public void setValidationErrors(Collection validationErrors) { + this.validationErrors = validationErrors; + } + + /** + * The active thread count for the referencing component. + * + * @return + */ + public Integer getActiveThreadCount() { + return activeThreadCount; + } + + public void setActiveThreadCount(Integer activeThreadCount) { + this.activeThreadCount = activeThreadCount; + } + + /** + * If this referencing component represents a ControllerService, these + * are the components that reference it. + * + * @return + */ + public Set getReferencingComponents() { + return referencingComponents; + } + + public void setReferencingComponents(Set referencingComponents) { + this.referencingComponents = referencingComponents; + } + + /** + * If this referencing component represents a ControllerService, this indicates + * whether it has already been represented in this hierarchy. + * + * @return + */ + public Boolean getReferenceCycle() { + return referenceCycle; + } + + public void setReferenceCycle(Boolean referenceCycle) { + this.referenceCycle = referenceCycle; + } + +} \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.java index 5a2d78905c..6e4aeb9b9b 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.java @@ -20,8 +20,7 @@ import java.util.Set; import javax.xml.bind.annotation.XmlType; /** - * Class used for providing documentation of a specified type that may be - * instantiated. + * Class used for providing documentation of a specified type. */ @XmlType(name = "documentedType") public class DocumentedTypeDTO { @@ -57,7 +56,7 @@ public class DocumentedTypeDTO { } /** - * The tags associated with this type + * The tags associated with this type. * * @return */ @@ -68,4 +67,5 @@ public class DocumentedTypeDTO { public void setTags(final Set tags) { this.tags = tags; } + } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.java index 61c3c33762..47a6871b9d 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.java @@ -34,7 +34,8 @@ public class FlowSnippetDTO { private Set connections = new LinkedHashSet<>(); private Set labels = new LinkedHashSet<>(); private Set funnels = new LinkedHashSet<>(); - + private Set controllerServices = new LinkedHashSet<>(); + /** * The connections in this flow snippet. * @@ -138,4 +139,16 @@ public class FlowSnippetDTO { public void setRemoteProcessGroups(Set remoteProcessGroups) { this.remoteProcessGroups = remoteProcessGroups; } + + /** + * Returns the Controller Services in this flow snippet + * @return + */ + public Set getControllerServices() { + return controllerServices; + } + + public void setControllerServices(Set controllerServices) { + this.controllerServices = controllerServices; + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.java index e3c844555b..2829287b66 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.java @@ -55,7 +55,7 @@ public class NiFiComponentDTO { } /** - * The id for the parent group of this component. + * The id for the parent group of this component if applicable, null otherwise. * * @return */ @@ -85,7 +85,7 @@ public class NiFiComponentDTO { } /** - * The position of this component in the UI. + * The position of this component in the UI if applicable, null otherwise. * * @return The position */ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java index 1481b0f9d7..63ed005026 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java @@ -54,10 +54,10 @@ public class ProcessorConfigDTO { } /** - * The amount of time that should elapse between task executions. This will - * not affect currently scheduled tasks. + * The frequency with which to schedule the processor. The format of the value will + * depend on the value of {@link #getSchedulingStrategy()}. * - * @return The scheduling period in seconds + * @return The scheduling period */ public String getSchedulingPeriod() { return schedulingPeriod; @@ -207,7 +207,8 @@ public class ProcessorConfigDTO { } /** - * Whether of not this processor has a custom UI. + * Returns the URL for this processors custom configuration UI + * if applicable. Null otherwise. * * @return */ @@ -274,213 +275,4 @@ public class ProcessorConfigDTO { this.defaultSchedulingPeriod = defaultSchedulingPeriod; } - /** - * The allowable values for a property with a constrained set of options. - */ - @XmlType(name = "allowableValue") - public static class AllowableValueDTO { - - private String displayName; - private String value; - private String description; - - /** - * Returns the human-readable value that is allowed for this - * PropertyDescriptor - * - * @return - */ - public String getDisplayName() { - return displayName; - } - - public void setDisplayName(String displayName) { - this.displayName = displayName; - } - - /** - * Returns the value for this allowable value. - * - * @return - */ - public String getValue() { - return value; - } - - public void setValue(String value) { - this.value = value; - } - - /** - * Returns a description of this Allowable Value, or null - * if no description is given - * - * @return - */ - public String getDescription() { - return description; - } - - public void setDescription(String description) { - this.description = description; - } - - @Override - public boolean equals(final Object obj) { - if (obj == this) { - return true; - } - - if (!(obj instanceof AllowableValueDTO)) { - return false; - } - - final AllowableValueDTO other = (AllowableValueDTO) obj; - return (this.value.equals(other.getValue())); - } - - @Override - public int hashCode() { - return 23984731 + 17 * value.hashCode(); - } - } - - /** - * A description of a processor property. - */ - @XmlType(name = "propertyDescriptor") - public static class PropertyDescriptorDTO { - - private String name; - private String displayName; - private String description; - private String defaultValue; - private Set allowableValues; - private boolean required; - private boolean sensitive; - private boolean dynamic; - private boolean supportsEl; - - /** - * The set of allowable values for this property. If empty then the - * allowable values are not constrained. - * - * @return - */ - public Set getAllowableValues() { - return allowableValues; - } - - public void setAllowableValues(Set allowableValues) { - this.allowableValues = allowableValues; - } - - /** - * The default value for this property. - * - * @return - */ - public String getDefaultValue() { - return defaultValue; - } - - public void setDefaultValue(String defaultValue) { - this.defaultValue = defaultValue; - } - - /** - * And explanation of the meaning of the given property. This - * description is meant to be displayed to a user or simply provide a - * mechanism of documenting intent. - * - * @return - */ - public String getDescription() { - return description; - } - - public void setDescription(String description) { - this.description = description; - } - - /** - * The property name. - * - * @return - */ - public String getName() { - return name; - } - - public void setName(String name) { - this.name = name; - } - - /** - * The human-readable name to display to users. - * - * @return - */ - public String getDisplayName() { - return displayName; - } - - public void setDisplayName(String displayName) { - this.displayName = displayName; - } - - /** - * Determines whether the property is required for this processor. - * - * @return - */ - public boolean isRequired() { - return required; - } - - public void setRequired(boolean required) { - this.required = required; - } - - /** - * Indicates that the value for this property should be considered - * sensitive and protected whenever stored or represented. - * - * @return - */ - public boolean isSensitive() { - return sensitive; - } - - public void setSensitive(boolean sensitive) { - this.sensitive = sensitive; - } - - /** - * Indicates whether this property is dynamic. - * - * @return - */ - public boolean isDynamic() { - return dynamic; - } - - public void setDynamic(boolean dynamic) { - this.dynamic = dynamic; - } - - /** - * Specifies whether or not this property support expression language. - * - * @return - */ - public boolean getSupportsEl() { - return supportsEl; - } - - public void setSupportsEl(boolean supportsEl) { - this.supportsEl = supportsEl; - } - } - } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java new file mode 100644 index 0000000000..ecde255022 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java @@ -0,0 +1,244 @@ +/* + * 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 java.util.List; +import javax.xml.bind.annotation.XmlType; + +/** + * A description of a property. + */ +@XmlType(name = "propertyDescriptor") +public class PropertyDescriptorDTO { + + private String name; + private String displayName; + private String description; + private String defaultValue; + private List allowableValues; + private boolean required; + private boolean sensitive; + private boolean dynamic; + private boolean supportsEl; + private String identifiesControllerService; + + /** + * The set of allowable values for this property. If empty then the + * allowable values are not constrained. + * + * @return + */ + public List getAllowableValues() { + return allowableValues; + } + + public void setAllowableValues(List allowableValues) { + this.allowableValues = allowableValues; + } + + /** + * The default value for this property. + * + * @return + */ + public String getDefaultValue() { + return defaultValue; + } + + public void setDefaultValue(String defaultValue) { + this.defaultValue = defaultValue; + } + + /** + * And explanation of the meaning of the given property. This + * description is meant to be displayed to a user or simply provide a + * mechanism of documenting intent. + * + * @return + */ + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + /** + * The property name. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The human-readable name to display to users. + * + * @return + */ + public String getDisplayName() { + return displayName; + } + + public void setDisplayName(String displayName) { + this.displayName = displayName; + } + + /** + * Determines whether the property is required for this processor. + * + * @return + */ + public boolean isRequired() { + return required; + } + + public void setRequired(boolean required) { + this.required = required; + } + + /** + * Indicates that the value for this property should be considered + * sensitive and protected whenever stored or represented. + * + * @return + */ + public boolean isSensitive() { + return sensitive; + } + + public void setSensitive(boolean sensitive) { + this.sensitive = sensitive; + } + + /** + * Indicates whether this property is dynamic. + * + * @return + */ + public boolean isDynamic() { + return dynamic; + } + + public void setDynamic(boolean dynamic) { + this.dynamic = dynamic; + } + + /** + * Specifies whether or not this property support expression language. + * + * @return + */ + public boolean getSupportsEl() { + return supportsEl; + } + + public void setSupportsEl(boolean supportsEl) { + this.supportsEl = supportsEl; + } + + /** + * If this property identifies a controller service, this returns the + * fully qualified type, null otherwise. + * + * @return + */ + public String getIdentifiesControllerService() { + return identifiesControllerService; + } + + public void setIdentifiesControllerService(String identifiesControllerService) { + this.identifiesControllerService = identifiesControllerService; + } + + /** + * The allowable values for a property with a constrained set of options. + */ + @XmlType(name = "allowableValue") + public static class AllowableValueDTO { + + private String displayName; + private String value; + private String description; + + /** + * Returns the human-readable value that is allowed for this + * PropertyDescriptor + * + * @return + */ + public String getDisplayName() { + return displayName; + } + + public void setDisplayName(String displayName) { + this.displayName = displayName; + } + + /** + * Returns the value for this allowable value. + * + * @return + */ + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } + + /** + * Returns a description of this Allowable Value, or null + * if no description is given + * + * @return + */ + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + @Override + public boolean equals(final Object obj) { + if (obj == this) { + return true; + } + + if (!(obj instanceof AllowableValueDTO)) { + return false; + } + + final AllowableValueDTO other = (AllowableValueDTO) obj; + return (this.value.equals(other.getValue())); + } + + @Override + public int hashCode() { + return 23984731 + 17 * value.hashCode(); + } + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ReportingTaskDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ReportingTaskDTO.java new file mode 100644 index 0000000000..a019f97341 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ReportingTaskDTO.java @@ -0,0 +1,228 @@ +/* + * 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 java.util.Collection; +import java.util.Map; + +import javax.xml.bind.annotation.XmlType; + +/** + * Component that is capable of reporting internal NiFi state to an external service + */ +@XmlType(name = "reportingTask") +public class ReportingTaskDTO extends NiFiComponentDTO { + private String name; + private String type; + private String state; + private String availability; + private String comments; + + private String schedulingPeriod; + private String schedulingStrategy; + private Map defaultSchedulingPeriod; + + private Map properties; + private Map descriptors; + + private String customUiUrl; + private String annotationData; + + private Collection validationErrors; + private Integer activeThreadCount; + + /** + * The user-defined name of the reporting task + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The user-defined comments for the reporting task + * @return + */ + public String getComments() { + return comments; + } + + public void setComments(String comments) { + this.comments = comments; + } + + /** + * The type of reporting task + * @return + */ + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + /** + * The frequency with which to schedule the reporting task. The format of the value will + * depend on the value of {@link #getSchedulingStrategy()}. + * + * @return The scheduling period + */ + public String getSchedulingPeriod() { + return schedulingPeriod; + } + + public void setSchedulingPeriod(String schedulingPeriod) { + this.schedulingPeriod = schedulingPeriod; + } + + /** + * The current scheduling state of the reporting task + * @return + */ + public String getState() { + return state; + } + + public void setState(String state) { + this.state = state; + } + + /** + * The scheduling strategy that determines how the {@link #getSchedulingPeriod()} value should + * be interpreted + * + * @return + */ + public String getSchedulingStrategy() { + return schedulingStrategy; + } + + public void setSchedulingStrategy(String schedulingStrategy) { + this.schedulingStrategy = schedulingStrategy; + } + + /** + * Where this service is available. Possible values are CLUSTER_MANAGER_ONLY, NODE_ONLY, BOTH. + * + * @return + */ + public String getAvailability() { + return availability; + } + + public void setAvailability(String availability) { + this.availability = availability; + } + + /** + * The reporting task's properties + * @return + */ + public Map getProperties() { + return properties; + } + + public void setProperties(Map properties) { + this.properties = properties; + } + + /** + * Map of property name to descriptor + * @return + */ + public Map getDescriptors() { + return descriptors; + } + + public void setDescriptors(Map descriptors) { + this.descriptors = descriptors; + } + + /** + * Returns the URL for this reporting task custom configuration UI + * if applicable. Null otherwise. + * + * @return + */ + public String getCustomUiUrl() { + return customUiUrl; + } + + public void setCustomUiUrl(String customUiUrl) { + this.customUiUrl = customUiUrl; + } + + /** + * The currently configured annotation data for the reporting task + * @return + */ + public String getAnnotationData() { + return annotationData; + } + + public void setAnnotationData(String annotationData) { + this.annotationData = annotationData; + } + + /** + * Gets the validation errors from this reporting task. These validation errors + * represent the problems with the reporting task that must be resolved before it + * can be scheduled to run. + * + * @return The validation errors + */ + public Collection getValidationErrors() { + return validationErrors; + } + + public void setValidationErrors(Collection validationErrors) { + this.validationErrors = validationErrors; + } + + /** + * The default scheduling period for the different scheduling strategies. + * + * @return + */ + public Map getDefaultSchedulingPeriod() { + return defaultSchedulingPeriod; + } + + public void setDefaultSchedulingPeriod(Map defaultSchedulingPeriod) { + this.defaultSchedulingPeriod = defaultSchedulingPeriod; + } + + /** + * The number of active threads for this reporting task. + * + * @return + */ + public Integer getActiveThreadCount() { + return activeThreadCount; + } + + public void setActiveThreadCount(Integer activeThreadCount) { + this.activeThreadCount = activeThreadCount; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.java index e608a7efcb..3327b49815 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.java @@ -26,8 +26,10 @@ public class RevisionDTO { private String clientId; private Long version; + private String lastModifier; /* getters / setters */ + /** * A client identifier used to make a request. By including a client * identifier, the API can allow multiple requests without needing the @@ -60,4 +62,17 @@ public class RevisionDTO { this.version = version; } + /** + * The user that last modified the flow. + * + * @return + */ + public String getLastModifier() { + return lastModifier; + } + + public void setLastModifier(String lastModifier) { + this.lastModifier = lastModifier; + } + } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.java index 58086ce285..b01a271ae1 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.java @@ -24,7 +24,7 @@ import javax.xml.bind.annotation.XmlType; */ @XmlType(name = "componentDetails") @XmlSeeAlso({ - ProcessorDetailsDTO.class, + ExtensionDetailsDTO.class, RemoteProcessGroupDetailsDTO.class }) public class ComponentDetailsDTO { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ExtensionDetailsDTO.java similarity index 87% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ExtensionDetailsDTO.java index 3523f62e94..e2e49d67a2 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ExtensionDetailsDTO.java @@ -19,15 +19,15 @@ package org.apache.nifi.web.api.dto.action.component.details; import javax.xml.bind.annotation.XmlType; /** - * Processor details for an action. + * Extension details for an action. */ -@XmlType(name = "processorDetails") -public class ProcessorDetailsDTO extends ComponentDetailsDTO { +@XmlType(name = "extensionDetails") +public class ExtensionDetailsDTO extends ComponentDetailsDTO { private String type; /** - * The processors type. + * The extension type. * * @return */ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentHistoryEntity.java similarity index 66% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentHistoryEntity.java index 19166f7c99..ca682110df 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentHistoryEntity.java @@ -17,29 +17,29 @@ package org.apache.nifi.web.api.entity; import javax.xml.bind.annotation.XmlRootElement; -import org.apache.nifi.web.api.dto.ProcessorHistoryDTO; +import org.apache.nifi.web.api.dto.ComponentHistoryDTO; /** * 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 ProcessorHistoryDTO. + * reference to a ComponentHistoryDTO. */ -@XmlRootElement(name = "processorHistoryEntity") -public class ProcessorHistoryEntity extends Entity { +@XmlRootElement(name = "componentHistoryEntity") +public class ComponentHistoryEntity extends Entity { - private ProcessorHistoryDTO propertyHistory; + private ComponentHistoryDTO componentHistory; /** - * The ProcessorHistoryDTO that is being serialized. + * The ComponentHistoryDTO that is being serialized. * - * @return The ProcessorHistoryDTO object + * @return The ComponentHistoryDTO object */ - public ProcessorHistoryDTO getProcessorHistory() { - return propertyHistory; + public ComponentHistoryDTO getComponentHistory() { + return componentHistory; } - public void setProcessorHistory(ProcessorHistoryDTO propertyHistory) { - this.propertyHistory = propertyHistory; + public void setComponentHistory(ComponentHistoryDTO componentHistory) { + this.componentHistory = componentHistory; } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceEntity.java new file mode 100644 index 0000000000..44364e712c --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceEntity.java @@ -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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; + +/** + * 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 + * controller service. + */ +@XmlRootElement(name = "controllerServiceEntity") +public class ControllerServiceEntity extends Entity { + + private ControllerServiceDTO controllerService; + + /** + * The controller service that is being serialized. + * + * @return + */ + public ControllerServiceDTO getControllerService() { + return controllerService; + } + + public void setControllerService(ControllerServiceDTO controllerService) { + this.controllerService = controllerService; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceReferencingComponentsEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceReferencingComponentsEntity.java new file mode 100644 index 0000000000..6010f93014 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceReferencingComponentsEntity.java @@ -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.entity; + +import java.util.Set; +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * controller services referencing components. + */ +@XmlRootElement(name = "controllerServiceReferencingComponentsEntity") +public class ControllerServiceReferencingComponentsEntity extends Entity { + + private Set controllerServiceReferencingComponents; + + /** + * The list of controller service referencing components that are being serialized. + * + * @return + */ + public Set getControllerServiceReferencingComponents() { + return controllerServiceReferencingComponents; + } + + public void setControllerServiceReferencingComponents(Set controllerServiceReferencingComponents) { + this.controllerServiceReferencingComponents = controllerServiceReferencingComponents; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceTypesEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceTypesEntity.java new file mode 100644 index 0000000000..dafb8c22af --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceTypesEntity.java @@ -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.entity; + +import java.util.Set; +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.DocumentedTypeDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * controller service types. + */ +@XmlRootElement(name = "controllerServiceTypesEntity") +public class ControllerServiceTypesEntity extends Entity { + + private Set controllerServiceTypes; + + /** + * The list of controller service types that are being serialized. + * + * @return + */ + public Set getControllerServiceTypes() { + return controllerServiceTypes; + } + + public void setControllerServiceTypes(Set controllerServiceTypes) { + this.controllerServiceTypes = controllerServiceTypes; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServicesEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServicesEntity.java new file mode 100644 index 0000000000..4485b43d15 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServicesEntity.java @@ -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.entity; + +import java.util.Set; +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * controller services. + */ +@XmlRootElement(name = "controllerServicesEntity") +public class ControllerServicesEntity extends Entity { + + private Set controllerServices; + + /** + * The list of controller services that are being serialized. + * + * @return + */ + public Set getControllerServices() { + return controllerServices; + } + + public void setControllerServices(Set controllerServices) { + this.controllerServices = controllerServices; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/PropertyDescriptorEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/PropertyDescriptorEntity.java new file mode 100644 index 0000000000..9251b7f220 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/PropertyDescriptorEntity.java @@ -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.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ProcessorDTO; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; + +/** + * 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 PropertyDescriptorDTO. + */ +@XmlRootElement(name = "propertyDescriptor") +public class PropertyDescriptorEntity extends Entity { + + private PropertyDescriptorDTO propertyDescriptor; + + /** + * The PropertyDescriptorDTO that is being serialized. + * + * @return The PropertyDescriptorDTO object + */ + public PropertyDescriptorDTO getPropertyDescriptor() { + return propertyDescriptor; + } + + public void setPropertyDescriptor(PropertyDescriptorDTO propertyDescriptor) { + this.propertyDescriptor = propertyDescriptor; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskEntity.java new file mode 100644 index 0000000000..a372751712 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskEntity.java @@ -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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; + +/** + * 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 + * reporting task. + */ +@XmlRootElement(name = "reportingTaskEntity") +public class ReportingTaskEntity extends Entity { + + private ReportingTaskDTO reportingTask; + + /** + * The reporting task that is being serialized. + * + * @return + */ + public ReportingTaskDTO getReportingTask() { + return reportingTask; + } + + public void setReportingTask(ReportingTaskDTO reportingTask) { + this.reportingTask = reportingTask; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskTypesEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskTypesEntity.java new file mode 100644 index 0000000000..4b021efb44 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskTypesEntity.java @@ -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.entity; + +import java.util.Set; +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.DocumentedTypeDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * reporting task types. + */ +@XmlRootElement(name = "reportingTaskTypesEntity") +public class ReportingTaskTypesEntity extends Entity { + + private Set reportingTaskTypes; + + /** + * The list of reporting task types that are being serialized. + * + * @return + */ + public Set getReportingTaskTypes() { + return reportingTaskTypes; + } + + public void setReportingTaskTypes(Set reportingTaskTypes) { + this.reportingTaskTypes = reportingTaskTypes; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTasksEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTasksEntity.java new file mode 100644 index 0000000000..4699d5d3c4 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTasksEntity.java @@ -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.entity; + +import java.util.Set; +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * reporting tasks. + */ +@XmlRootElement(name = "reportingTasksEntity") +public class ReportingTasksEntity extends Entity { + + private Set reportingTasks; + + /** + * The list of reporting tasks that are being serialized. + * + * @return + */ + public Set getReportingTasks() { + return reportingTasks; + } + + public void setReportingTasks(Set reportingTasks) { + this.reportingTasks = reportingTasks; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml index e522d30ff7..5995f5c327 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml @@ -32,14 +32,6 @@ org.apache.nifi nifi-properties - - commons-io - commons-io - - - org.apache.commons - commons-lang3 - org.apache.nifi nifi-processor-utils diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java index 34b1327513..243aaa35ae 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java @@ -26,7 +26,6 @@ import javax.xml.stream.XMLOutputFactory; import javax.xml.stream.XMLStreamException; import javax.xml.stream.XMLStreamWriter; -import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.DynamicProperties; import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.documentation.CapabilityDescription; @@ -212,13 +211,23 @@ public class HtmlDocumentationWriter implements DocumentationWriter { xmlStreamWriter.writeEndElement(); xmlStreamWriter.writeStartElement("p"); if (tags != null) { - final String tagString = StringUtils.join(tags.value(), ", "); + final String tagString = join(tags.value(), ", "); xmlStreamWriter.writeCharacters(tagString); } else { xmlStreamWriter.writeCharacters("None."); } xmlStreamWriter.writeEndElement(); - + } + + static String join(final String[] toJoin, final String delimiter) { + final StringBuilder sb = new StringBuilder(); + for (int i=0; i < toJoin.length; i++) { + sb.append(toJoin[i]); + if ( i < toJoin.length - 1 ) { + sb.append(delimiter); + } + } + return sb.toString(); } /** diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java index bc355f7712..c253a4d8f2 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java @@ -23,7 +23,6 @@ import java.util.List; import javax.xml.stream.XMLStreamException; import javax.xml.stream.XMLStreamWriter; -import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.DynamicRelationship; import org.apache.nifi.annotation.behavior.ReadsAttribute; import org.apache.nifi.annotation.behavior.ReadsAttributes; @@ -67,6 +66,13 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { handleWritesAttributes(xmlStreamWriter, processor); } + private String defaultIfBlank(final String test, final String defaultValue) { + if ( test == null || test.trim().isEmpty() ) { + return defaultValue; + } + return test; + } + /** * Writes out just the attributes that are being read in a table form. * @@ -91,10 +97,10 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { for (ReadsAttribute attribute : attributesRead) { xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "td", - StringUtils.defaultIfBlank(attribute.attribute(), "Not Specified")); + defaultIfBlank(attribute.attribute(), "Not Specified")); // TODO allow for HTML characters here. writeSimpleElement(xmlStreamWriter, "td", - StringUtils.defaultIfBlank(attribute.description(), "Not Specified")); + defaultIfBlank(attribute.description(), "Not Specified")); xmlStreamWriter.writeEndElement(); } @@ -129,10 +135,10 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { for (WritesAttribute attribute : attributesRead) { xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "td", - StringUtils.defaultIfBlank(attribute.attribute(), "Not Specified")); + defaultIfBlank(attribute.attribute(), "Not Specified")); // TODO allow for HTML characters here. writeSimpleElement(xmlStreamWriter, "td", - StringUtils.defaultIfBlank(attribute.description(), "Not Specified")); + defaultIfBlank(attribute.description(), "Not Specified")); xmlStreamWriter.writeEndElement(); } xmlStreamWriter.writeEndElement(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java index fcd3ea340f..69ce8d9ca4 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java @@ -18,6 +18,7 @@ package org.apache.nifi.documentation.mock; import org.apache.nifi.controller.ControllerServiceInitializationContext; import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.logging.ComponentLog; /** * A Mock ControllerServiceInitializationContext so that ControllerServices can @@ -37,4 +38,9 @@ public class MockControllerServiceInitializationContext implements ControllerSer return new MockControllerServiceLookup(); } + @Override + public ComponentLog getLogger() { + return null; + } + } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceLookup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceLookup.java index f11bc6874e..5c60881cee 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceLookup.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceLookup.java @@ -52,4 +52,14 @@ public class MockControllerServiceLookup implements ControllerServiceLookup { return Collections.emptySet(); } + @Override + public boolean isControllerServiceEnabling(String serviceIdentifier) { + return false; + } + + @Override + public String getControllerServiceName(String serviceIdentifier) { + return serviceIdentifier; + } + } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockReportingInitializationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockReportingInitializationContext.java index 910ce5a5e2..dc6e236f8f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockReportingInitializationContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockReportingInitializationContext.java @@ -19,6 +19,7 @@ package org.apache.nifi.documentation.mock; import java.util.concurrent.TimeUnit; import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.reporting.ReportingInitializationContext; import org.apache.nifi.scheduling.SchedulingStrategy; @@ -26,8 +27,6 @@ import org.apache.nifi.scheduling.SchedulingStrategy; * A Mock ReportingInitializationContext that can be used to initialize a * ReportingTask for the purposes of documentation generation. * - * @author Alligator - * */ public class MockReportingInitializationContext implements ReportingInitializationContext { @@ -60,4 +59,9 @@ public class MockReportingInitializationContext implements ReportingInitializati public SchedulingStrategy getSchedulingStrategy() { return SchedulingStrategy.TIMER_DRIVEN; } + + @Override + public ComponentLog getLogger() { + return null; + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java index 9d7926e858..90ff09f19c 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java @@ -30,9 +30,17 @@ import org.apache.nifi.reporting.ReportingTask; import org.junit.Test; import static org.apache.nifi.documentation.html.XmlValidator.assertContains; +import static org.junit.Assert.assertEquals; public class HtmlDocumentationWriterTest { + @Test + public void testJoin() { + assertEquals("a, b, c", HtmlDocumentationWriter.join(new String[] {"a", "b", "c"}, ", ")); + assertEquals("a, b", HtmlDocumentationWriter.join(new String[] {"a", "b"}, ", ")); + assertEquals("a", HtmlDocumentationWriter.join(new String[] {"a"}, ", ")); + } + @Test public void testDocumentControllerService() throws InitializationException, IOException { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/pom.xml index f9ee703bb0..70dcc8120d 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/pom.xml @@ -32,10 +32,6 @@ org.apache.nifi nifi-properties - - org.apache.nifi - nifi-web-optimistic-locking - org.apache.nifi nifi-administration diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextThreadLocal.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextThreadLocal.java index 012e7c7081..c8c7206947 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextThreadLocal.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextThreadLocal.java @@ -32,12 +32,7 @@ public class ClusterContextThreadLocal { } public static ClusterContext getContext() { - ClusterContext ctx = contextHolder.get(); - if(ctx == null) { - ctx = createEmptyContext(); - contextHolder.set(ctx); - } - return ctx; + return contextHolder.get(); } public static void setContext(final ClusterContext context) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java deleted file mode 100644 index 90b8a370e2..0000000000 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.web; - -import org.apache.nifi.cluster.context.ClusterContext; -import org.apache.nifi.cluster.context.ClusterContextThreadLocal; - -/** - * An optimistic locking manager that provides for optimistic locking in a clustered - * environment. - * - * @author unattributed - */ -public class ClusterAwareOptimisticLockingManager implements OptimisticLockingManager { - - private final OptimisticLockingManager optimisticLockingManager; - - public ClusterAwareOptimisticLockingManager(final OptimisticLockingManager optimisticLockingManager) { - this.optimisticLockingManager = optimisticLockingManager; - } - - @Override - public Revision checkRevision(Revision revision) throws InvalidRevisionException { - final Revision currentRevision = getRevision(); - if(currentRevision.equals(revision) == false) { - throw new InvalidRevisionException(String.format("Given revision %s does not match current revision %s.", revision, currentRevision)); - } else { - return revision.increment(revision.getClientId()); - } - } - - @Override - public boolean isCurrent(Revision revision) { - return getRevision().equals(revision); - } - - @Override - public Revision getRevision() { - final ClusterContext ctx = ClusterContextThreadLocal.getContext(); - if(ctx == null || ctx.getRevision() == null) { - return optimisticLockingManager.getRevision(); - } else { - return ctx.getRevision(); - } - } - - @Override - public void setRevision(final Revision revision) { - final ClusterContext ctx = ClusterContextThreadLocal.getContext(); - if(ctx != null) { - ctx.setRevision(revision); - } - optimisticLockingManager.setRevision(revision); - } - - @Override - public Revision incrementRevision() { - final Revision currentRevision = getRevision(); - final Revision incRevision = currentRevision.increment(); - setRevision(incRevision); - return incRevision; - } - - @Override - public Revision incrementRevision(final String clientId) { - final Revision currentRevision = getRevision(); - final Revision incRevision = currentRevision.increment(clientId); - setRevision(incRevision); - return incRevision; - } - - @Override - public String getLastModifier() { - return optimisticLockingManager.getLastModifier(); - } - - @Override - public void setLastModifier(final String lastModifier) { - optimisticLockingManager.setLastModifier(lastModifier); - } - -} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/pom.xml index 7b6a4183fa..bdff00fcf0 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/pom.xml @@ -45,6 +45,10 @@ org.apache.nifi nifi-client-dto + + org.apache.nifi + nifi-web-optimistic-locking + org.apache.nifi nifi-framework-core diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java index eedb88fb80..c17b429518 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java @@ -27,14 +27,25 @@ import org.apache.nifi.cluster.protocol.StandardDataFlow; public class ClusterDataFlow { private final StandardDataFlow dataFlow; - private final NodeIdentifier primaryNodeId; + private final byte[] controllerServices; + private final byte[] reportingTasks; - public ClusterDataFlow(final StandardDataFlow dataFlow, final NodeIdentifier primaryNodeId) { + public ClusterDataFlow(final StandardDataFlow dataFlow, final NodeIdentifier primaryNodeId, final byte[] controllerServices, final byte[] reportingTasks) { this.dataFlow = dataFlow; this.primaryNodeId = primaryNodeId; + this.controllerServices = controllerServices; + this.reportingTasks = reportingTasks; } + public byte[] getControllerServices() { + return controllerServices; + } + + public byte[] getReportingTasks() { + return reportingTasks; + } + public NodeIdentifier getPrimaryNodeId() { return primaryNodeId; } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java index 339d9047f3..082d65e512 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java @@ -17,6 +17,7 @@ package org.apache.nifi.cluster.flow; import java.util.Set; + import org.apache.nifi.cluster.protocol.NodeIdentifier; /** @@ -66,6 +67,22 @@ public interface DataFlowManagementService { */ void updatePrimaryNode(NodeIdentifier nodeId) throws DaoException; + /** + * Updates the dataflow with the given serialized form of the Controller Services that are to exist on the NCM. + * + * @param serializedControllerServices + * @throws DaoException + */ + void updateControllerServices(byte[] serializedControllerServices) throws DaoException; + + /** + * Updates the dataflow with the given serialized form of Reporting Tasks that are to exist on the NCM. + * + * @param serviceNodes + * @throws DaoException + */ + void updateReportingTasks(byte[] serializedReportingTasks) throws DaoException; + /** * Sets the state of the flow. * diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java index 72b594afd1..dd9d2a3c6f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java @@ -111,6 +111,8 @@ public class DataFlowDaoImpl implements DataFlowDao { public static final String FLOW_XML_FILENAME = "flow.xml"; public static final String TEMPLATES_FILENAME = "templates.xml"; public static final String SNIPPETS_FILENAME = "snippets.xml"; + public static final String CONTROLLER_SERVICES_FILENAME = "controller-services.xml"; + public static final String REPORTING_TASKS_FILENAME = "reporting-tasks.xml"; public static final String CLUSTER_INFO_FILENAME = "cluster-info.xml"; private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(DataFlowDaoImpl.class)); @@ -408,13 +410,7 @@ public class DataFlowDaoImpl implements DataFlowDao { final File stateFile = new File(dir, FLOW_PACKAGE); stateFile.createNewFile(); - final byte[] flowBytes = getEmptyFlowBytes(); - final byte[] templateBytes = new byte[0]; - final byte[] snippetBytes = new byte[0]; - final DataFlow dataFlow = new StandardDataFlow(flowBytes, templateBytes, snippetBytes); - - final ClusterMetadata clusterMetadata = new ClusterMetadata(); - writeDataFlow(stateFile, dataFlow, clusterMetadata); + writeDataFlow(stateFile, new ClusterDataFlow(null, null, new byte[0], new byte[0]), new ClusterMetadata()); return stateFile; } @@ -479,7 +475,9 @@ public class DataFlowDaoImpl implements DataFlowDao { byte[] templateBytes = new byte[0]; byte[] snippetBytes = new byte[0]; byte[] clusterInfoBytes = new byte[0]; - + byte[] controllerServiceBytes = new byte[0]; + byte[] reportingTaskBytes = new byte[0]; + try (final InputStream inStream = new FileInputStream(file); final TarArchiveInputStream tarIn = new TarArchiveInputStream(new BufferedInputStream(inStream))) { TarArchiveEntry tarEntry; @@ -501,6 +499,14 @@ public class DataFlowDaoImpl implements DataFlowDao { clusterInfoBytes = new byte[(int) tarEntry.getSize()]; StreamUtils.fillBuffer(tarIn, clusterInfoBytes, true); break; + case CONTROLLER_SERVICES_FILENAME: + controllerServiceBytes = new byte[(int) tarEntry.getSize()]; + StreamUtils.fillBuffer(tarIn, controllerServiceBytes, true); + break; + case REPORTING_TASKS_FILENAME: + reportingTaskBytes = new byte[(int) tarEntry.getSize()]; + StreamUtils.fillBuffer(tarIn, reportingTaskBytes, true); + break; default: throw new DaoException("Found Unexpected file in dataflow configuration: " + tarEntry.getName()); } @@ -518,7 +524,7 @@ public class DataFlowDaoImpl implements DataFlowDao { final StandardDataFlow dataFlow = new StandardDataFlow(flowBytes, templateBytes, snippetBytes); dataFlow.setAutoStartProcessors(autoStart); - return new ClusterDataFlow(dataFlow, (clusterMetadata == null) ? null : clusterMetadata.getPrimaryNodeId()); + return new ClusterDataFlow(dataFlow, (clusterMetadata == null) ? null : clusterMetadata.getPrimaryNodeId(), controllerServiceBytes, reportingTaskBytes); } private void writeDataFlow(final File file, final ClusterDataFlow clusterDataFlow) throws IOException, JAXBException { @@ -536,7 +542,7 @@ public class DataFlowDaoImpl implements DataFlowDao { clusterMetadata.setPrimaryNodeId(clusterDataFlow.getPrimaryNodeId()); // write to disk - writeDataFlow(file, dataFlow, clusterMetadata); + writeDataFlow(file, clusterDataFlow, clusterMetadata); } private void writeTarEntry(final TarArchiveOutputStream tarOut, final String filename, final byte[] bytes) throws IOException { @@ -547,14 +553,23 @@ public class DataFlowDaoImpl implements DataFlowDao { tarOut.closeArchiveEntry(); } - private void writeDataFlow(final File file, final DataFlow dataFlow, final ClusterMetadata clusterMetadata) throws IOException, JAXBException { + private void writeDataFlow(final File file, final ClusterDataFlow clusterDataFlow, final ClusterMetadata clusterMetadata) throws IOException, JAXBException { try (final OutputStream fos = new FileOutputStream(file); final TarArchiveOutputStream tarOut = new TarArchiveOutputStream(new BufferedOutputStream(fos))) { - writeTarEntry(tarOut, FLOW_XML_FILENAME, dataFlow.getFlow()); - writeTarEntry(tarOut, TEMPLATES_FILENAME, dataFlow.getTemplates()); - writeTarEntry(tarOut, SNIPPETS_FILENAME, dataFlow.getSnippets()); + final DataFlow dataFlow = clusterDataFlow.getDataFlow(); + if ( dataFlow == null ) { + writeTarEntry(tarOut, FLOW_XML_FILENAME, getEmptyFlowBytes()); + writeTarEntry(tarOut, TEMPLATES_FILENAME, new byte[0]); + writeTarEntry(tarOut, SNIPPETS_FILENAME, new byte[0]); + } else { + writeTarEntry(tarOut, FLOW_XML_FILENAME, dataFlow.getFlow()); + writeTarEntry(tarOut, TEMPLATES_FILENAME, dataFlow.getTemplates()); + writeTarEntry(tarOut, SNIPPETS_FILENAME, dataFlow.getSnippets()); + } + writeTarEntry(tarOut, CONTROLLER_SERVICES_FILENAME, clusterDataFlow.getControllerServices()); + writeTarEntry(tarOut, REPORTING_TASKS_FILENAME, clusterDataFlow.getReportingTasks()); final ByteArrayOutputStream baos = new ByteArrayOutputStream(256); writeClusterMetadata(clusterMetadata, baos); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java index e135af311f..1bb8ca3c92 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java @@ -41,7 +41,6 @@ import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; import org.apache.nifi.logging.NiFiLog; import org.apache.nifi.util.FormatUtils; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -154,17 +153,74 @@ public class DataFlowManagementServiceImpl implements DataFlowManagementService final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow(); final StandardDataFlow dataFlow; + final byte[] controllerServiceBytes; + final byte[] reportingTaskBytes; if (existingClusterDataFlow == null) { dataFlow = null; + controllerServiceBytes = new byte[0]; + reportingTaskBytes = new byte[0]; } else { dataFlow = existingClusterDataFlow.getDataFlow(); + controllerServiceBytes = existingClusterDataFlow.getControllerServices(); + reportingTaskBytes = existingClusterDataFlow.getReportingTasks(); } - flowDao.saveDataFlow(new ClusterDataFlow(dataFlow, nodeId)); + flowDao.saveDataFlow(new ClusterDataFlow(dataFlow, nodeId, controllerServiceBytes, reportingTaskBytes)); } finally { resourceLock.unlock("updatePrimaryNode"); } } + + + @Override + public void updateControllerServices(final byte[] controllerServiceBytes) throws DaoException { + resourceLock.lock(); + try { + final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow(); + + final StandardDataFlow dataFlow; + final byte[] reportingTaskBytes; + final NodeIdentifier nodeId; + if (existingClusterDataFlow == null) { + dataFlow = null; + nodeId = null; + reportingTaskBytes = new byte[0]; + } else { + dataFlow = existingClusterDataFlow.getDataFlow(); + nodeId = existingClusterDataFlow.getPrimaryNodeId(); + reportingTaskBytes = existingClusterDataFlow.getReportingTasks(); + } + + flowDao.saveDataFlow(new ClusterDataFlow(dataFlow, nodeId, controllerServiceBytes, reportingTaskBytes)); + } finally { + resourceLock.unlock("updateControllerServices"); + } + } + + @Override + public void updateReportingTasks(final byte[] reportingTaskBytes) throws DaoException { + resourceLock.lock(); + try { + final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow(); + + final StandardDataFlow dataFlow; + final byte[] controllerServiceBytes; + final NodeIdentifier nodeId; + if (existingClusterDataFlow == null) { + dataFlow = null; + nodeId = null; + controllerServiceBytes = null; + } else { + dataFlow = existingClusterDataFlow.getDataFlow(); + nodeId = existingClusterDataFlow.getPrimaryNodeId(); + controllerServiceBytes = existingClusterDataFlow.getControllerServices(); + } + + flowDao.saveDataFlow(new ClusterDataFlow(dataFlow, nodeId, controllerServiceBytes, reportingTaskBytes)); + } finally { + resourceLock.unlock("updateControllerServices"); + } + } @Override public PersistedFlowState getPersistedFlowState() { @@ -303,9 +359,10 @@ public class DataFlowManagementServiceImpl implements DataFlowManagementService final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow(); final ClusterDataFlow currentClusterDataFlow; if (existingClusterDataFlow == null) { - currentClusterDataFlow = new ClusterDataFlow(dataFlow, null); + currentClusterDataFlow = new ClusterDataFlow(dataFlow, null, new byte[0], new byte[0]); } else { - currentClusterDataFlow = new ClusterDataFlow(dataFlow, existingClusterDataFlow.getPrimaryNodeId()); + currentClusterDataFlow = new ClusterDataFlow(dataFlow, existingClusterDataFlow.getPrimaryNodeId(), + existingClusterDataFlow.getControllerServices(), existingClusterDataFlow.getReportingTasks()); } flowDao.saveDataFlow(currentClusterDataFlow); flowDao.setPersistedFlowState(PersistedFlowState.CURRENT); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java index 3f966e579d..8bc73abb9d 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java @@ -191,6 +191,20 @@ public class NodeResponse { return clientResponse; } + /** + * If this node response has been merged returns the updated entity, + * otherwise null. Also returns null if hasThrowable() is true. The + * intent of this method is to support getting the response entity + * when it was already consumed during the merge operation. In this + * case the client response rom getClientResponse() will not support + * a getEntity(...) or getEntityInputStream() call. + * + * @return + */ + public Entity getUpdatedEntity() { + return updatedEntity; + } + /** * Creates a Response by mapping the ClientResponse values to it. Since the * ClientResponse's input stream can only be read once, this method should diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java index 4d5455f55b..db6421ecd3 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java @@ -16,12 +16,12 @@ */ package org.apache.nifi.cluster.manager.impl; -import java.io.File; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.OutputStream; import java.io.Serializable; import java.net.URI; -import java.net.URL; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -41,7 +41,9 @@ import java.util.TimerTask; import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.CompletionService; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -54,17 +56,20 @@ import javax.net.ssl.SSLContext; import javax.ws.rs.HttpMethod; import javax.ws.rs.WebApplicationException; import javax.ws.rs.core.StreamingOutput; -import javax.xml.XMLConstants; import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilderFactory; import javax.xml.parsers.ParserConfigurationException; +import javax.xml.transform.OutputKeys; +import javax.xml.transform.Transformer; +import javax.xml.transform.TransformerException; +import javax.xml.transform.TransformerFactory; import javax.xml.transform.dom.DOMSource; -import javax.xml.validation.Schema; -import javax.xml.validation.SchemaFactory; -import javax.xml.validation.Validator; +import javax.xml.transform.stream.StreamResult; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.admin.service.AuditService; +import org.apache.nifi.annotation.lifecycle.OnAdded; +import org.apache.nifi.annotation.lifecycle.OnRemoved; import org.apache.nifi.cluster.BulletinsPayload; import org.apache.nifi.cluster.HeartbeatPayload; import org.apache.nifi.cluster.context.ClusterContext; @@ -122,12 +127,18 @@ import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.Heartbeater; import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.StandardFlowSerializer; import org.apache.nifi.controller.ValidationContextFactory; +import org.apache.nifi.controller.exception.ComponentLifeCycleException; import org.apache.nifi.controller.reporting.ClusteredReportingTaskNode; import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException; +import org.apache.nifi.controller.reporting.ReportingTaskProvider; import org.apache.nifi.controller.reporting.StandardReportingInitializationContext; +import org.apache.nifi.controller.scheduling.QuartzSchedulingAgent; import org.apache.nifi.controller.scheduling.StandardProcessScheduler; import org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent; +import org.apache.nifi.controller.service.ControllerServiceLoader; import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; import org.apache.nifi.controller.service.StandardControllerServiceProvider; @@ -147,10 +158,12 @@ import org.apache.nifi.events.BulletinFactory; import org.apache.nifi.events.VolatileBulletinRepository; import org.apache.nifi.framework.security.util.SslContextFactory; import org.apache.nifi.io.socket.multicast.DiscoverableService; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.logging.NiFiLog; import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.nar.NarThreadContextClassLoader; +import org.apache.nifi.processor.SimpleProcessLogger; import org.apache.nifi.processor.StandardValidationContextFactory; import org.apache.nifi.remote.RemoteResourceManager; import org.apache.nifi.remote.RemoteSiteListener; @@ -168,7 +181,11 @@ import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.util.DomUtils; import org.apache.nifi.util.FormatUtils; import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.util.ObjectHolder; +import org.apache.nifi.util.ReflectionUtils; +import org.apache.nifi.web.OptimisticLockingManager; import org.apache.nifi.web.Revision; +import org.apache.nifi.web.UpdateRevision; import org.apache.nifi.web.api.dto.FlowSnippetDTO; import org.apache.nifi.web.api.dto.NodeDTO; import org.apache.nifi.web.api.dto.ProcessGroupDTO; @@ -204,6 +221,16 @@ import org.xml.sax.SAXParseException; import com.sun.jersey.api.client.ClientResponse; +import org.apache.nifi.controller.service.ControllerServiceState; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; +import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; +import org.apache.nifi.web.api.entity.ControllerServiceEntity; +import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity; +import org.apache.nifi.web.api.entity.ControllerServicesEntity; +import org.apache.nifi.web.api.entity.ReportingTaskEntity; +import org.apache.nifi.web.api.entity.ReportingTasksEntity; + /** * Provides a cluster manager implementation. The manager federates incoming * HTTP client requests to the nodes' external API using the HTTP protocol. The @@ -222,7 +249,7 @@ import com.sun.jersey.api.client.ClientResponse; * * @author unattributed */ -public class WebClusterManager implements HttpClusterManager, ProtocolHandler, ControllerServiceProvider { +public class WebClusterManager implements HttpClusterManager, ProtocolHandler, ControllerServiceProvider, ReportingTaskProvider { public static final String ROOT_GROUP_ID_ALIAS = "root"; public static final String BULLETIN_CATEGORY = "Clustering"; @@ -279,6 +306,7 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C public static final Pattern PROCESSORS_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/processors"); public static final Pattern PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/processors/[a-f0-9\\-]{36}"); + public static final Pattern CLUSTER_PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/cluster/processors/[a-f0-9\\-]{36}"); public static final Pattern REMOTE_PROCESS_GROUPS_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/remote-process-groups"); public static final Pattern REMOTE_PROCESS_GROUP_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/remote-process-groups/[a-f0-9\\-]{36}"); @@ -290,12 +318,19 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C public static final String PROVENANCE_URI = "/nifi-api/controller/provenance"; public static final Pattern PROVENANCE_QUERY_URI = Pattern.compile("/nifi-api/controller/provenance/[a-f0-9\\-]{36}"); public static final Pattern PROVENANCE_EVENT_URI = Pattern.compile("/nifi-api/controller/provenance/events/[0-9]+"); - + + public static final String CONTROLLER_SERVICES_URI = "/nifi-api/controller/controller-services/node"; + public static final Pattern CONTROLLER_SERVICE_URI_PATTERN = Pattern.compile("/nifi-api/controller/controller-services/node/[a-f0-9\\-]{36}"); + public static final Pattern CONTROLLER_SERVICE_REFERENCES_URI_PATTERN = Pattern.compile("/nifi-api/controller/controller-services/node/[a-f0-9\\-]{36}/references"); + public static final String REPORTING_TASKS_URI = "/nifi-api/controller/reporting-tasks/node"; + public static final Pattern REPORTING_TASK_URI_PATTERN = Pattern.compile("/nifi-api/controller/reporting-tasks/node/[a-f0-9\\-]{36}"); + private final NiFiProperties properties; private final HttpRequestReplicator httpRequestReplicator; private final HttpResponseMapper httpResponseMapper; private final DataFlowManagementService dataFlowManagementService; private final ClusterManagerProtocolSenderListener senderListener; + private final OptimisticLockingManager optimisticLockingManager; private final StringEncryptor encryptor; private final Queue pendingHeartbeats = new ConcurrentLinkedQueue<>(); private final ReentrantReadWriteLock resourceRWLock = new ReentrantReadWriteLock(); @@ -303,12 +338,11 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C private final ClusterManagerLock writeLock = new ClusterManagerLock(resourceRWLock.writeLock(), "Write"); private final Set nodes = new HashSet<>(); - private final Set reportingTasks = new HashSet<>(); + private final ConcurrentMap reportingTasks = new ConcurrentHashMap<>(); // null means the dataflow should be read from disk private StandardDataFlow cachedDataFlow = null; private NodeIdentifier primaryNodeId = null; - private Revision revision = new Revision(0L, ""); private Timer heartbeatMonitor; private Timer heartbeatProcessor; private volatile ClusterServicesBroadcaster servicesBroadcaster = null; @@ -329,7 +363,7 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C public WebClusterManager(final HttpRequestReplicator httpRequestReplicator, final HttpResponseMapper httpResponseMapper, final DataFlowManagementService dataFlowManagementService, final ClusterManagerProtocolSenderListener senderListener, - final NiFiProperties properties, final StringEncryptor encryptor) { + final NiFiProperties properties, final StringEncryptor encryptor, final OptimisticLockingManager optimisticLockingManager) { if (httpRequestReplicator == null) { throw new IllegalArgumentException("HttpRequestReplicator may not be null."); @@ -348,11 +382,11 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C this.httpResponseMapper = httpResponseMapper; this.dataFlowManagementService = dataFlowManagementService; this.properties = properties; - this.controllerServiceProvider = new StandardControllerServiceProvider(); this.bulletinRepository = new VolatileBulletinRepository(); this.instanceId = UUID.randomUUID().toString(); this.senderListener = senderListener; this.encryptor = encryptor; + this.optimisticLockingManager = optimisticLockingManager; senderListener.addHandler(this); senderListener.setBulletinRepository(bulletinRepository); @@ -393,9 +427,15 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C public void heartbeat() { } }, this, encryptor); + + // When we construct the scheduling agents, we can pass null for a lot of the arguments because we are only + // going to be scheduling Reporting Tasks. Otherwise, it would not be okay. processScheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, new TimerDrivenSchedulingAgent(null, reportingTaskEngine, null, encryptor)); + processScheduler.setSchedulingAgent(SchedulingStrategy.CRON_DRIVEN, new QuartzSchedulingAgent(null, reportingTaskEngine, null, encryptor)); processScheduler.setMaxThreadCount(SchedulingStrategy.TIMER_DRIVEN, 10); processScheduler.setMaxThreadCount(SchedulingStrategy.CRON_DRIVEN, 10); + + controllerServiceProvider = new StandardControllerServiceProvider(processScheduler, bulletinRepository); } public void start() throws IOException { @@ -429,14 +469,20 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C } // load flow + final ClusterDataFlow clusterDataFlow; if (dataFlowManagementService.isFlowCurrent()) { - final ClusterDataFlow clusterDataFlow = dataFlowManagementService.loadDataFlow(); + clusterDataFlow = dataFlowManagementService.loadDataFlow(); cachedDataFlow = clusterDataFlow.getDataFlow(); primaryNodeId = clusterDataFlow.getPrimaryNodeId(); } else { throw new IOException("Flow is not current."); } + final byte[] serializedServices = clusterDataFlow.getControllerServices(); + if ( serializedServices != null && serializedServices.length > 0 ) { + ControllerServiceLoader.loadControllerServices(this, new ByteArrayInputStream(serializedServices), encryptor, bulletinRepository, properties.getAutoResumeState()); + } + // start multicast broadcasting service, if configured if (servicesBroadcaster != null) { servicesBroadcaster.start(); @@ -446,8 +492,10 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C executeSafeModeTask(); // Load and start running Reporting Tasks - final File taskFile = new File(properties.getProperty(NiFiProperties.TASK_CONFIGURATION_FILE)); - reportingTasks.addAll(loadReportingTasks(taskFile)); + final byte[] serializedReportingTasks = clusterDataFlow.getReportingTasks(); + if ( serializedReportingTasks != null && serializedReportingTasks.length > 0 ) { + loadReportingTasks(serializedReportingTasks); + } } catch (final IOException ioe) { logger.warn("Failed to initialize cluster services due to: " + ioe, ioe); stop(); @@ -861,22 +909,17 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C reconnectionThread.start(); } - private List loadReportingTasks(final File taskConfigXml) { - final List tasks = new ArrayList<>(); - if (taskConfigXml == null) { - logger.info("No controller tasks to start"); - return tasks; - } + private Map loadReportingTasks(final byte[] serialized) { + final Map tasks = new HashMap<>(); try { - final URL schemaUrl = getClass().getResource("/ReportingTaskConfiguration.xsd"); - final Document document = parse(taskConfigXml, schemaUrl); + final Document document = parse(serialized); - final NodeList tasksNodes = document.getElementsByTagName("tasks"); + final NodeList tasksNodes = document.getElementsByTagName("reportingTasks"); final Element tasksElement = (Element) tasksNodes.item(0); //optional properties for all ReportingTasks - for (final Element taskElement : DomUtils.getChildElementsByTagName(tasksElement, "task")) { + for (final Element taskElement : DomUtils.getChildElementsByTagName(tasksElement, "reportingTask")) { //add global properties common to all tasks Map properties = new HashMap<>(); @@ -901,17 +944,20 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C final String taskSchedulingPeriod = DomUtils.getChild(taskElement, "schedulingPeriod").getTextContent().trim(); final String taskClass = DomUtils.getChild(taskElement, "class").getTextContent().trim(); - //optional task-specific properties - for (final Element optionalProperty : DomUtils.getChildElementsByTagName(taskElement, "property")) { - final String name = optionalProperty.getAttribute("name"); - final String value = optionalProperty.getTextContent().trim(); + final String scheduleStateValue = DomUtils.getChild(taskElement, "scheduledState").getTextContent().trim(); + final ScheduledState scheduledState = ScheduledState.valueOf(scheduleStateValue); + + // Reporting Task Properties + for (final Element property : DomUtils.getChildElementsByTagName(taskElement, "property")) { + final String name = DomUtils.getChildText(property, "name"); + final String value = DomUtils.getChildText(property, "value"); properties.put(name, value); } //set the class to be used for the configured reporting task final ReportingTaskNode reportingTaskNode; try { - reportingTaskNode = createReportingTask(taskClass, taskId); + reportingTaskNode = createReportingTask(taskClass, taskId, false); } catch (final ReportingTaskInstantiationException e) { logger.error("Unable to load reporting task {} due to {}", new Object[]{taskId, e}); if (logger.isDebugEnabled()) { @@ -922,27 +968,61 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C final ReportingTask reportingTask = reportingTaskNode.getReportingTask(); - final ReportingInitializationContext config = new StandardReportingInitializationContext(taskId, taskName, schedulingStrategy, taskSchedulingPeriod, this); + final ComponentLog componentLog = new SimpleProcessLogger(taskId, reportingTask); + final ReportingInitializationContext config = new StandardReportingInitializationContext(taskId, taskName, + schedulingStrategy, taskSchedulingPeriod, componentLog, this); reportingTask.initialize(config); + final String annotationData = DomUtils.getChildText(taskElement, "annotationData"); + if ( annotationData != null ) { + reportingTaskNode.setAnnotationData(annotationData.trim()); + } + final Map resolvedProps; try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { resolvedProps = new HashMap<>(); for (final Map.Entry entry : properties.entrySet()) { final PropertyDescriptor descriptor = reportingTask.getPropertyDescriptor(entry.getKey()); - resolvedProps.put(descriptor, entry.getValue()); + if ( entry.getValue() == null ) { + resolvedProps.put(descriptor, descriptor.getDefaultValue()); + } else { + resolvedProps.put(descriptor, entry.getValue()); + } } } for (final Map.Entry entry : resolvedProps.entrySet()) { - reportingTaskNode.setProperty(entry.getKey().getName(), entry.getValue()); + if ( entry.getValue() != null ) { + reportingTaskNode.setProperty(entry.getKey().getName(), entry.getValue()); + } + } + + final String comments = DomUtils.getChildText(taskElement, "comment"); + if ( comments != null ) { + reportingTaskNode.setComments(comments); } - processScheduler.schedule(reportingTaskNode); - tasks.add(reportingTaskNode); + reportingTaskNode.setScheduledState(scheduledState); + if ( ScheduledState.RUNNING.equals(scheduledState) ) { + if ( reportingTaskNode.isValid() ) { + try { + processScheduler.schedule(reportingTaskNode); + } catch (final Exception e) { + logger.error("Failed to start {} due to {}", reportingTaskNode, e); + if ( logger.isDebugEnabled() ) { + logger.error("", e); + } + } + } else { + logger.error("Failed to start {} because it is invalid due to {}", reportingTaskNode, reportingTaskNode.getValidationErrors()); + } + } + + + tasks.put(reportingTaskNode.getIdentifier(), reportingTaskNode); } } catch (final SAXException | ParserConfigurationException | IOException | DOMException | NumberFormatException | InitializationException t) { - logger.error("Unable to load reporting tasks from {} due to {}", new Object[]{taskConfigXml, t}); + logger.error("Unable to load reporting tasks due to {}", new Object[]{t}); if (logger.isDebugEnabled()) { logger.error("", t); } @@ -951,7 +1031,9 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C return tasks; } - private ReportingTaskNode createReportingTask(final String type, final String id) throws ReportingTaskInstantiationException { + + @Override + public ReportingTaskNode createReportingTask(final String type, final String id, final boolean firstTimeAdded) throws ReportingTaskInstantiationException { if (type == null) { throw new NullPointerException(); } @@ -981,14 +1063,22 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(this); final ReportingTaskNode taskNode = new ClusteredReportingTaskNode(task, id, processScheduler, new ClusteredEventAccess(this), bulletinRepository, controllerServiceProvider, validationContextFactory); + taskNode.setName(task.getClass().getSimpleName()); + + reportingTasks.put(id, taskNode); + if ( firstTimeAdded ) { + try (final NarCloseable x = NarCloseable.withNarLoader()) { + ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, task); + } catch (final Exception e) { + throw new ComponentLifeCycleException("Failed to invoke On-Added Lifecycle methods of " + task, e); + } + } + return taskNode; } - private Document parse(final File xmlFile, final URL schemaUrl) throws SAXException, ParserConfigurationException, IOException { - final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI); - final Schema schema = schemaFactory.newSchema(schemaUrl); + private Document parse(final byte[] serialized) throws SAXException, ParserConfigurationException, IOException { final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance(); - docFactory.setSchema(schema); final DocumentBuilder builder = docFactory.newDocumentBuilder(); builder.setErrorHandler(new org.xml.sax.ErrorHandler() { @@ -1021,12 +1111,7 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C }); // build the docuemnt - final Document document = builder.parse(xmlFile); - - // ensure schema compliance - final Validator validator = schema.newValidator(); - validator.validate(new DOMSource(document)); - + final Document document = builder.parse(new ByteArrayInputStream(serialized)); return document; } @@ -1287,7 +1372,19 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C writeLock.unlock("handleControllerStartupFailure"); } } - + + /** + * Adds an instance of a specified controller service. + * + * @param type + * @param id + * @param properties + * @return + */ + @Override + public ControllerServiceNode createControllerService(final String type, final String id, final boolean firstTimeAdded) { + return controllerServiceProvider.createControllerService(type, id, firstTimeAdded); + } @Override public ControllerService getControllerService(String serviceIdentifier) { @@ -1310,10 +1407,15 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C } @Override - public ControllerServiceNode createControllerService(final String type, final String id, final boolean firstTimeAdded) { - return controllerServiceProvider.createControllerService(type, id, firstTimeAdded); + public boolean isControllerServiceEnabling(final String serviceIdentifier) { + return controllerServiceProvider.isControllerServiceEnabling(serviceIdentifier); } + @Override + public String getControllerServiceName(final String serviceIdentifier) { + return controllerServiceProvider.getControllerServiceName(serviceIdentifier); + } + @Override public void removeControllerService(final ControllerServiceNode serviceNode) { controllerServiceProvider.removeControllerService(serviceNode); @@ -1325,11 +1427,215 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C controllerServiceProvider.enableControllerService(serviceNode); } + @Override + public void enableControllerServices(final Collection serviceNodes) { + controllerServiceProvider.enableControllerServices(serviceNodes); + } + @Override public void disableControllerService(final ControllerServiceNode serviceNode) { controllerServiceProvider.disableControllerService(serviceNode); } + @Override + public Set getAllControllerServices() { + return controllerServiceProvider.getAllControllerServices(); + } + + + @Override + public void disableReferencingServices(final ControllerServiceNode serviceNode) { + controllerServiceProvider.disableReferencingServices(serviceNode); + } + + @Override + public void enableReferencingServices(final ControllerServiceNode serviceNode) { + controllerServiceProvider.enableReferencingServices(serviceNode); + } + + @Override + public void scheduleReferencingComponents(final ControllerServiceNode serviceNode) { + controllerServiceProvider.scheduleReferencingComponents(serviceNode); + } + + @Override + public void unscheduleReferencingComponents(final ControllerServiceNode serviceNode) { + controllerServiceProvider.unscheduleReferencingComponents(serviceNode); + } + + @Override + public void verifyCanEnableReferencingServices(final ControllerServiceNode serviceNode) { + controllerServiceProvider.verifyCanEnableReferencingServices(serviceNode); + } + + @Override + public void verifyCanScheduleReferencingComponents(final ControllerServiceNode serviceNode) { + controllerServiceProvider.verifyCanScheduleReferencingComponents(serviceNode); + } + + @Override + public void verifyCanDisableReferencingServices(final ControllerServiceNode serviceNode) { + controllerServiceProvider.verifyCanDisableReferencingServices(serviceNode); + } + + @Override + public void verifyCanStopReferencingComponents(final ControllerServiceNode serviceNode) { + controllerServiceProvider.verifyCanStopReferencingComponents(serviceNode); + } + + private byte[] serialize(final Document doc) throws TransformerException { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final DOMSource domSource = new DOMSource(doc); + final StreamResult streamResult = new StreamResult(baos); + + // configure the transformer and convert the DOM + final TransformerFactory transformFactory = TransformerFactory.newInstance(); + final Transformer transformer = transformFactory.newTransformer(); + transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "2"); + transformer.setOutputProperty(OutputKeys.INDENT, "yes"); + + // transform the document to byte stream + transformer.transform(domSource, streamResult); + return baos.toByteArray(); + } + + private byte[] serializeControllerServices() throws ParserConfigurationException, TransformerException { + final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance(); + final DocumentBuilder docBuilder = docFactory.newDocumentBuilder(); + final Document document = docBuilder.newDocument(); + final Element rootElement = document.createElement("controllerServices"); + document.appendChild(rootElement); + + for ( final ControllerServiceNode serviceNode : getAllControllerServices() ) { + StandardFlowSerializer.addControllerService(rootElement, serviceNode, encryptor); + } + + return serialize(document); + } + + private byte[] serializeReportingTasks() throws ParserConfigurationException, TransformerException { + final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance(); + final DocumentBuilder docBuilder = docFactory.newDocumentBuilder(); + final Document document = docBuilder.newDocument(); + final Element rootElement = document.createElement("reportingTasks"); + document.appendChild(rootElement); + + for ( final ReportingTaskNode taskNode : getAllReportingTasks() ) { + StandardFlowSerializer.addReportingTask(rootElement, taskNode, encryptor); + } + + return serialize(document); + } + + + public void saveControllerServices() { + try { + dataFlowManagementService.updateControllerServices(serializeControllerServices()); + } catch (final Exception e) { + logger.error("Failed to save changes to NCM's Controller Services; changes may be lost on restart due to " + e); + if ( logger.isDebugEnabled() ) { + logger.error("", e); + } + + getBulletinRepository().addBulletin(BulletinFactory.createBulletin("Controller Services", Severity.ERROR.name(), + "Failed to save changes to NCM's Controller Services; changes may be lost on restart. See logs for more details.")); + } + } + + public void saveReportingTasks() { + try { + dataFlowManagementService.updateReportingTasks(serializeReportingTasks()); + } catch (final Exception e) { + logger.error("Failed to save changes to NCM's Reporting Tasks; changes may be lost on restart due to " + e); + if ( logger.isDebugEnabled() ) { + logger.error("", e); + } + + getBulletinRepository().addBulletin(BulletinFactory.createBulletin("Reporting Tasks", Severity.ERROR.name(), + "Failed to save changes to NCM's Reporting Tasks; changes may be lost on restart. See logs for more details.")); + } + } + + @Override + public Set getAllReportingTasks() { + readLock.lock(); + try { + return new HashSet<>(reportingTasks.values()); + } finally { + readLock.unlock("getReportingTasks"); + } + } + + @Override + public ReportingTaskNode getReportingTaskNode(final String taskId) { + readLock.lock(); + try { + return reportingTasks.get(taskId); + } finally { + readLock.unlock("getReportingTaskNode"); + } + } + + @Override + public void startReportingTask(final ReportingTaskNode reportingTaskNode) { + reportingTaskNode.verifyCanStart(); + processScheduler.schedule(reportingTaskNode); + } + + + @Override + public void stopReportingTask(final ReportingTaskNode reportingTaskNode) { + reportingTaskNode.verifyCanStop(); + processScheduler.unschedule(reportingTaskNode); + } + + @Override + public void removeReportingTask(final ReportingTaskNode reportingTaskNode) { + writeLock.lock(); + try { + final ReportingTaskNode existing = reportingTasks.get(reportingTaskNode.getIdentifier()); + if ( existing == null || existing != reportingTaskNode ) { + throw new IllegalStateException("Reporting Task " + reportingTaskNode + " does not exist in this Flow"); + } + + reportingTaskNode.verifyCanDelete(); + + try (final NarCloseable x = NarCloseable.withNarLoader()) { + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, reportingTaskNode.getReportingTask(), reportingTaskNode.getConfigurationContext()); + } + + for ( final Map.Entry entry : reportingTaskNode.getProperties().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(reportingTaskNode); + } + } + } + } + + reportingTasks.remove(reportingTaskNode.getIdentifier()); + } finally { + writeLock.unlock("removeReportingTask"); + } + } + + + @Override + public void disableReportingTask(final ReportingTaskNode reportingTask) { + reportingTask.verifyCanDisable(); + processScheduler.disableReportingTask(reportingTask); + } + + @Override + public void enableReportingTask(final ReportingTaskNode reportingTask) { + reportingTask.verifyCanEnable(); + processScheduler.enableReportingTask(reportingTask); + } + /** * Handle a bulletins message. @@ -1966,65 +2272,114 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C // check if this request can change the flow final boolean mutableRequest = canChangeNodeState(method, uri); - // update headers to contain cluster contextual information to send to the node - final Map updatedHeaders = new HashMap<>(headers); - final ClusterContext clusterCtx = new ClusterContextImpl(); - clusterCtx.setRequestSentByClusterManager(true); // indicate request is sent from cluster manager - clusterCtx.setRevision(revision); + final ObjectHolder holder = new ObjectHolder<>(null); + final UpdateRevision federateRequest = new UpdateRevision() { + @Override + public Revision execute(Revision currentRevision) { + // update headers to contain cluster contextual information to send to the node + final Map updatedHeaders = new HashMap<>(headers); + final ClusterContext clusterCtx = new ClusterContextImpl(); + clusterCtx.setRequestSentByClusterManager(true); // indicate request is sent from cluster manager + clusterCtx.setRevision(currentRevision); - // serialize cluster context and add to request header - final String serializedClusterCtx = WebUtils.serializeObjectToHex(clusterCtx); - updatedHeaders.put(CLUSTER_CONTEXT_HTTP_HEADER, serializedClusterCtx); + // serialize cluster context and add to request header + final String serializedClusterCtx = WebUtils.serializeObjectToHex(clusterCtx); + updatedHeaders.put(CLUSTER_CONTEXT_HTTP_HEADER, serializedClusterCtx); - // if the request is mutable, we need to verify that it is a valid request for all nodes in the cluster. - if (mutableRequest) { - updatedHeaders.put(NCM_EXPECTS_HTTP_HEADER, "150-NodeContinue"); + // if the request is mutable, we need to verify that it is a valid request for all nodes in the cluster. + if (mutableRequest) { + updatedHeaders.put(NCM_EXPECTS_HTTP_HEADER, "150-NodeContinue"); - final Set nodeResponses; - if (entity == null) { - nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, parameters, updatedHeaders); - } else { - nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, entity, updatedHeaders); - } - - updatedHeaders.remove(NCM_EXPECTS_HTTP_HEADER); - - for (final NodeResponse response : nodeResponses) { - if (response.getStatus() != NODE_CONTINUE_STATUS_CODE) { - final String nodeDescription = response.getNodeId().getApiAddress() + ":" + response.getNodeId().getApiPort(); - final ClientResponse clientResponse = response.getClientResponse(); - if (clientResponse == null) { - throw new IllegalClusterStateException("Node " + nodeDescription + " is unable to fulfill this request due to: Unexpected Response Code " + response.getStatus()); + final Set nodeResponses; + if (entity == null) { + nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, parameters, updatedHeaders); + } else { + nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, entity, updatedHeaders); } - final String nodeExplanation = clientResponse.getEntity(String.class); - throw new IllegalClusterStateException("Node " + nodeDescription + " is unable to fulfill this request due to: " + nodeExplanation, response.getThrowable()); + + updatedHeaders.remove(NCM_EXPECTS_HTTP_HEADER); + + for (final NodeResponse response : nodeResponses) { + if (response.getStatus() != NODE_CONTINUE_STATUS_CODE) { + final String nodeDescription = response.getNodeId().getApiAddress() + ":" + response.getNodeId().getApiPort(); + final ClientResponse clientResponse = response.getClientResponse(); + if (clientResponse == null) { + throw new IllegalClusterStateException("Node " + nodeDescription + " is unable to fulfill this request due to: Unexpected Response Code " + response.getStatus()); + } + final String nodeExplanation = clientResponse.getEntity(String.class); + throw new IllegalClusterStateException("Node " + nodeDescription + " is unable to fulfill this request due to: " + nodeExplanation, response.getThrowable()); + } + } + + // set flow state to unknown to denote a mutable request replication in progress + logger.debug("Setting Flow State to UNKNOWN due to mutable request to {} {}", method, uri); + notifyDataFlowManagmentServiceOfFlowStateChange(PersistedFlowState.UNKNOWN); } - } - // set flow state to unknown to denote a mutable request replication in progress - logger.debug("Setting Flow State to UNKNOWN due to mutable request to {} {}", method, uri); - notifyDataFlowManagmentServiceOfFlowStateChange(PersistedFlowState.UNKNOWN); + // replicate request + final Set nodeResponses; + try { + if (entity == null) { + nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, parameters, updatedHeaders); + } else { + nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, entity, updatedHeaders); + } + } catch (final UriConstructionException uce) { + // request was not replicated, so mark the flow with its original state + if (mutableRequest) { + notifyDataFlowManagmentServiceOfFlowStateChange(originalPersistedFlowState); + } + + throw uce; + } + + // merge the response + final NodeResponse clientResponse = mergeResponses(uri, method, nodeResponses, mutableRequest); + holder.set(clientResponse); + + // if we have a response get the updated cluster context for auditing and revision updating + Revision updatedRevision = null; + if (mutableRequest && clientResponse != null) { + try { + // get the cluster context from the response header + final String serializedClusterContext = clientResponse.getClientResponse().getHeaders().getFirst(CLUSTER_CONTEXT_HTTP_HEADER); + if (StringUtils.isNotBlank(serializedClusterContext)) { + // deserialize object + final Serializable clusterContextObj = WebUtils.deserializeHexToObject(serializedClusterContext); + + // if we have a valid object, audit the actions + if (clusterContextObj instanceof ClusterContext) { + final ClusterContext clusterContext = (ClusterContext) clusterContextObj; + if (auditService != null) { + try { + auditService.addActions(clusterContext.getActions()); + } catch (Throwable t) { + logger.warn("Unable to record actions: " + t.getMessage()); + if (logger.isDebugEnabled()) { + logger.warn(StringUtils.EMPTY, t); + } + } + } + updatedRevision = clusterContext.getRevision(); + } + } + } catch (final ClassNotFoundException cnfe) { + logger.warn("Classpath issue detected because failed to deserialize cluster context from node response due to: " + cnfe, cnfe); + } + } + + return updatedRevision; + } + }; + + // federate the request and lock on the revision + if (mutableRequest) { + optimisticLockingManager.setRevision(federateRequest); + } else { + federateRequest.execute(optimisticLockingManager.getLastModification().getRevision()); } - - // replicate request - final Set nodeResponses; - try { - if (entity == null) { - nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, parameters, updatedHeaders); - } else { - nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, entity, updatedHeaders); - } - } catch (final UriConstructionException uce) { - // request was not replicated, so mark the flow with its original state - if (mutableRequest) { - notifyDataFlowManagmentServiceOfFlowStateChange(originalPersistedFlowState); - } - - throw uce; - } - - final NodeResponse clientResponse = mergeResponses(uri, method, nodeResponses, mutableRequest); - return clientResponse; + + return holder.get(); } private static boolean isProcessorsEndpoint(final URI uri, final String method) { @@ -2032,7 +2387,7 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C } private static boolean isProcessorEndpoint(final URI uri, final String method) { - if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && PROCESSOR_URI_PATTERN.matcher(uri.getPath()).matches()) { + if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && (PROCESSOR_URI_PATTERN.matcher(uri.getPath()).matches() || CLUSTER_PROCESSOR_URI_PATTERN.matcher(uri.getPath()).matches()) ) { return true; } else if ("POST".equalsIgnoreCase(method) && PROCESSORS_URI_PATTERN.matcher(uri.getPath()).matches()) { return true; @@ -2079,13 +2434,51 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C private static boolean isProvenanceEventEndpoint(final URI uri, final String method) { return "GET".equalsIgnoreCase(method) && PROVENANCE_EVENT_URI.matcher(uri.getPath()).matches(); } + + private static boolean isControllerServicesEndpoint(final URI uri, final String method) { + return "GET".equalsIgnoreCase(method) && CONTROLLER_SERVICES_URI.equals(uri.getPath()); + } + + private static boolean isControllerServiceEndpoint(final URI uri, final String method) { + if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && CONTROLLER_SERVICE_URI_PATTERN.matcher(uri.getPath()).matches()) { + return true; + } else if ("POST".equalsIgnoreCase(method) && CONTROLLER_SERVICES_URI.equals(uri.getPath())) { + return true; + } + + return false; + } + + private static boolean isControllerServiceReferenceEndpoint(final URI uri, final String method) { + if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && CONTROLLER_SERVICE_REFERENCES_URI_PATTERN.matcher(uri.getPath()).matches()) { + return true; + } + + return false; + } + + private static boolean isReportingTasksEndpoint(final URI uri, final String method) { + return "GET".equalsIgnoreCase(method) && REPORTING_TASKS_URI.equals(uri.getPath()); + } + + private static boolean isReportingTaskEndpoint(final URI uri, final String method) { + if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && REPORTING_TASK_URI_PATTERN.matcher(uri.getPath()).matches()) { + return true; + } else if ("POST".equalsIgnoreCase(method) && REPORTING_TASKS_URI.equals(uri.getPath())) { + return true; + } + + return false; + } static boolean isResponseInterpreted(final URI uri, final String method) { return isProcessorsEndpoint(uri, method) || isProcessorEndpoint(uri, method) || isRemoteProcessGroupsEndpoint(uri, method) || isRemoteProcessGroupEndpoint(uri, method) || isProcessGroupEndpoint(uri, method) || isTemplateEndpoint(uri, method) || isFlowSnippetEndpoint(uri, method) - || isProvenanceQueryEndpoint(uri, method) || isProvenanceEventEndpoint(uri, method); + || isProvenanceQueryEndpoint(uri, method) || isProvenanceEventEndpoint(uri, method) + || isControllerServicesEndpoint(uri, method) || isControllerServiceEndpoint(uri, method) || isControllerServiceReferenceEndpoint(uri, method) + || isReportingTasksEndpoint(uri, method) || isReportingTaskEndpoint(uri, method); } private void mergeProcessorValidationErrors(final ProcessorDTO processor, Map processorMap) { @@ -2095,37 +2488,12 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C final NodeIdentifier nodeId = nodeEntry.getKey(); final ProcessorDTO nodeProcessor = nodeEntry.getValue(); - // get the processor's validation errors and put them into a map - // where the key is the validation error and the value is the set of all - // nodes that reported that validation error. - final Collection nodeValidationErrors = nodeProcessor.getValidationErrors(); - if (nodeValidationErrors != null) { - for (final String nodeValidationError : nodeValidationErrors) { - Set nodeSet = validationErrorMap.get(nodeValidationError); - if (nodeSet == null) { - nodeSet = new HashSet<>(); - validationErrorMap.put(nodeValidationError, nodeSet); - } - nodeSet.add(nodeId); - } - } + // merge the validation errors + mergeValidationErrors(validationErrorMap, nodeId, nodeProcessor.getValidationErrors()); } - final Set normalizedValidationErrors = new HashSet<>(); - for (final Map.Entry> validationEntry : validationErrorMap.entrySet()) { - final String msg = validationEntry.getKey(); - final Set nodeIds = validationEntry.getValue(); - - if (nodeIds.size() == processorMap.size()) { - normalizedValidationErrors.add(msg); - } else { - for (final NodeIdentifier nodeId : nodeIds) { - normalizedValidationErrors.add(nodeId.getApiAddress() + ":" + nodeId.getApiPort() + " -- " + msg); - } - } - } - - processor.setValidationErrors(normalizedValidationErrors); + // set the merged the validation errors + processor.setValidationErrors(normalizedMergedValidationErrors(validationErrorMap, processorMap.size())); } private void mergeProvenanceQueryResults(final ProvenanceDTO provenanceDto, final Map resultMap, final Set problematicResponses) { @@ -2293,7 +2661,158 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C remoteProcessGroup.setAuthorizationIssues(mergedAuthorizationIssues); } } + + private void mergeControllerServiceReferences(final Set referencingComponents, final Map> referencingComponentMap) { + final Map activeThreadCounts = new HashMap<>(); + final Map states = new HashMap<>(); + for (final Map.Entry> nodeEntry : referencingComponentMap.entrySet()) { + final Set nodeReferencingComponents = nodeEntry.getValue(); + // go through all the nodes referencing components + if ( nodeReferencingComponents != null ) { + for (final ControllerServiceReferencingComponentDTO nodeReferencingComponent : nodeReferencingComponents) { + // handle active thread counts + if (nodeReferencingComponent.getActiveThreadCount() != null && nodeReferencingComponent.getActiveThreadCount() > 0) { + final Integer current = activeThreadCounts.get(nodeReferencingComponent.getId()); + if (current == null) { + activeThreadCounts.put(nodeReferencingComponent.getId(), nodeReferencingComponent.getActiveThreadCount()); + } else { + activeThreadCounts.put(nodeReferencingComponent.getId(), nodeReferencingComponent.getActiveThreadCount() + current); + } + } + + // handle controller service state + final String state = states.get(nodeReferencingComponent.getId()); + if (state == null) { + if (ControllerServiceState.DISABLING.name().equals(nodeReferencingComponent.getState())) { + states.put(nodeReferencingComponent.getId(), ControllerServiceState.DISABLING.name()); + } else if (ControllerServiceState.ENABLING.name().equals(nodeReferencingComponent.getState())) { + states.put(nodeReferencingComponent.getId(), ControllerServiceState.ENABLING.name()); + } + } + } + } + } + + // go through each referencing components + for (final ControllerServiceReferencingComponentDTO referencingComponent : referencingComponents) { + final Integer activeThreadCount = activeThreadCounts.get(referencingComponent.getId()); + if (activeThreadCount != null) { + referencingComponent.setActiveThreadCount(activeThreadCount); + } + + final String state = states.get(referencingComponent.getId()); + if (state != null) { + referencingComponent.setState(state); + } + } + } + + private void mergeControllerService(final ControllerServiceDTO controllerService, final Map controllerServiceMap) { + final Map> validationErrorMap = new HashMap<>(); + final Set referencingComponents = controllerService.getReferencingComponents(); + final Map> nodeReferencingComponentsMap = new HashMap<>(); + + String state = null; + for (final Map.Entry nodeEntry : controllerServiceMap.entrySet()) { + final NodeIdentifier nodeId = nodeEntry.getKey(); + final ControllerServiceDTO nodeControllerService = nodeEntry.getValue(); + + if (state == null) { + if (ControllerServiceState.DISABLING.name().equals(nodeControllerService.getState())) { + state = ControllerServiceState.DISABLING.name(); + } else if (ControllerServiceState.ENABLING.name().equals(nodeControllerService.getState())) { + state = ControllerServiceState.ENABLING.name(); + } + } + + for (final ControllerServiceReferencingComponentDTO nodeReferencingComponents : nodeControllerService.getReferencingComponents()) { + nodeReferencingComponentsMap.put(nodeId, nodeReferencingComponents.getReferencingComponents()); + } + + // merge the validation errors + mergeValidationErrors(validationErrorMap, nodeId, nodeControllerService.getValidationErrors()); + } + + // merge the referencing components + mergeControllerServiceReferences(referencingComponents, nodeReferencingComponentsMap); + + // store the 'transition' state is applicable + if (state != null) { + controllerService.setState(state); + } + + // set the merged the validation errors + controllerService.setValidationErrors(normalizedMergedValidationErrors(validationErrorMap, controllerServiceMap.size())); + } + + private void mergeReportingTask(final ReportingTaskDTO reportingTask, final Map reportingTaskMap) { + final Map> validationErrorMap = new HashMap<>(); + + int activeThreadCount = 0; + for (final Map.Entry nodeEntry : reportingTaskMap.entrySet()) { + final NodeIdentifier nodeId = nodeEntry.getKey(); + final ReportingTaskDTO nodeReportingTask = nodeEntry.getValue(); + + if (nodeReportingTask.getActiveThreadCount() != null) { + activeThreadCount += nodeReportingTask.getActiveThreadCount(); + } + + // merge the validation errors + mergeValidationErrors(validationErrorMap, nodeId, nodeReportingTask.getValidationErrors()); + } + + // set the merged active thread counts + reportingTask.setActiveThreadCount(activeThreadCount); + + // set the merged the validation errors + reportingTask.setValidationErrors(normalizedMergedValidationErrors(validationErrorMap, reportingTaskMap.size())); + } + + /** + * Merges the validation errors into the specified map, recording the corresponding node identifier. + * + * @param validationErrorMap + * @param nodeId + * @param nodeValidationErrors + */ + public void mergeValidationErrors(final Map> validationErrorMap, final NodeIdentifier nodeId, final Collection nodeValidationErrors) { + if (nodeValidationErrors != null) { + for (final String nodeValidationError : nodeValidationErrors) { + Set nodeSet = validationErrorMap.get(nodeValidationError); + if (nodeSet == null) { + nodeSet = new HashSet<>(); + validationErrorMap.put(nodeValidationError, nodeSet); + } + nodeSet.add(nodeId); + } + } + } + + /** + * Normalizes the validation errors by prepending the corresponding nodes when the error does not exist across all nodes. + * + * @param validationErrorMap + * @param totalNodes + * @return + */ + public Set normalizedMergedValidationErrors(final Map> validationErrorMap, int totalNodes) { + final Set normalizedValidationErrors = new HashSet<>(); + for (final Map.Entry> validationEntry : validationErrorMap.entrySet()) { + final String msg = validationEntry.getKey(); + final Set nodeIds = validationEntry.getValue(); + + if (nodeIds.size() == totalNodes) { + normalizedValidationErrors.add(msg); + } else { + for (final NodeIdentifier nodeId : nodeIds) { + normalizedValidationErrors.add(nodeId.getApiAddress() + ":" + nodeId.getApiPort() + " -- " + msg); + } + } + } + return normalizedValidationErrors; + } + // requires write lock to be already acquired unless request is not mutable private NodeResponse mergeResponses(final URI uri, final String method, final Set nodeResponses, final boolean mutableRequest) { // holds the one response of all the node responses to return to the client @@ -2582,6 +3101,126 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C event.setClusterNodeId(nodeId.getId()); event.setClusterNodeAddress(nodeId.getApiAddress() + ":" + nodeId.getApiPort()); + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && isControllerServiceEndpoint(uri, method)) { + final ControllerServiceEntity responseEntity = clientResponse.getClientResponse().getEntity(ControllerServiceEntity.class); + final ControllerServiceDTO controllerService = responseEntity.getControllerService(); + + final Map resultsMap = new HashMap<>(); + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final ControllerServiceEntity nodeResponseEntity = (nodeResponse == clientResponse) ? responseEntity : nodeResponse.getClientResponse().getEntity(ControllerServiceEntity.class); + final ControllerServiceDTO nodeControllerService = nodeResponseEntity.getControllerService(); + + resultsMap.put(nodeResponse.getNodeId(), nodeControllerService); + } + mergeControllerService(controllerService, resultsMap); + + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && isControllerServicesEndpoint(uri, method)) { + final ControllerServicesEntity responseEntity = clientResponse.getClientResponse().getEntity(ControllerServicesEntity.class); + final Set controllerServices = responseEntity.getControllerServices(); + + final Map> controllerServiceMap = new HashMap<>(); + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final ControllerServicesEntity nodeResponseEntity = (nodeResponse == clientResponse) ? responseEntity : nodeResponse.getClientResponse().getEntity(ControllerServicesEntity.class); + final Set nodeControllerServices = nodeResponseEntity.getControllerServices(); + + for (final ControllerServiceDTO nodeControllerService : nodeControllerServices) { + Map innerMap = controllerServiceMap.get(nodeControllerService.getId()); + if (innerMap == null) { + innerMap = new HashMap<>(); + controllerServiceMap.put(nodeControllerService.getId(), innerMap); + } + + innerMap.put(nodeResponse.getNodeId(), nodeControllerService); + } + } + + for (final ControllerServiceDTO controllerService : controllerServices) { + final String procId = controllerService.getId(); + final Map mergeMap = controllerServiceMap.get(procId); + + mergeControllerService(controllerService, mergeMap); + } + + // create a new client response + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && isControllerServiceReferenceEndpoint(uri, method)) { + final ControllerServiceReferencingComponentsEntity responseEntity = clientResponse.getClientResponse().getEntity(ControllerServiceReferencingComponentsEntity.class); + final Set referencingComponents = responseEntity.getControllerServiceReferencingComponents(); + + final Map> resultsMap = new HashMap<>(); + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final ControllerServiceReferencingComponentsEntity nodeResponseEntity = (nodeResponse == clientResponse) ? responseEntity : nodeResponse.getClientResponse().getEntity(ControllerServiceReferencingComponentsEntity.class); + final Set nodeReferencingComponents = nodeResponseEntity.getControllerServiceReferencingComponents(); + + resultsMap.put(nodeResponse.getNodeId(), nodeReferencingComponents); + } + mergeControllerServiceReferences(referencingComponents, resultsMap); + + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && isReportingTaskEndpoint(uri, method)) { + final ReportingTaskEntity responseEntity = clientResponse.getClientResponse().getEntity(ReportingTaskEntity.class); + final ReportingTaskDTO reportingTask = responseEntity.getReportingTask(); + + final Map resultsMap = new HashMap<>(); + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final ReportingTaskEntity nodeResponseEntity = (nodeResponse == clientResponse) ? responseEntity : nodeResponse.getClientResponse().getEntity(ReportingTaskEntity.class); + final ReportingTaskDTO nodeReportingTask = nodeResponseEntity.getReportingTask(); + + resultsMap.put(nodeResponse.getNodeId(), nodeReportingTask); + } + mergeReportingTask(reportingTask, resultsMap); + + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && isReportingTasksEndpoint(uri, method)) { + final ReportingTasksEntity responseEntity = clientResponse.getClientResponse().getEntity(ReportingTasksEntity.class); + final Set reportingTaskSet = responseEntity.getReportingTasks(); + + final Map> reportingTaskMap = new HashMap<>(); + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final ReportingTasksEntity nodeResponseEntity = (nodeResponse == clientResponse) ? responseEntity : nodeResponse.getClientResponse().getEntity(ReportingTasksEntity.class); + final Set nodeReportingTasks = nodeResponseEntity.getReportingTasks(); + + for (final ReportingTaskDTO nodeReportingTask : nodeReportingTasks) { + Map innerMap = reportingTaskMap.get(nodeReportingTask.getId()); + if (innerMap == null) { + innerMap = new HashMap<>(); + reportingTaskMap.put(nodeReportingTask.getId(), innerMap); + } + + innerMap.put(nodeResponse.getNodeId(), nodeReportingTask); + } + } + + for (final ReportingTaskDTO reportingTask : reportingTaskSet) { + final String procId = reportingTask.getId(); + final Map mergeMap = reportingTaskMap.get(procId); + + mergeReportingTask(reportingTask, mergeMap); + } + + // create a new client response clientResponse = new NodeResponse(clientResponse, responseEntity); } else { if (!nodeResponsesToDrain.isEmpty()) { @@ -2616,36 +3255,6 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C logger.warn("All nodes failed to process URI {}. As a result, no node will be disconnected from cluster", uri); } } - - // if at least one node satisfied the request, then audit the action - if (hasClientResponse) { - try { - // get the cluster context from the response header - final String serializedClusterContext = clientResponse.getClientResponse().getHeaders().getFirst(CLUSTER_CONTEXT_HTTP_HEADER); - if (StringUtils.isNotBlank(serializedClusterContext)) { - // deserialize object - final Serializable clusterContextObj = WebUtils.deserializeHexToObject(serializedClusterContext); - - // if we have a valid object, audit the actions - if (clusterContextObj instanceof ClusterContext) { - final ClusterContext clusterContext = (ClusterContext) clusterContextObj; - if (auditService != null) { - try { - auditService.addActions(clusterContext.getActions()); - } catch (Throwable t) { - logger.warn("Unable to record actions: " + t.getMessage()); - if (logger.isDebugEnabled()) { - logger.warn(StringUtils.EMPTY, t); - } - } - } - revision = clusterContext.getRevision(); - } - } - } catch (final ClassNotFoundException cnfe) { - logger.warn("Classpath issue detected because failed to deserialize cluster context from node response due to: " + cnfe, cnfe); - } - } } return clientResponse; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java index 7169730a56..d3cff3b26f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java @@ -16,7 +16,6 @@ */ package org.apache.nifi.cluster.spring; -import java.nio.file.Paths; import org.apache.nifi.admin.service.AuditService; import org.apache.nifi.cluster.event.EventManager; import org.apache.nifi.cluster.firewall.ClusterNodeFirewall; @@ -26,11 +25,11 @@ import org.apache.nifi.cluster.manager.HttpResponseMapper; import org.apache.nifi.cluster.manager.impl.WebClusterManager; import org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderListener; import org.apache.nifi.cluster.protocol.impl.ClusterServicesBroadcaster; -import org.apache.nifi.controller.service.ControllerServiceLoader; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.io.socket.multicast.DiscoverableService; import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.OptimisticLockingManager; import org.springframework.beans.BeansException; import org.springframework.beans.factory.FactoryBean; import org.springframework.context.ApplicationContext; @@ -50,6 +49,8 @@ public class WebClusterManagerFactoryBean implements FactoryBean, ApplicationCon private NiFiProperties properties; private StringEncryptor encryptor; + + private OptimisticLockingManager optimisticLockingManager; @Override public Object getObject() throws Exception { @@ -62,13 +63,6 @@ public class WebClusterManagerFactoryBean implements FactoryBean, ApplicationCon */ return null; } else if (clusterManager == null) { - - // get the service configuration path (fail early) - final String serviceConfigurationFile = properties.getProperty(NiFiProperties.SERVICE_CONFIGURATION_FILE); - if (serviceConfigurationFile == null) { - throw new NullPointerException("The service configuration file has not been specified."); - } - final HttpRequestReplicator requestReplicator = applicationContext.getBean("httpRequestReplicator", HttpRequestReplicator.class); final HttpResponseMapper responseMapper = applicationContext.getBean("httpResponseMapper", HttpResponseMapper.class); final DataFlowManagementService dataFlowService = applicationContext.getBean("dataFlowManagementService", DataFlowManagementService.class); @@ -81,7 +75,8 @@ public class WebClusterManagerFactoryBean implements FactoryBean, ApplicationCon dataFlowService, senderListener, properties, - encryptor + encryptor, + optimisticLockingManager ); // set the service broadcaster @@ -106,10 +101,6 @@ public class WebClusterManagerFactoryBean implements FactoryBean, ApplicationCon // set the audit service clusterManager.setAuditService(applicationContext.getBean("auditService", AuditService.class)); - - // load the controller services - final ControllerServiceLoader serviceLoader = new ControllerServiceLoader(Paths.get(serviceConfigurationFile)); - serviceLoader.loadControllerServices(clusterManager); } return clusterManager; } @@ -136,4 +127,8 @@ public class WebClusterManagerFactoryBean implements FactoryBean, ApplicationCon public void setEncryptor(final StringEncryptor encryptor) { this.encryptor = encryptor; } + + public void setOptimisticLockingManager(OptimisticLockingManager optimisticLockingManager) { + this.optimisticLockingManager = optimisticLockingManager; + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml index 68c29bca00..72c7bff22a 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml @@ -91,10 +91,14 @@ + + + + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/.gitignore b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/.gitignore index ea8c4bf7f3..29546b567b 100755 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/.gitignore +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/.gitignore @@ -1 +1,2 @@ /target +/target/ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java index ef4b72aad8..c44161fac8 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java @@ -23,6 +23,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; @@ -149,6 +150,16 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone final PropertyDescriptor descriptor = component.getPropertyDescriptor(name); String value = null; if (!descriptor.isRequired() && (value = properties.remove(descriptor)) != null) { + + if ( descriptor.getControllerServiceDefinition() != null ) { + if (value != null) { + final ControllerServiceNode oldNode = serviceProvider.getControllerServiceNode(value); + if (oldNode != null) { + oldNode.removeReference(this); + } + } + } + component.onPropertyModified(descriptor, value, null); return true; } @@ -250,12 +261,17 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone return true; } + @Override public Collection getValidationErrors() { + return getValidationErrors(Collections.emptySet()); + } + + public Collection getValidationErrors(final Set serviceIdentifiersNotToValidate) { final List results = new ArrayList<>(); lock.lock(); try { - final ValidationContext validationContext = validationContextFactory.newValidationContext(getProperties(), getAnnotationData()); + final ValidationContext validationContext = validationContextFactory.newValidationContext(serviceIdentifiersNotToValidate, getProperties(), getAnnotationData()); final Collection validationResults; try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java index 303f5401ad..c3b6613238 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java @@ -19,8 +19,7 @@ package org.apache.nifi.controller; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Funnel; import org.apache.nifi.connectable.Port; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.OnUnscheduled; +import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.scheduling.SchedulingStrategy; public interface ProcessScheduler { @@ -143,4 +142,28 @@ public interface ProcessScheduler { * @param procNode */ void yield(ProcessorNode procNode); + + /** + * Stops scheduling the given Reporting Task to run + * @param taskNode + */ + void unschedule(ReportingTaskNode taskNode); + + /** + * Begins scheduling the given Reporting Task to run + * @param taskNode + */ + void schedule(ReportingTaskNode taskNode); + + /** + * Enables the Controller Service so that it can be used by Reporting Tasks and Processors + * @param service + */ + void enableControllerService(ControllerServiceNode service); + + /** + * Disables the Controller Service so that it can be updated + * @param service + */ + void disableControllerService(ControllerServiceNode service); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java index f6786fa9a5..3189edd782 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java @@ -21,6 +21,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; import org.apache.nifi.logging.LogLevel; import org.apache.nifi.processor.Processor; @@ -77,4 +78,19 @@ public abstract class ProcessorNode extends AbstractConfiguredComponent implemen public abstract void setStyle(Map style); + /** + * Returns the number of threads (concurrent tasks) currently being used by this Processor + * @return + */ + public abstract int getActiveThreadCount(); + + /** + * Verifies that this Processor can be started if the provided set of + * services are enabled. This is introduced because we need to verify that all components + * can be started before starting any of them. In order to do that, we need to know that this + * component can be started if the given services are enabled, as we will then enable the given + * services before starting this component. + * @param ignoredReferences + */ + public abstract void verifyCanStart(Set ignoredReferences); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java index fa48cb386e..c932f30058 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java @@ -16,18 +16,16 @@ */ package org.apache.nifi.controller; +import java.util.Set; import java.util.concurrent.TimeUnit; +import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.reporting.ReportingContext; import org.apache.nifi.reporting.ReportingTask; import org.apache.nifi.scheduling.SchedulingStrategy; public interface ReportingTaskNode extends ConfiguredComponent { - Availability getAvailability(); - - void setAvailability(Availability availability); - void setSchedulingStrategy(SchedulingStrategy schedulingStrategy); SchedulingStrategy getSchedulingStrategy(); @@ -53,6 +51,12 @@ public interface ReportingTaskNode extends ConfiguredComponent { ConfigurationContext getConfigurationContext(); boolean isRunning(); + + /** + * Returns the number of threads (concurrent tasks) currently being used by this ReportingTask + * @return + */ + int getActiveThreadCount(); /** * Indicates the {@link ScheduledState} of this ReportingTask. A @@ -68,6 +72,20 @@ public interface ReportingTaskNode extends ConfiguredComponent { void setScheduledState(ScheduledState state); + String getComments(); + + void setComments(String comment); + + /** + * Verifies that this Reporting Task can be enabled if the provided set of + * services are enabled. This is introduced because we need to verify that all components + * can be started before starting any of them. In order to do that, we need to know that this + * component can be started if the given services are enabled, as we will then enable the given + * services before starting this component. + * @param ignoredReferences + */ + void verifyCanStart(Set ignoredReferences); + void verifyCanStart(); void verifyCanStop(); void verifyCanDisable(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java index df3c251c85..09479d570b 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java @@ -17,6 +17,7 @@ package org.apache.nifi.controller; import java.util.Map; +import java.util.Set; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; @@ -24,4 +25,7 @@ import org.apache.nifi.components.ValidationContext; public interface ValidationContextFactory { ValidationContext newValidationContext(Map properties, String annotationData); + + ValidationContext newValidationContext(Set serviceIdentifiersToNotValidate, Map properties, String annotationData); + } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorLifeCycleException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ComponentLifeCycleException.java similarity index 84% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorLifeCycleException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ComponentLifeCycleException.java index 5acca16552..9e82b97353 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorLifeCycleException.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ComponentLifeCycleException.java @@ -16,15 +16,15 @@ */ package org.apache.nifi.controller.exception; -public class ProcessorLifeCycleException extends RuntimeException { +public class ComponentLifeCycleException extends RuntimeException { private static final long serialVersionUID = 8392341500511490941L; - public ProcessorLifeCycleException(final String message, final Throwable t) { + public ComponentLifeCycleException(final String message, final Throwable t) { super(message, t); } - public ProcessorLifeCycleException(final Throwable t) { + public ComponentLifeCycleException(final Throwable t) { super(t); } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceNotFoundException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceInstantiationException.java similarity index 77% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceNotFoundException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceInstantiationException.java index 4cdbe5406f..18cfcda659 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceNotFoundException.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceInstantiationException.java @@ -16,28 +16,28 @@ */ package org.apache.nifi.controller.exception; -public class ControllerServiceNotFoundException extends RuntimeException { +public class ControllerServiceInstantiationException extends RuntimeException { private static final long serialVersionUID = -544424320587059277L; /** * Constructs a default exception */ - public ControllerServiceNotFoundException() { + public ControllerServiceInstantiationException() { super(); } /** * @param message */ - public ControllerServiceNotFoundException(String message) { + public ControllerServiceInstantiationException(String message) { super(message); } /** * @param cause */ - public ControllerServiceNotFoundException(Throwable cause) { + public ControllerServiceInstantiationException(Throwable cause) { super(cause); } @@ -45,7 +45,7 @@ public class ControllerServiceNotFoundException extends RuntimeException { * @param message * @param cause */ - public ControllerServiceNotFoundException(String message, Throwable cause) { + public ControllerServiceInstantiationException(String message, Throwable cause) { super(message, cause); } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskProvider.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskProvider.java new file mode 100644 index 0000000000..bb6f3f7086 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskProvider.java @@ -0,0 +1,103 @@ +/* + * 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.reporting; + +import java.util.Set; +import org.apache.nifi.controller.ReportingTaskNode; + +/** + * A ReportingTaskProvider is responsible for providing management of, and access to, Reporting Tasks + */ +public interface ReportingTaskProvider { + + /** + * Creates a new instance of a reporting task + * + * @param type the type (fully qualified class name) of the reporting task to instantiate + * @param id the identifier for the Reporting Task + * @param firstTimeAdded whether or not this is the first time that the reporting task is being added + * to the flow. I.e., this will be true only when the user adds the reporting task to the flow, not when + * the flow is being restored after a restart of the software + * + * @return the ReportingTaskNode that is used to manage the reporting task + * + * @throws ReportingTaskInstantiationException if unable to create the Reporting Task + */ + ReportingTaskNode createReportingTask(String type, String id, boolean firstTimeAdded) throws ReportingTaskInstantiationException; + + /** + * Returns the reporting task that has the given identifier, or null if no reporting task + * exists with that ID. + * + * @param identifier + * @return + */ + ReportingTaskNode getReportingTaskNode(String identifier); + + /** + * Returns a Set of all Reporting Tasks that exist for this service provider. + * @return + */ + Set getAllReportingTasks(); + + /** + * Removes the given reporting task from the flow + * + * @param reportingTask + * + * @throws IllegalStateException if the reporting task cannot be removed because it is not stopped, or + * if the reporting task is not known in the flow + */ + void removeReportingTask(ReportingTaskNode reportingTask); + + /** + * Begins scheduling the reporting task to run and invokes appropriate lifecycle methods + * @param reportingTask + * + * @throws IllegalStateException if the ReportingTask's state is not STOPPED, or if the Reporting Task has active + * threads, or if the ReportingTask is not valid + */ + void startReportingTask(ReportingTaskNode reportingTask); + + /** + * Stops scheduling the reporting task to run and invokes appropriate lifecycle methods + * @param reportingTask + * + * @throws IllegalStateException if the ReportingTask's state is not RUNNING + */ + void stopReportingTask(ReportingTaskNode reportingTask); + + + /** + * Enables the reporting task to be scheduled to run + * @param reportingTask + * + * @throws IllegalStateException if the ReportingTask's state is not DISABLED + */ + void enableReportingTask(ReportingTaskNode reportingTask); + + + /** + * Disables the ability to schedul the reporting task to run + * + * @param reportingTask + * + * @throws IllegalStateException if the ReportingTask's state is not STOPPED, or if the Reporting Task has active + * threads + */ + void disableReportingTask(ReportingTaskNode reportingTask); +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java index 66bad39a4a..50bf469e77 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java @@ -16,7 +16,8 @@ */ package org.apache.nifi.controller.service; -import org.apache.nifi.controller.Availability; +import java.util.Set; + import org.apache.nifi.controller.ConfiguredComponent; import org.apache.nifi.controller.ControllerService; @@ -26,22 +27,42 @@ public interface ControllerServiceNode extends ConfiguredComponent { ControllerService getControllerServiceImplementation(); - Availability getAvailability(); - - void setAvailability(Availability availability); - - boolean isDisabled(); - - void setDisabled(boolean disabled); - + ControllerServiceState getState(); + void setState(ControllerServiceState state); + ControllerServiceReference getReferences(); void addReference(ConfiguredComponent referringComponent); void removeReference(ConfiguredComponent referringComponent); + void setComments(String comment); + String getComments(); + void verifyCanEnable(); void verifyCanDisable(); + + /** + * Verifies that this Controller Service can be disabled if the provided set of + * services are also disabled. This is introduced because we can have an instance + * where A references B, which references C, which references A and we want + * to disable service C. In this case, the cycle needs to not cause us to fail, + * so we want to verify that C can be disabled if A and B also are. + * + * @param ignoredReferences + */ + void verifyCanDisable(Set ignoredReferences); + + /** + * Verifies that this Controller Service can be enabled if the provided set of + * services are also enabled. This is introduced because we can have an instance where + * A reference B, which references C, which references A and we want to enable + * Service A. In this case, the cycle needs to not cause us to fail, so we want to verify + * that A can be enabled if A and B also are. + * @param ignoredReferences + */ + void verifyCanEnable(Set ignoredReferences); + void verifyCanDelete(); void verifyCanUpdate(); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java index 5f2fc2ef85..1901fb6521 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java @@ -16,6 +16,9 @@ */ package org.apache.nifi.controller.service; +import java.util.Collection; +import java.util.Set; + import org.apache.nifi.annotation.lifecycle.OnAdded; import org.apache.nifi.controller.ControllerServiceLookup; @@ -25,7 +28,7 @@ import org.apache.nifi.controller.ControllerServiceLookup; public interface ControllerServiceProvider extends ControllerServiceLookup { /** - * Creates a new Controller Service of the given type and assigns it the given id. If firstTimeadded + * Creates a new Controller Service of the specified type and assigns it the given id. If firstTimeadded * is true, calls any methods that are annotated with {@link OnAdded} * * @param type @@ -60,10 +63,88 @@ public interface ControllerServiceProvider extends ControllerServiceLookup { */ void enableControllerService(ControllerServiceNode serviceNode); + /** + * Enables the collection of services. If a service in this collection depends on another service, + * the service being depended on must either already be enabled or must be in the collection as well. + * @param serviceNodes + */ + void enableControllerServices(Collection serviceNodes); + /** * Disables the given controller service so that it cannot be used by other components. This allows * configuration to be updated or allows service to be removed. * @param serviceNode */ void disableControllerService(ControllerServiceNode serviceNode); + + /** + * Returns a Set of all Controller Services that exist for this service provider. + * @return + */ + Set getAllControllerServices(); + + /** + * Verifies that all running Processors and Reporting Tasks referencing the Controller Service (or a service + * that depends on the provided service) can be stopped. + * @param serviceNode + * + * @throws IllegalStateException if any referencing component cannot be stopped + */ + void verifyCanStopReferencingComponents(ControllerServiceNode serviceNode); + + /** + * Recursively unschedules all schedulable components (Processors and Reporting Tasks) that reference the given + * Controller Service. For any Controller services that reference this one, its schedulable referencing components will also + * be unscheduled. + * @param serviceNode + */ + void unscheduleReferencingComponents(ControllerServiceNode serviceNode); + + /** + * Verifies that all Controller Services referencing the provided Controller Service can be disabled. + * @param serviceNode + * + * @throws IllegalStateException if any referencing service cannot be disabled + */ + void verifyCanDisableReferencingServices(ControllerServiceNode serviceNode); + + /** + * Disables any Controller Service that references the provided Controller Service. This action is performed recursively + * so that if service A references B and B references C, disabling references for C will first disable A, then B. + * @param serviceNode + */ + void disableReferencingServices(ControllerServiceNode serviceNode); + + /** + * Verifies that all Controller Services referencing the provided ControllerService can be enabled. + * @param serviceNode + * + * @throws IllegalStateException if any referencing component cannot be enabled + */ + void verifyCanEnableReferencingServices(ControllerServiceNode serviceNode); + + + /** + * Enables all Controller Services that are referencing the given service. If Service A references Service B and Service + * B references serviceNode, Service A and B will both be enabled. + * @param serviceNode + */ + void enableReferencingServices(ControllerServiceNode serviceNode); + + /** + * Verifies that all enabled Processors referencing the ControllerService (or a service that depends on + * the provided service) can be scheduled to run. + * @param serviceNode + * + * @throws IllegalStateException if any referencing component cannot be scheduled + */ + void verifyCanScheduleReferencingComponents(ControllerServiceNode serviceNode); + + /** + * Schedules any schedulable component (Processor, ReportingTask) that is referencing the given Controller Service + * to run. This is performed recursively, so if a Processor is referencing Service A, which is referencing serviceNode, + * then the Processor will also be started. + * @param serviceNode + */ + void scheduleReferencingComponents(ControllerServiceNode serviceNode); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java index 5cb676f494..67ffb6cb44 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java @@ -41,10 +41,11 @@ public interface ControllerServiceReference { Set getReferencingComponents(); /** - * Returns a {@link Set} of all Processors and Reporting Tasks that are - * referencing the Controller Service and are running, in addition to all + * Returns a {@link Set} of all Processors, Reporting Tasks, and Controller Services that are + * referencing the Controller Service and are running (in the case of Processors and Reporting Tasks) + * or enabled (in the case of Controller Services) * * @return */ - Set getRunningReferences(); + Set getActiveReferences(); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceState.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceState.java new file mode 100644 index 0000000000..2ed8fd9316 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceState.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.controller.service; + + +/** + * Represents the valid states for a Controller Service. + */ +public enum ControllerServiceState { + /** + * Controller Service is disabled and cannot be used. + */ + DISABLED, + + /** + * Controller Service has been disabled but has not yet finished its lifecycle + * methods. + */ + DISABLING, + + /** + * Controller Service has been enabled but has not yet finished its lifecycle methods. + */ + ENABLING, + + /** + * Controller Service has been enabled and has finished its lifecycle methods. The Controller SErvice + * is ready to be used. + */ + ENABLED; +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index 06ef203644..ec25ab1c98 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -52,6 +52,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.nifi.admin.service.UserService; import org.apache.nifi.annotation.lifecycle.OnAdded; import org.apache.nifi.annotation.lifecycle.OnRemoved; +import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.cluster.BulletinsPayload; import org.apache.nifi.cluster.HeartbeatPayload; import org.apache.nifi.cluster.protocol.DataFlow; @@ -62,6 +63,7 @@ import org.apache.nifi.cluster.protocol.NodeProtocolSender; import org.apache.nifi.cluster.protocol.UnknownServiceAddressException; import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; import org.apache.nifi.cluster.protocol.message.NodeBulletinsMessage; +import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; @@ -73,10 +75,12 @@ import org.apache.nifi.connectable.Size; import org.apache.nifi.connectable.StandardConnection; import org.apache.nifi.controller.exception.CommunicationsException; import org.apache.nifi.controller.exception.ProcessorInstantiationException; -import org.apache.nifi.controller.exception.ProcessorLifeCycleException; +import org.apache.nifi.controller.exception.ComponentLifeCycleException; import org.apache.nifi.controller.label.Label; import org.apache.nifi.controller.label.StandardLabel; import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException; +import org.apache.nifi.controller.reporting.ReportingTaskProvider; +import org.apache.nifi.controller.reporting.StandardReportingInitializationContext; import org.apache.nifi.controller.reporting.StandardReportingTaskNode; import org.apache.nifi.controller.repository.ContentRepository; import org.apache.nifi.controller.repository.CounterRepository; @@ -103,6 +107,7 @@ import org.apache.nifi.controller.scheduling.StandardProcessScheduler; import org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent; 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.controller.service.StandardControllerServiceProvider; import org.apache.nifi.controller.status.ConnectionStatus; import org.apache.nifi.controller.status.PortStatus; @@ -129,6 +134,7 @@ import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.groups.RemoteProcessGroup; import org.apache.nifi.groups.RemoteProcessGroupPortDescriptor; import org.apache.nifi.groups.StandardProcessGroup; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.logging.LogLevel; import org.apache.nifi.logging.LogRepository; import org.apache.nifi.logging.LogRepositoryFactory; @@ -161,6 +167,8 @@ import org.apache.nifi.remote.protocol.socket.SocketFlowFileServerProtocol; import org.apache.nifi.reporting.Bulletin; import org.apache.nifi.reporting.BulletinRepository; import org.apache.nifi.reporting.EventAccess; +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.reporting.ReportingInitializationContext; import org.apache.nifi.reporting.ReportingTask; import org.apache.nifi.reporting.Severity; import org.apache.nifi.scheduling.SchedulingStrategy; @@ -170,6 +178,7 @@ import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.ReflectionUtils; import org.apache.nifi.web.api.dto.ConnectableDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.FlowSnippetDTO; import org.apache.nifi.web.api.dto.FunnelDTO; import org.apache.nifi.web.api.dto.LabelDTO; @@ -189,7 +198,7 @@ import org.slf4j.LoggerFactory; import com.sun.jersey.api.client.ClientHandlerException; -public class FlowController implements EventAccess, ControllerServiceProvider, Heartbeater, QueueProvider { +public class FlowController implements EventAccess, ControllerServiceProvider, ReportingTaskProvider, Heartbeater, QueueProvider { // default repository implementations public static final String DEFAULT_FLOWFILE_REPO_IMPLEMENTATION = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository"; @@ -374,7 +383,6 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H this.properties = properties; sslContext = SslContextFactory.createSslContext(properties, false); extensionManager = new ExtensionManager(); - controllerServiceProvider = new StandardControllerServiceProvider(); timerDrivenEngineRef = new AtomicReference<>(new FlowEngine(maxTimerDrivenThreads.get(), "Timer-Driven Process")); eventDrivenEngineRef = new AtomicReference<>(new FlowEngine(maxEventDrivenThreads.get(), "Event-Driven Process")); @@ -398,6 +406,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H processScheduler = new StandardProcessScheduler(this, this, encryptor); eventDrivenWorkerQueue = new EventDrivenWorkerQueue(false, false, processScheduler); + controllerServiceProvider = new StandardControllerServiceProvider(processScheduler, bulletinRepository); final ProcessContextFactory contextFactory = new ProcessContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceEventRepository); processScheduler.setSchedulingAgent(SchedulingStrategy.EVENT_DRIVEN, new EventDrivenSchedulingAgent( @@ -593,7 +602,10 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H startConnectable(connectable); } } catch (final Throwable t) { - LOG.error("Unable to start {} due to {}", new Object[]{connectable, t}); + LOG.error("Unable to start {} due to {}", new Object[]{connectable, t.toString()}); + if ( LOG.isDebugEnabled() ) { + LOG.error("", t); + } } } @@ -818,7 +830,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, org.apache.nifi.processor.annotation.OnAdded.class, processor); } catch (final Exception e) { logRepository.removeObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID); - throw new ProcessorLifeCycleException("Failed to invoke @OnAdded methods of " + procNode.getProcessor(), e); + throw new ComponentLifeCycleException("Failed to invoke @OnAdded methods of " + procNode.getProcessor(), e); } } @@ -1063,7 +1075,23 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H // Trigger any processors' methods marked with @OnShutdown to be called rootGroup.shutdown(); - + + // invoke any methods annotated with @OnShutdown on Controller Services + for ( final ControllerServiceNode serviceNode : getAllControllerServices() ) { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + final ConfigurationContext configContext = new StandardConfigurationContext(serviceNode, controllerServiceProvider); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, serviceNode.getControllerServiceImplementation(), configContext); + } + } + + // invoke any methods annotated with @OnShutdown on Reporting Tasks + for ( final ReportingTaskNode taskNode : getAllReportingTasks() ) { + final ConfigurationContext configContext = taskNode.getConfigurationContext(); + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, taskNode.getReportingTask(), configContext); + } + } + try { this.timerDrivenEngineRef.get().awaitTermination(gracefulShutdownSeconds / 2, TimeUnit.SECONDS); this.eventDrivenEngineRef.get().awaitTermination(gracefulShutdownSeconds / 2, TimeUnit.SECONDS); @@ -1401,6 +1429,30 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H try { validateSnippetContents(requireNonNull(group), dto); + // + // Instantiate Controller Services + // + for ( final ControllerServiceDTO controllerServiceDTO : dto.getControllerServices() ) { + final ControllerServiceNode serviceNode = createControllerService(controllerServiceDTO.getType(), controllerServiceDTO.getId(), true); + + serviceNode.setAnnotationData(controllerServiceDTO.getAnnotationData()); + serviceNode.setComments(controllerServiceDTO.getComments()); + serviceNode.setName(controllerServiceDTO.getName()); + } + + // configure controller services. We do this after creating all of them in case 1 service + // references another service. + for ( final ControllerServiceDTO controllerServiceDTO : dto.getControllerServices() ) { + final String serviceId = controllerServiceDTO.getId(); + final ControllerServiceNode serviceNode = getControllerServiceNode(serviceId); + + for ( final Map.Entry entry : controllerServiceDTO.getProperties().entrySet() ) { + if ( entry.getValue() != null ) { + serviceNode.setProperty(entry.getKey(), entry.getValue()); + } + } + } + // // Instantiate the labels // @@ -1411,7 +1463,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H label.setSize(new Size(labelDTO.getWidth(), labelDTO.getHeight())); } - // TODO: Update the label's "style" + label.setStyle(labelDTO.getStyle()); group.addLabel(label); } @@ -1737,14 +1789,18 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H } // validate that all Processor Types and Prioritizer Types are valid - final List processorClasses = new ArrayList<>(); + final Set processorClasses = new HashSet<>(); for (final Class c : ExtensionManager.getExtensions(Processor.class)) { processorClasses.add(c.getName()); } - final List prioritizerClasses = new ArrayList<>(); + final Set prioritizerClasses = new HashSet<>(); for (final Class c : ExtensionManager.getExtensions(FlowFilePrioritizer.class)) { prioritizerClasses.add(c.getName()); } + final Set controllerServiceClasses = new HashSet<>(); + for (final Class c : ExtensionManager.getExtensions(ControllerService.class)) { + controllerServiceClasses.add(c.getName()); + } final Set allProcs = new HashSet<>(); final Set allConns = new HashSet<>(); @@ -1760,6 +1816,15 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H throw new IllegalStateException("Invalid Processor Type: " + proc.getType()); } } + + final Set controllerServices = templateContents.getControllerServices(); + if (controllerServices != null) { + for (final ControllerServiceDTO service : controllerServices) { + if (!controllerServiceClasses.contains(service.getType())) { + throw new IllegalStateException("Invalid Controller Service Type: " + service.getType()); + } + } + } for (final ConnectionDTO conn : allConns) { final List prioritizers = conn.getPrioritizers(); @@ -2480,17 +2545,20 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H lookupGroup(groupId).stopProcessing(); } - public ReportingTaskNode createReportingTask(final String type, String id) throws ReportingTaskInstantiationException { - return createReportingTask(type, id, true); + public ReportingTaskNode createReportingTask(final String type) throws ReportingTaskInstantiationException { + return createReportingTask(type, true); } - public ReportingTaskNode createReportingTask(final String type, String id, final boolean firstTimeAdded) throws ReportingTaskInstantiationException { - if (type == null) { + public ReportingTaskNode createReportingTask(final String type, final boolean firstTimeAdded) throws ReportingTaskInstantiationException { + return createReportingTask(type, UUID.randomUUID().toString(), firstTimeAdded); + } + + @Override + public ReportingTaskNode createReportingTask(final String type, final String id, final boolean firstTimeAdded) throws ReportingTaskInstantiationException { + if (type == null || id == null) { throw new NullPointerException(); } - - id = requireNonNull(id).intern(); - + ReportingTask task = null; final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader(); try { @@ -2516,12 +2584,23 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider); final ReportingTaskNode taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory); + taskNode.setName(task.getClass().getSimpleName()); if ( firstTimeAdded ) { + final ComponentLog componentLog = new SimpleProcessLogger(id, taskNode.getReportingTask()); + final ReportingInitializationContext config = new StandardReportingInitializationContext(id, taskNode.getName(), + SchedulingStrategy.TIMER_DRIVEN, "1 min", componentLog, this); + + try { + task.initialize(config); + } catch (final InitializationException ie) { + throw new ReportingTaskInstantiationException("Failed to initialize reporting task of type " + type, ie); + } + try (final NarCloseable x = NarCloseable.withNarLoader()) { ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, task); } catch (final Exception e) { - throw new ProcessorLifeCycleException("Failed to invoke On-Added Lifecycle methods of " + task, e); + throw new ComponentLifeCycleException("Failed to invoke On-Added Lifecycle methods of " + task, e); } } @@ -2529,30 +2608,33 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H return taskNode; } + @Override public ReportingTaskNode getReportingTaskNode(final String taskId) { return reportingTasks.get(taskId); } + @Override public void startReportingTask(final ReportingTaskNode reportingTaskNode) { if (isTerminated()) { throw new IllegalStateException("Cannot start reporting task " + reportingTaskNode + " because the controller is terminated"); } reportingTaskNode.verifyCanStart(); - - processScheduler.schedule(reportingTaskNode); + processScheduler.schedule(reportingTaskNode); } + + @Override public void stopReportingTask(final ReportingTaskNode reportingTaskNode) { if (isTerminated()) { return; } reportingTaskNode.verifyCanStop(); - processScheduler.unschedule(reportingTaskNode); } + @Override public void removeReportingTask(final ReportingTaskNode reportingTaskNode) { final ReportingTaskNode existing = reportingTasks.get(reportingTaskNode.getIdentifier()); if ( existing == null || existing != reportingTaskNode ) { @@ -2565,43 +2647,100 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, reportingTaskNode.getReportingTask(), reportingTaskNode.getConfigurationContext()); } + for ( final Map.Entry entry : reportingTaskNode.getProperties().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(reportingTaskNode); + } + } + } + } + reportingTasks.remove(reportingTaskNode.getIdentifier()); } - Collection getReportingTasks() { - return reportingTasks.values(); + @Override + public Set getAllReportingTasks() { + return new HashSet<>(reportingTasks.values()); } - + @Override + public ControllerServiceNode createControllerService(final String type, final String id, final boolean firstTimeAdded) { + return controllerServiceProvider.createControllerService(type, id, firstTimeAdded); + } + + @Override public void enableReportingTask(final ReportingTaskNode reportingTaskNode) { reportingTaskNode.verifyCanEnable(); - processScheduler.enableReportingTask(reportingTaskNode); } + @Override public void disableReportingTask(final ReportingTaskNode reportingTaskNode) { reportingTaskNode.verifyCanDisable(); - processScheduler.disableReportingTask(reportingTaskNode); } + @Override + public void disableReferencingServices(final ControllerServiceNode serviceNode) { + controllerServiceProvider.disableReferencingServices(serviceNode); + } + + @Override + public void enableReferencingServices(final ControllerServiceNode serviceNode) { + controllerServiceProvider.enableReferencingServices(serviceNode); + } + + @Override + public void scheduleReferencingComponents(final ControllerServiceNode serviceNode) { + controllerServiceProvider.scheduleReferencingComponents(serviceNode); + } + + @Override + public void unscheduleReferencingComponents(final ControllerServiceNode serviceNode) { + controllerServiceProvider.unscheduleReferencingComponents(serviceNode); + } + @Override public void enableControllerService(final ControllerServiceNode serviceNode) { - serviceNode.verifyCanEnable(); controllerServiceProvider.enableControllerService(serviceNode); } + @Override + public void enableControllerServices(final Collection serviceNodes) { + controllerServiceProvider.enableControllerServices(serviceNodes); + } + @Override public void disableControllerService(final ControllerServiceNode serviceNode) { serviceNode.verifyCanDisable(); controllerServiceProvider.disableControllerService(serviceNode); } - + @Override - public ControllerServiceNode createControllerService(final String type, final String id, final boolean firstTimeAdded) { - return controllerServiceProvider.createControllerService(type, id.intern(), firstTimeAdded); + public void verifyCanEnableReferencingServices(final ControllerServiceNode serviceNode) { + controllerServiceProvider.verifyCanEnableReferencingServices(serviceNode); + } + + @Override + public void verifyCanScheduleReferencingComponents(final ControllerServiceNode serviceNode) { + controllerServiceProvider.verifyCanScheduleReferencingComponents(serviceNode); } + @Override + public void verifyCanDisableReferencingServices(final ControllerServiceNode serviceNode) { + controllerServiceProvider.verifyCanDisableReferencingServices(serviceNode); + } + + @Override + public void verifyCanStopReferencingComponents(final ControllerServiceNode serviceNode) { + controllerServiceProvider.verifyCanStopReferencingComponents(serviceNode); + } + @Override public ControllerService getControllerService(final String serviceIdentifier) { return controllerServiceProvider.getControllerService(serviceIdentifier); @@ -2623,10 +2762,24 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H } @Override + public boolean isControllerServiceEnabling(final String serviceIdentifier) { + return controllerServiceProvider.isControllerServiceEnabling(serviceIdentifier); + } + + @Override + public String getControllerServiceName(final String serviceIdentifier) { + return controllerServiceProvider.getControllerServiceName(serviceIdentifier); + } + public void removeControllerService(final ControllerServiceNode serviceNode) { controllerServiceProvider.removeControllerService(serviceNode); } + @Override + public Set getAllControllerServices() { + return controllerServiceProvider.getAllControllerServices(); + } + // // Counters // diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowFromDOMFactory.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowFromDOMFactory.java index c67181a063..85ad1590ee 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowFromDOMFactory.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowFromDOMFactory.java @@ -26,6 +26,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import org.apache.nifi.connectable.Size; +import org.apache.nifi.controller.service.ControllerServiceState; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.groups.RemoteProcessGroupPortDescriptor; import org.apache.nifi.remote.StandardRemoteProcessGroupPortDescriptor; @@ -33,6 +34,7 @@ import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.util.DomUtils; import org.apache.nifi.web.api.dto.ConnectableDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.FlowSnippetDTO; import org.apache.nifi.web.api.dto.FunnelDTO; import org.apache.nifi.web.api.dto.LabelDTO; @@ -42,7 +44,7 @@ import org.apache.nifi.web.api.dto.ProcessGroupDTO; import org.apache.nifi.web.api.dto.ProcessorConfigDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; - +import org.apache.nifi.web.api.dto.ReportingTaskDTO; import org.w3c.dom.Element; import org.w3c.dom.NodeList; @@ -77,6 +79,40 @@ public class FlowFromDOMFactory { return styles; } + + public static ControllerServiceDTO getControllerService(final Element element, final StringEncryptor encryptor) { + final ControllerServiceDTO dto = new ControllerServiceDTO(); + + dto.setId(getString(element, "id")); + dto.setName(getString(element, "name")); + dto.setComments(getString(element, "comment")); + dto.setType(getString(element, "class")); + + final boolean enabled = getBoolean(element, "enabled"); + dto.setState(enabled ? ControllerServiceState.ENABLED.name() : ControllerServiceState.DISABLED.name()); + + dto.setProperties(getProperties(element, encryptor)); + dto.setAnnotationData(getString(element, "annotationData")); + + return dto; + } + + public static ReportingTaskDTO getReportingTask(final Element element, final StringEncryptor encryptor) { + final ReportingTaskDTO dto = new ReportingTaskDTO(); + + dto.setId(getString(element, "id")); + dto.setName(getString(element, "name")); + dto.setComments(getString(element, "comment")); + dto.setType(getString(element, "class")); + dto.setSchedulingPeriod(getString(element, "schedulingPeriod")); + dto.setState(getString(element, "scheduledState")); + dto.setSchedulingStrategy(getString(element, "schedulingStrategy")); + + dto.setProperties(getProperties(element, encryptor)); + dto.setAnnotationData(getString(element, "annotationData")); + + return dto; + } public static ProcessGroupDTO getProcessGroup(final String parentId, final Element element, final StringEncryptor encryptor) { final ProcessGroupDTO dto = new ProcessGroupDTO(); @@ -310,7 +346,6 @@ public class FlowFromDOMFactory { final ProcessorConfigDTO configDto = new ProcessorConfigDTO(); dto.setConfig(configDto); configDto.setComments(getString(element, "comment")); - configDto.setAnnotationData(getString(element, "annotationData")); configDto.setConcurrentlySchedulableTaskCount(getInt(element, "maxConcurrentTasks")); final String schedulingPeriod = getString(element, "schedulingPeriod"); configDto.setSchedulingPeriod(schedulingPeriod); @@ -334,14 +369,8 @@ public class FlowFromDOMFactory { configDto.setRunDurationMillis(TimeUnit.NANOSECONDS.toMillis(runDurationNanos)); } - final LinkedHashMap properties = new LinkedHashMap<>(); - final List propertyNodeList = getChildrenByTagName(element, "property"); - for (final Element propertyElement : propertyNodeList) { - final String name = getString(propertyElement, "name"); - final String value = decrypt(getString(propertyElement, "value"), encryptor); - properties.put(name, value); - } - configDto.setProperties(properties); + configDto.setProperties(getProperties(element, encryptor)); + configDto.setAnnotationData(getString(element, "annotationData")); final Set autoTerminatedRelationships = new HashSet<>(); final List autoTerminateList = getChildrenByTagName(element, "autoTerminatedRelationship"); @@ -353,6 +382,17 @@ public class FlowFromDOMFactory { return dto; } + private static LinkedHashMap getProperties(final Element element, final StringEncryptor encryptor) { + final LinkedHashMap properties = new LinkedHashMap<>(); + final List propertyNodeList = getChildrenByTagName(element, "property"); + for (final Element propertyElement : propertyNodeList) { + final String name = getString(propertyElement, "name"); + final String value = decrypt(getString(propertyElement, "value"), encryptor); + properties.put(name, value); + } + return properties; + } + private static String getString(final Element element, final String childElementName) { final List nodeList = getChildrenByTagName(element, childElementName); if (nodeList == null || nodeList.isEmpty()) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSerializer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSerializer.java index e08a94dffb..7cd9d3bb06 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSerializer.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSerializer.java @@ -40,6 +40,8 @@ import org.apache.nifi.connectable.Port; import org.apache.nifi.connectable.Position; import org.apache.nifi.connectable.Size; import org.apache.nifi.controller.label.Label; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceState; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.flowfile.FlowFilePrioritizer; import org.apache.nifi.groups.ProcessGroup; @@ -47,7 +49,6 @@ import org.apache.nifi.groups.RemoteProcessGroup; import org.apache.nifi.processor.Relationship; import org.apache.nifi.remote.RemoteGroupPort; import org.apache.nifi.remote.RootGroupPort; - import org.w3c.dom.DOMException; import org.w3c.dom.Document; import org.w3c.dom.Element; @@ -79,6 +80,18 @@ public class StandardFlowSerializer implements FlowSerializer { addTextElement(rootNode, "maxTimerDrivenThreadCount", controller.getMaxTimerDrivenThreadCount()); addTextElement(rootNode, "maxEventDrivenThreadCount", controller.getMaxEventDrivenThreadCount()); addProcessGroup(rootNode, controller.getGroup(controller.getRootGroupId()), "rootGroup"); + + final Element controllerServicesNode = doc.createElement("controllerServices"); + rootNode.appendChild(controllerServicesNode); + for ( final ControllerServiceNode serviceNode : controller.getAllControllerServices() ) { + addControllerService(controllerServicesNode, serviceNode, encryptor); + } + + final Element reportingTasksNode = doc.createElement("reportingTasks"); + rootNode.appendChild(reportingTasksNode); + for ( final ReportingTaskNode taskNode : controller.getAllReportingTasks() ) { + addReportingTask(reportingTasksNode, taskNode, encryptor); + } final DOMSource domSource = new DOMSource(doc); final StreamResult streamResult = new StreamResult(new BufferedOutputStream(os)); @@ -300,8 +313,16 @@ public class StandardFlowSerializer implements FlowSerializer { addTextElement(element, "schedulingStrategy", processor.getSchedulingStrategy().name()); addTextElement(element, "runDurationNanos", processor.getRunDuration(TimeUnit.NANOSECONDS)); - // properties. - for (final Map.Entry entry : processor.getProperties().entrySet()) { + addConfiguration(element, processor.getProperties(), processor.getAnnotationData(), encryptor); + + for (final Relationship rel : processor.getAutoTerminatedRelationships()) { + addTextElement(element, "autoTerminatedRelationship", rel.getName()); + } + } + + private static void addConfiguration(final Element element, final Map properties, final String annotationData, final StringEncryptor encryptor) { + final Document doc = element.getOwnerDocument(); + for (final Map.Entry entry : properties.entrySet()) { final PropertyDescriptor descriptor = entry.getKey(); String value = entry.getValue(); @@ -322,14 +343,9 @@ public class StandardFlowSerializer implements FlowSerializer { element.appendChild(propElement); } - final String annotationData = processor.getAnnotationData(); if (annotationData != null) { addTextElement(element, "annotationData", annotationData); } - - for (final Relationship rel : processor.getAutoTerminatedRelationships()) { - addTextElement(element, "autoTerminatedRelationship", rel.getName()); - } } private void addConnection(final Element parentElement, final Connection connection) { @@ -390,11 +406,43 @@ public class StandardFlowSerializer implements FlowSerializer { parentElement.appendChild(element); } - private void addTextElement(final Element element, final String name, final long value) { + + public static void addControllerService(final Element element, final ControllerServiceNode serviceNode, final StringEncryptor encryptor) { + final Element serviceElement = element.getOwnerDocument().createElement("controllerService"); + addTextElement(serviceElement, "id", serviceNode.getIdentifier()); + addTextElement(serviceElement, "name", serviceNode.getName()); + addTextElement(serviceElement, "comment", serviceNode.getComments()); + addTextElement(serviceElement, "class", serviceNode.getControllerServiceImplementation().getClass().getCanonicalName()); + + final ControllerServiceState state = serviceNode.getState(); + final boolean enabled = (state == ControllerServiceState.ENABLED || state == ControllerServiceState.ENABLING); + addTextElement(serviceElement, "enabled", String.valueOf(enabled)); + + addConfiguration(serviceElement, serviceNode.getProperties(), serviceNode.getAnnotationData(), encryptor); + + element.appendChild(serviceElement); + } + + public static void addReportingTask(final Element element, final ReportingTaskNode taskNode, final StringEncryptor encryptor) { + final Element taskElement = element.getOwnerDocument().createElement("reportingTask"); + addTextElement(taskElement, "id", taskNode.getIdentifier()); + addTextElement(taskElement, "name", taskNode.getName()); + addTextElement(taskElement, "comment", taskNode.getComments()); + addTextElement(taskElement, "class", taskNode.getReportingTask().getClass().getCanonicalName()); + addTextElement(taskElement, "schedulingPeriod", taskNode.getSchedulingPeriod()); + addTextElement(taskElement, "scheduledState", taskNode.getScheduledState().name()); + addTextElement(taskElement, "schedulingStrategy", taskNode.getSchedulingStrategy().name()); + + addConfiguration(taskElement, taskNode.getProperties(), taskNode.getAnnotationData(), encryptor); + + element.appendChild(taskElement); + } + + private static void addTextElement(final Element element, final String name, final long value) { addTextElement(element, name, String.valueOf(value)); } - private void addTextElement(final Element element, final String name, final String value) { + private static void addTextElement(final Element element, final String name, final String value) { final Document doc = element.getOwnerDocument(); final Element toAdd = doc.createElement(name); toAdd.setTextContent(value); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java index 64ce5c4142..fcfee83e7c 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java @@ -81,8 +81,6 @@ public class StandardFlowService implements FlowService, ProtocolHandler { private final FlowController controller; private final Path flowXml; - private final Path taskConfigXml; - private final Path serviceConfigXml; private final FlowConfigurationDAO dao; private final int gracefulShutdownSeconds; private final boolean autoResumeState; @@ -154,14 +152,12 @@ public class StandardFlowService implements FlowService, ProtocolHandler { this.controller = controller; this.encryptor = encryptor; flowXml = Paths.get(properties.getProperty(NiFiProperties.FLOW_CONFIGURATION_FILE)); - taskConfigXml = Paths.get(properties.getProperty(NiFiProperties.TASK_CONFIGURATION_FILE)); - serviceConfigXml = Paths.get(properties.getProperty(NiFiProperties.SERVICE_CONFIGURATION_FILE)); gracefulShutdownSeconds = (int) FormatUtils.getTimeDuration(properties.getProperty(NiFiProperties.FLOW_CONTROLLER_GRACEFUL_SHUTDOWN_PERIOD), TimeUnit.SECONDS); autoResumeState = properties.getAutoResumeState(); connectionRetryMillis = (int) FormatUtils.getTimeDuration(properties.getClusterManagerFlowRetrievalDelay(), TimeUnit.MILLISECONDS); - dao = new StandardXMLFlowConfigurationDAO(flowXml, taskConfigXml, serviceConfigXml, encryptor); + dao = new StandardXMLFlowConfigurationDAO(flowXml, encryptor); if (configuredForClustering) { @@ -605,7 +601,6 @@ public class StandardFlowService implements FlowService, ProtocolHandler { if (firstControllerInitialization) { // load the controller services logger.debug("Loading controller services"); - dao.loadControllerServices(controller); } // load the flow @@ -622,7 +617,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler { logger.debug("First controller initialization. Loading reporting tasks and initializing controller."); // load the controller tasks - dao.loadReportingTasks(controller); +// dao.loadReportingTasks(controller); // initialize the flow controller.initializeFlow(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java index 05a8f019d8..201482c015 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java @@ -26,6 +26,7 @@ import java.nio.file.Path; import java.nio.file.StandardOpenOption; import java.security.NoSuchAlgorithmException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -40,6 +41,7 @@ import javax.xml.parsers.ParserConfigurationException; import javax.xml.validation.Schema; import javax.xml.validation.SchemaFactory; +import org.apache.commons.collections4.CollectionUtils; import org.apache.nifi.cluster.protocol.DataFlow; import org.apache.nifi.cluster.protocol.StandardDataFlow; import org.apache.nifi.connectable.Connectable; @@ -51,23 +53,35 @@ import org.apache.nifi.connectable.Position; import org.apache.nifi.connectable.Size; import org.apache.nifi.controller.exception.ProcessorInstantiationException; import org.apache.nifi.controller.label.Label; +import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException; +import org.apache.nifi.controller.reporting.StandardReportingInitializationContext; +import org.apache.nifi.controller.service.ControllerServiceLoader; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceState; +import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.events.BulletinFactory; -import org.apache.nifi.util.file.FileUtils; import org.apache.nifi.fingerprint.FingerprintException; import org.apache.nifi.fingerprint.FingerprintFactory; import org.apache.nifi.flowfile.FlowFilePrioritizer; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.groups.RemoteProcessGroup; import org.apache.nifi.groups.RemoteProcessGroupPortDescriptor; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.logging.LogLevel; import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.SimpleProcessLogger; import org.apache.nifi.remote.RemoteGroupPort; import org.apache.nifi.remote.RootGroupPort; +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.reporting.ReportingInitializationContext; import org.apache.nifi.reporting.Severity; import org.apache.nifi.scheduling.SchedulingStrategy; +import org.apache.nifi.util.DomUtils; import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.util.file.FileUtils; import org.apache.nifi.web.api.dto.ConnectableDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.FlowSnippetDTO; import org.apache.nifi.web.api.dto.FunnelDTO; import org.apache.nifi.web.api.dto.LabelDTO; @@ -77,9 +91,7 @@ import org.apache.nifi.web.api.dto.ProcessGroupDTO; import org.apache.nifi.web.api.dto.ProcessorConfigDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; - -import org.apache.commons.collections4.CollectionUtils; -import org.apache.nifi.encrypt.StringEncryptor; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.w3c.dom.Document; @@ -96,9 +108,11 @@ public class StandardFlowSynchronizer implements FlowSynchronizer { private static final Logger logger = LoggerFactory.getLogger(StandardFlowSynchronizer.class); public static final URL FLOW_XSD_RESOURCE = StandardFlowSynchronizer.class.getResource("/FlowConfiguration.xsd"); private final StringEncryptor encryptor; + private final boolean autoResumeState; public StandardFlowSynchronizer(final StringEncryptor encryptor) { this.encryptor = encryptor; + autoResumeState = NiFiProperties.getInstance().getAutoResumeState(); } public static boolean isEmpty(final DataFlow dataFlow, final StringEncryptor encryptor) { @@ -157,10 +171,26 @@ public class StandardFlowSynchronizer implements FlowSynchronizer { controller.setMaxEventDrivenThreadCount(maxThreadCount / 3); } + final Element reportingTasksElement = (Element) DomUtils.getChild(rootElement, "reportingTasks"); + final List taskElements; + if ( reportingTasksElement == null ) { + taskElements = Collections.emptyList(); + } else { + taskElements = DomUtils.getChildElementsByTagName(reportingTasksElement, "reportingTask"); + } + + final Element controllerServicesElement = (Element) DomUtils.getChild(rootElement, "controllerServices"); + final List controllerServiceElements; + if ( controllerServicesElement == null ) { + controllerServiceElements = Collections.emptyList(); + } else { + controllerServiceElements = DomUtils.getChildElementsByTagName(controllerServicesElement, "controllerService"); + } + logger.trace("Parsing process group from DOM"); final Element rootGroupElement = (Element) rootElement.getElementsByTagName("rootGroup").item(0); final ProcessGroupDTO rootGroupDto = FlowFromDOMFactory.getProcessGroup(null, rootGroupElement, encryptor); - existingFlowEmpty = isEmpty(rootGroupDto); + existingFlowEmpty = taskElements.isEmpty() && controllerServiceElements.isEmpty() && isEmpty(rootGroupDto); logger.debug("Existing Flow Empty = {}", existingFlowEmpty); } } @@ -200,37 +230,64 @@ public class StandardFlowSynchronizer implements FlowSynchronizer { // create document by parsing proposed flow bytes logger.trace("Parsing proposed flow bytes as DOM document"); final Document configuration = parseFlowBytes(proposedFlow.getFlow()); - + // attempt to sync controller with proposed flow try { if (configuration != null) { - // get the root element - final Element rootElement = (Element) configuration.getElementsByTagName("flowController").item(0); - - // set controller config - logger.trace("Updating flow config"); - final Integer maxThreadCount = getInteger(rootElement, "maxThreadCount"); - if (maxThreadCount == null) { - controller.setMaxTimerDrivenThreadCount(getInt(rootElement, "maxTimerDrivenThreadCount")); - controller.setMaxEventDrivenThreadCount(getInt(rootElement, "maxEventDrivenThreadCount")); - } else { - controller.setMaxTimerDrivenThreadCount(maxThreadCount * 2 / 3); - controller.setMaxEventDrivenThreadCount(maxThreadCount / 3); - } - - // get the root group XML element - final Element rootGroupElement = (Element) rootElement.getElementsByTagName("rootGroup").item(0); - - // if this controller isn't initialized or its emtpy, add the root group, otherwise update - if (!initialized || existingFlowEmpty) { - logger.trace("Adding root process group"); - addProcessGroup(controller, /* parent group */ null, rootGroupElement, encryptor); - } else { - logger.trace("Updating root process group"); - updateProcessGroup(controller, /* parent group */ null, rootGroupElement, encryptor); + synchronized (configuration) { + // get the root element + final Element rootElement = (Element) configuration.getElementsByTagName("flowController").item(0); + + // set controller config + logger.trace("Updating flow config"); + final Integer maxThreadCount = getInteger(rootElement, "maxThreadCount"); + if (maxThreadCount == null) { + controller.setMaxTimerDrivenThreadCount(getInt(rootElement, "maxTimerDrivenThreadCount")); + controller.setMaxEventDrivenThreadCount(getInt(rootElement, "maxEventDrivenThreadCount")); + } else { + controller.setMaxTimerDrivenThreadCount(maxThreadCount * 2 / 3); + controller.setMaxEventDrivenThreadCount(maxThreadCount / 3); + } + + // get the root group XML element + final Element rootGroupElement = (Element) rootElement.getElementsByTagName("rootGroup").item(0); + + final Element controllerServicesElement = (Element) DomUtils.getChild(rootElement, "controllerServices"); + if ( controllerServicesElement != null ) { + final List serviceElements = DomUtils.getChildElementsByTagName(controllerServicesElement, "controllerService"); + + if ( !initialized || existingFlowEmpty ) { + ControllerServiceLoader.loadControllerServices(serviceElements, controller, encryptor, controller.getBulletinRepository(), autoResumeState); + } else { + for ( final Element serviceElement : serviceElements ) { + updateControllerService(controller, serviceElement, encryptor); + } + } + } + + // if this controller isn't initialized or its emtpy, add the root group, otherwise update + if (!initialized || existingFlowEmpty) { + logger.trace("Adding root process group"); + addProcessGroup(controller, /* parent group */ null, rootGroupElement, encryptor); + } else { + logger.trace("Updating root process group"); + updateProcessGroup(controller, /* parent group */ null, rootGroupElement, encryptor); + } + + final Element reportingTasksElement = (Element) DomUtils.getChild(rootElement, "reportingTasks"); + if ( reportingTasksElement != null ) { + final List taskElements = DomUtils.getChildElementsByTagName(reportingTasksElement, "reportingTask"); + for ( final Element taskElement : taskElements ) { + if ( !initialized || existingFlowEmpty ) { + addReportingTask(controller, taskElement, encryptor); + } else { + updateReportingTask(controller, taskElement, encryptor); + } + } + } } } - + logger.trace("Synching templates"); if ((existingTemplates == null || existingTemplates.length == 0) && proposedFlow.getTemplates() != null && proposedFlow.getTemplates().length > 0) { // need to load templates @@ -313,7 +370,124 @@ public class StandardFlowSynchronizer implements FlowSynchronizer { return baos.toByteArray(); } + + + private void updateControllerService(final FlowController controller, final Element controllerServiceElement, final StringEncryptor encryptor) { + final ControllerServiceDTO dto = FlowFromDOMFactory.getControllerService(controllerServiceElement, encryptor); + + final ControllerServiceState dtoState = ControllerServiceState.valueOf(dto.getState()); + final boolean dtoEnabled = (dtoState == ControllerServiceState.ENABLED || dtoState == ControllerServiceState.ENABLING); + + final ControllerServiceNode serviceNode = controller.getControllerServiceNode(dto.getId()); + final ControllerServiceState serviceState = serviceNode.getState(); + final boolean serviceEnabled = (serviceState == ControllerServiceState.ENABLED || serviceState == ControllerServiceState.ENABLING); + + if (dtoEnabled && !serviceEnabled) { + controller.enableControllerService(controller.getControllerServiceNode(dto.getId())); + } else if (!dtoEnabled && serviceEnabled) { + controller.disableControllerService(controller.getControllerServiceNode(dto.getId())); + } + } + + private void addReportingTask(final FlowController controller, final Element reportingTaskElement, final StringEncryptor encryptor) throws ReportingTaskInstantiationException { + final ReportingTaskDTO dto = FlowFromDOMFactory.getReportingTask(reportingTaskElement, encryptor); + + final ReportingTaskNode reportingTask = controller.createReportingTask(dto.getType(), dto.getId(), false); + reportingTask.setName(dto.getName()); + reportingTask.setComments(dto.getComments()); + reportingTask.setScheduldingPeriod(dto.getSchedulingPeriod()); + reportingTask.setSchedulingStrategy(SchedulingStrategy.valueOf(dto.getSchedulingStrategy())); + + reportingTask.setAnnotationData(dto.getAnnotationData()); + + for (final Map.Entry entry : dto.getProperties().entrySet()) { + if (entry.getValue() == null) { + reportingTask.removeProperty(entry.getKey()); + } else { + reportingTask.setProperty(entry.getKey(), entry.getValue()); + } + } + + final ComponentLog componentLog = new SimpleProcessLogger(dto.getId(), reportingTask.getReportingTask()); + final ReportingInitializationContext config = new StandardReportingInitializationContext(dto.getId(), dto.getName(), + SchedulingStrategy.valueOf(dto.getSchedulingStrategy()), dto.getSchedulingPeriod(), componentLog, controller); + + try { + reportingTask.getReportingTask().initialize(config); + } catch (final InitializationException ie) { + throw new ReportingTaskInstantiationException("Failed to initialize reporting task of type " + dto.getType(), ie); + } + + if ( autoResumeState ) { + if ( ScheduledState.RUNNING.name().equals(dto.getState()) ) { + try { + controller.startReportingTask(reportingTask); + } catch (final Exception e) { + logger.error("Failed to start {} due to {}", reportingTask, e); + if ( logger.isDebugEnabled() ) { + logger.error("", e); + } + controller.getBulletinRepository().addBulletin(BulletinFactory.createBulletin( + "Reporting Tasks", Severity.ERROR.name(), "Failed to start " + reportingTask + " due to " + e)); + } + } else if ( ScheduledState.DISABLED.name().equals(dto.getState()) ) { + try { + controller.disableReportingTask(reportingTask); + } catch (final Exception e) { + logger.error("Failed to mark {} as disabled due to {}", reportingTask, e); + if ( logger.isDebugEnabled() ) { + logger.error("", e); + } + controller.getBulletinRepository().addBulletin(BulletinFactory.createBulletin( + "Reporting Tasks", Severity.ERROR.name(), "Failed to mark " + reportingTask + " as disabled due to " + e)); + } + } + } + } + private void updateReportingTask(final FlowController controller, final Element reportingTaskElement, final StringEncryptor encryptor) { + final ReportingTaskDTO dto = FlowFromDOMFactory.getReportingTask(reportingTaskElement, encryptor); + final ReportingTaskNode taskNode = controller.getReportingTaskNode(dto.getId()); + + if (!taskNode.getScheduledState().name().equals(dto.getState())) { + try { + switch (ScheduledState.valueOf(dto.getState())) { + case DISABLED: + if ( taskNode.isRunning() ) { + controller.stopReportingTask(taskNode); + } + controller.disableReportingTask(taskNode); + break; + case RUNNING: + if ( taskNode.getScheduledState() == ScheduledState.DISABLED ) { + controller.enableReportingTask(taskNode); + } + controller.startReportingTask(taskNode); + break; + case STOPPED: + if (taskNode.getScheduledState() == ScheduledState.DISABLED) { + controller.enableReportingTask(taskNode); + } else if (taskNode.getScheduledState() == ScheduledState.RUNNING) { + controller.stopReportingTask(taskNode); + } + break; + } + } catch (final IllegalStateException ise) { + logger.error("Failed to change Scheduled State of {} from {} to {} due to {}", taskNode, taskNode.getScheduledState().name(), dto.getState(), ise.toString()); + logger.error("", ise); + + // create bulletin for the Processor Node + controller.getBulletinRepository().addBulletin(BulletinFactory.createBulletin("Node Reconnection", Severity.ERROR.name(), + "Failed to change Scheduled State of " + taskNode + " from " + taskNode.getScheduledState().name() + " to " + dto.getState() + " due to " + ise.toString())); + + // create bulletin at Controller level. + controller.getBulletinRepository().addBulletin(BulletinFactory.createBulletin("Node Reconnection", Severity.ERROR.name(), + "Failed to change Scheduled State of " + taskNode + " from " + taskNode.getScheduledState().name() + " to " + dto.getState() + " due to " + ise.toString())); + } + } + } + + private ProcessGroup updateProcessGroup(final FlowController controller, final ProcessGroup parentGroup, final Element processGroupElement, final StringEncryptor encryptor) throws ProcessorInstantiationException { // get the parent group ID diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java index fe72ae45ca..355e303a61 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java @@ -52,6 +52,7 @@ import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; import org.apache.nifi.connectable.Position; +import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.logging.LogLevel; @@ -120,7 +121,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable private SchedulingStrategy schedulingStrategy; // guarded by read/write lock @SuppressWarnings("deprecation") - StandardProcessorNode(final Processor processor, final String uuid, final ValidationContextFactory validationContextFactory, + public StandardProcessorNode(final Processor processor, final String uuid, final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler, final ControllerServiceProvider controllerServiceProvider) { super(processor, uuid, validationContextFactory, controllerServiceProvider); @@ -985,6 +986,16 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable readLock.unlock(); } } + + @Override + public int getActiveThreadCount() { + readLock.lock(); + try { + return processScheduler.getActiveThreadCount(this); + } finally { + readLock.unlock(); + } + } @Override public boolean isValid() { @@ -1182,8 +1193,13 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable public void verifyCanStart() { readLock.lock(); try { - if (scheduledState.get() != ScheduledState.STOPPED) { - throw new IllegalStateException(this + " is not stopped"); + switch (getScheduledState()) { + case DISABLED: + throw new IllegalStateException(this + " cannot be started because it is disabled"); + case RUNNING: + throw new IllegalStateException(this + " cannot be started because it is already running"); + case STOPPED: + break; } verifyNoActiveThreads(); @@ -1194,6 +1210,31 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable readLock.unlock(); } } + + @Override + public void verifyCanStart(final Set ignoredReferences) { + switch (getScheduledState()) { + case DISABLED: + throw new IllegalStateException(this + " cannot be started because it is disabled"); + case RUNNING: + throw new IllegalStateException(this + " cannot be started because it is already running"); + case STOPPED: + break; + } + verifyNoActiveThreads(); + + final Set ids = new HashSet<>(); + for ( final ControllerServiceNode node : ignoredReferences ) { + ids.add(node.getIdentifier()); + } + + final Collection validationResults = getValidationErrors(ids); + for ( final ValidationResult result : validationResults ) { + if ( !result.isValid() ) { + throw new IllegalStateException(this + " cannot be started because it is not valid: " + result); + } + } + } @Override public void verifyCanStop() { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateManager.java index aa095d132d..30d4365d34 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateManager.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateManager.java @@ -42,24 +42,25 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.persistence.TemplateDeserializer; +import org.apache.nifi.persistence.TemplateSerializer; import org.apache.nifi.stream.io.ByteArrayInputStream; import org.apache.nifi.stream.io.ByteArrayOutputStream; import org.apache.nifi.stream.io.DataOutputStream; import org.apache.nifi.stream.io.StreamUtils; -import org.apache.nifi.persistence.TemplateDeserializer; -import org.apache.nifi.persistence.TemplateSerializer; import org.apache.nifi.web.api.dto.ConnectableDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.FlowSnippetDTO; import org.apache.nifi.web.api.dto.ProcessGroupDTO; import org.apache.nifi.web.api.dto.ProcessorConfigDTO; -import org.apache.nifi.web.api.dto.ProcessorConfigDTO.PropertyDescriptorDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupContentsDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO; import org.apache.nifi.web.api.dto.TemplateDTO; -import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -272,6 +273,11 @@ public class TemplateManager { if (snippet.getProcessGroups() != null) { scrubProcessGroups(snippet.getProcessGroups()); } + + // go through each controller service if specified + if (snippet.getControllerServices() != null) { + scrubControllerServices(snippet.getControllerServices()); + } } } @@ -315,7 +321,6 @@ public class TemplateManager { } } - processorConfig.setDescriptors(null); processorConfig.setCustomUiUrl(null); } @@ -323,6 +328,24 @@ public class TemplateManager { processorDTO.setValidationErrors(null); } } + + private void scrubControllerServices(final Set controllerServices) { + for ( final ControllerServiceDTO serviceDTO : controllerServices ) { + final Map properties = serviceDTO.getProperties(); + final Map descriptors = serviceDTO.getDescriptors(); + + if ( properties != null && descriptors != null ) { + for ( final PropertyDescriptorDTO descriptor : descriptors.values() ) { + if ( descriptor.isSensitive() ) { + properties.put(descriptor.getName(), null); + } + } + } + + serviceDTO.setCustomUiUrl(null); + serviceDTO.setValidationErrors(null); + } + } /** * Scrubs connections prior to saving. This includes removing available diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java index 7c3734ad45..18d2c5fcfa 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java @@ -16,11 +16,14 @@ */ package org.apache.nifi.controller.reporting; +import java.util.Collection; +import java.util.HashSet; +import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import org.apache.nifi.components.ValidationResult; import org.apache.nifi.controller.AbstractConfiguredComponent; -import org.apache.nifi.controller.Availability; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.controller.ControllerServiceLookup; import org.apache.nifi.controller.ProcessScheduler; @@ -28,7 +31,8 @@ import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.ValidationContextFactory; import org.apache.nifi.controller.annotation.OnConfigured; -import org.apache.nifi.controller.exception.ProcessorLifeCycleException; +import org.apache.nifi.controller.exception.ComponentLifeCycleException; +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.nar.NarCloseable; @@ -45,8 +49,8 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon private final AtomicReference schedulingStrategy = new AtomicReference<>(SchedulingStrategy.TIMER_DRIVEN); private final AtomicReference schedulingPeriod = new AtomicReference<>("5 mins"); - private final AtomicReference availability = new AtomicReference<>(Availability.NODE_ONLY); - + + private volatile String comment; private volatile ScheduledState scheduledState = ScheduledState.STOPPED; public AbstractReportingTaskNode(final ReportingTask reportingTask, final String id, @@ -58,16 +62,6 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon this.serviceLookup = controllerServiceProvider; } - @Override - public Availability getAvailability() { - return availability.get(); - } - - @Override - public void setAvailability(final Availability availability) { - this.availability.set(availability); - } - @Override public void setSchedulingStrategy(final SchedulingStrategy schedulingStrategy) { this.schedulingStrategy.set(schedulingStrategy); @@ -102,6 +96,11 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon public boolean isRunning() { return processScheduler.isScheduled(this) || processScheduler.getActiveThreadCount(this) > 0; } + + @Override + public int getActiveThreadCount() { + return processScheduler.getActiveThreadCount(this); + } @Override public ConfigurationContext getConfigurationContext() { @@ -142,14 +141,15 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon return removed; } - private void onConfigured() { + @SuppressWarnings("deprecation") + private void onConfigured() { // We need to invoke any method annotation with the OnConfigured annotation in order to // maintain backward compatibility. This will be removed when we remove the old, deprecated annotations. try (final NarCloseable x = NarCloseable.withNarLoader()) { final ConfigurationContext configContext = new StandardConfigurationContext(this, serviceLookup); ReflectionUtils.invokeMethodsWithAnnotation(OnConfigured.class, reportingTask, configContext); } catch (final Exception e) { - throw new ProcessorLifeCycleException("Failed to invoke On-Configured Lifecycle methods of " + reportingTask, e); + throw new ComponentLifeCycleException("Failed to invoke On-Configured Lifecycle methods of " + reportingTask, e); } } @@ -158,6 +158,16 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon } @Override + public String getComments() { + return comment; + } + + @Override + public void setComments(final String comment) { + this.comment = comment; + } + + @Override public void verifyCanDelete() { if (isRunning()) { throw new IllegalStateException("Cannot delete " + reportingTask + " because it is currently running"); @@ -207,4 +217,38 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon throw new IllegalStateException("Cannot update " + reportingTask + " because it is currently running"); } } + + @Override + public void verifyCanStart(final Set ignoredReferences) { + switch (getScheduledState()) { + case DISABLED: + throw new IllegalStateException(this + " cannot be started because it is disabled"); + case RUNNING: + throw new IllegalStateException(this + " cannot be started because it is already running"); + case STOPPED: + break; + } + final int activeThreadCount = getActiveThreadCount(); + if ( activeThreadCount > 0 ) { + throw new IllegalStateException(this + " cannot be started because it has " + activeThreadCount + " active threads already"); + } + + final Set ids = new HashSet<>(); + for ( final ControllerServiceNode node : ignoredReferences ) { + ids.add(node.getIdentifier()); + } + + final Collection validationResults = getValidationErrors(ids); + for ( final ValidationResult result : validationResults ) { + if ( !result.isValid() ) { + throw new IllegalStateException(this + " cannot be started because it is not valid: " + result); + } + } + } + + + @Override + public String toString() { + return "ReportingTask[id=" + getIdentifier() + ", name=" + getName() + "]"; + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingContext.java index ed48e20495..3d57533b2d 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingContext.java @@ -124,9 +124,20 @@ public class StandardReportingContext implements ReportingContext, ControllerSer public boolean isControllerServiceEnabled(final String serviceIdentifier) { return serviceProvider.isControllerServiceEnabled(serviceIdentifier); } + + @Override + public boolean isControllerServiceEnabling(final String serviceIdentifier) { + return serviceProvider.isControllerServiceEnabling(serviceIdentifier); + } @Override public ControllerServiceLookup getControllerServiceLookup() { return this; } + + @Override + public String getControllerServiceName(final String serviceIdentifier) { + return serviceProvider.getControllerServiceName(serviceIdentifier); + } + } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingInitializationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingInitializationContext.java index d576f9c0c0..435dbd0582 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingInitializationContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingInitializationContext.java @@ -22,6 +22,7 @@ import java.util.concurrent.TimeUnit; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerServiceLookup; import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.reporting.ReportingInitializationContext; import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.util.FormatUtils; @@ -33,13 +34,16 @@ public class StandardReportingInitializationContext implements ReportingInitiali private final String schedulingPeriod; private final SchedulingStrategy schedulingStrategy; private final ControllerServiceProvider serviceProvider; - - public StandardReportingInitializationContext(final String id, final String name, final SchedulingStrategy schedulingStrategy, final String schedulingPeriod, final ControllerServiceProvider serviceProvider) { + private final ComponentLog logger; + + public StandardReportingInitializationContext(final String id, final String name, final SchedulingStrategy schedulingStrategy, + final String schedulingPeriod, final ComponentLog logger, final ControllerServiceProvider serviceProvider) { this.id = id; this.name = name; this.schedulingPeriod = schedulingPeriod; this.serviceProvider = serviceProvider; this.schedulingStrategy = schedulingStrategy; + this.logger = logger; } @Override @@ -89,8 +93,23 @@ public class StandardReportingInitializationContext implements ReportingInitiali return serviceProvider.isControllerServiceEnabled(serviceIdentifier); } + @Override + public boolean isControllerServiceEnabling(final String serviceIdentifier) { + return serviceProvider.isControllerServiceEnabling(serviceIdentifier); + } + @Override public ControllerServiceLookup getControllerServiceLookup() { return this; } + + @Override + public String getControllerServiceName(final String serviceIdentifier) { + return serviceProvider.getControllerServiceName(serviceIdentifier); + } + + @Override + public ComponentLog getLogger() { + return logger; + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java index 1fde9aabd8..6ecb9910e7 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java @@ -116,12 +116,23 @@ public class StandardRepositoryRecord implements RepositoryRecord { public void setWorking(final FlowFileRecord flowFile, final String attributeKey, final String attributeValue) { workingFlowFileRecord = flowFile; - updatedAttributes.put(attributeKey, attributeValue); + + // If setting attribute to same value as original, don't add to updated attributes + final String currentValue = originalAttributes.get(attributeKey); + if ( currentValue == null || !currentValue.equals(attributeValue) ) { + updatedAttributes.put(attributeKey, attributeValue); + } } public void setWorking(final FlowFileRecord flowFile, final Map updatedAttribs) { workingFlowFileRecord = flowFile; - updatedAttributes.putAll(updatedAttribs); + + for ( final Map.Entry entry : updatedAttribs.entrySet() ) { + final String currentValue = originalAttributes.get(entry.getKey()); + if ( currentValue == null || !currentValue.equals(entry.getValue()) ) { + updatedAttributes.put(entry.getKey(), entry.getValue()); + } + } } @Override diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java index 4407451a8a..772582358d 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java @@ -19,6 +19,8 @@ package org.apache.nifi.controller.scheduling; import static java.util.Objects.requireNonNull; import java.lang.reflect.InvocationTargetException; +import java.util.HashSet; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; @@ -32,20 +34,26 @@ import org.apache.nifi.annotation.lifecycle.OnEnabled; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.annotation.lifecycle.OnUnscheduled; +import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Funnel; import org.apache.nifi.connectable.Port; import org.apache.nifi.controller.AbstractPort; import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.Heartbeater; import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.annotation.OnConfigured; +import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.controller.service.ControllerServiceState; +import org.apache.nifi.controller.service.StandardConfigurationContext; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.engine.FlowEngine; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.logging.ProcessorLog; import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.processor.SchedulingContext; @@ -144,6 +152,8 @@ public final class StandardProcessScheduler implements ProcessScheduler { componentLifeCycleThreadPool.shutdown(); } + + @Override public void schedule(final ReportingTaskNode taskNode) { final ScheduleState scheduleState = getScheduleState(requireNonNull(taskNode)); if (scheduleState.isScheduled()) { @@ -176,16 +186,11 @@ public final class StandardProcessScheduler implements ProcessScheduler { } break; - } catch (final InvocationTargetException ite) { - LOG.error("Failed to invoke the On-Scheduled Lifecycle methods of {} due to {}; administratively yielding this ReportingTask and will attempt to schedule it again after {}", - new Object[]{reportingTask, ite.getTargetException(), administrativeYieldDuration}); - LOG.error("", ite.getTargetException()); - - try { - Thread.sleep(administrativeYieldMillis); - } catch (final InterruptedException ie) { - } } catch (final Exception e) { + final Throwable cause = (e instanceof InvocationTargetException) ? e.getCause() : e; + final ComponentLog componentLog = new SimpleProcessLogger(reportingTask.getIdentifier(), reportingTask); + componentLog.error("Failed to invoke @OnEnabled method due to {}", cause); + LOG.error("Failed to invoke the On-Scheduled Lifecycle methods of {} due to {}; administratively yielding this ReportingTask and will attempt to schedule it again after {}", new Object[]{reportingTask, e.toString(), administrativeYieldDuration}, e); try { @@ -200,18 +205,23 @@ public final class StandardProcessScheduler implements ProcessScheduler { }; componentLifeCycleThreadPool.execute(startReportingTaskRunnable); + taskNode.setScheduledState(ScheduledState.RUNNING); } + + @Override public void unschedule(final ReportingTaskNode taskNode) { final ScheduleState scheduleState = getScheduleState(requireNonNull(taskNode)); if (!scheduleState.isScheduled()) { return; } - + + taskNode.verifyCanStop(); final SchedulingAgent agent = getSchedulingAgent(taskNode.getSchedulingStrategy()); final ReportingTask reportingTask = taskNode.getReportingTask(); scheduleState.setScheduled(false); - + taskNode.setScheduledState(ScheduledState.STOPPED); + final Runnable unscheduleReportingTaskRunnable = new Runnable() { @SuppressWarnings("deprecation") @Override @@ -222,18 +232,15 @@ public final class StandardProcessScheduler implements ProcessScheduler { try (final NarCloseable x = NarCloseable.withNarLoader()) { ReflectionUtils.invokeMethodsWithAnnotation(OnUnscheduled.class, org.apache.nifi.processor.annotation.OnUnscheduled.class, reportingTask, configurationContext); } - } catch (final InvocationTargetException ite) { - LOG.error("Failed to invoke the @OnConfigured methods of {} due to {}; administratively yielding this ReportingTask and will attempt to schedule it again after {}", - new Object[]{reportingTask, ite.getTargetException(), administrativeYieldDuration}); - LOG.error("", ite.getTargetException()); - - try { - Thread.sleep(administrativeYieldMillis); - } catch (final InterruptedException ie) { - } } catch (final Exception e) { - LOG.error("Failed to invoke the @OnConfigured methods of {} due to {}; administratively yielding this ReportingTask and will attempt to schedule it again after {}", - new Object[]{reportingTask, e.toString(), administrativeYieldDuration}, e); + final Throwable cause = (e instanceof InvocationTargetException) ? e.getCause() : e; + final ComponentLog componentLog = new SimpleProcessLogger(reportingTask.getIdentifier(), reportingTask); + componentLog.error("Failed to invoke @OnUnscheduled method due to {}", cause); + + LOG.error("Failed to invoke the @OnUnscheduled methods of {} due to {}; administratively yielding this ReportingTask and will attempt to schedule it again after {}", + reportingTask, cause.toString(), administrativeYieldDuration); + LOG.error("", cause); + try { Thread.sleep(administrativeYieldMillis); } catch (final InterruptedException ie) { @@ -274,20 +281,40 @@ public final class StandardProcessScheduler implements ProcessScheduler { } if (!procNode.isValid()) { - throw new IllegalStateException("Processor " + procNode.getName() + " is not in a valid state"); + throw new IllegalStateException("Processor " + procNode.getName() + " is not in a valid state due to " + procNode.getValidationErrors()); } final Runnable startProcRunnable = new Runnable() { - @SuppressWarnings("deprecation") @Override + @SuppressWarnings("deprecation") public void run() { try (final NarCloseable x = NarCloseable.withNarLoader()) { long lastStopTime = scheduleState.getLastStopTime(); final StandardProcessContext processContext = new StandardProcessContext(procNode, controllerServiceProvider, encryptor); - while (true) { + final Set serviceIds = new HashSet<>(); + for ( final PropertyDescriptor descriptor : processContext.getProperties().keySet() ) { + final Class serviceDefinition = descriptor.getControllerServiceDefinition(); + if ( serviceDefinition != null ) { + final String serviceId = processContext.getProperty(descriptor).getValue(); + if ( serviceId != null ) { + serviceIds.add(serviceId); + } + } + } + + attemptOnScheduled: while (true) { try { synchronized (scheduleState) { + for ( final String serviceId : serviceIds ) { + final boolean enabled = processContext.isControllerServiceEnabled(serviceId); + if ( !enabled ) { + LOG.debug("Controller Service with ID {} is not yet enabled, so will not start {} yet", serviceId, procNode); + Thread.sleep(administrativeYieldMillis); + continue attemptOnScheduled; + } + } + // if no longer scheduled to run, then we're finished. This can happen, for example, // if the @OnScheduled method throws an Exception and the user stops the processor // while we're administratively yielded. @@ -308,11 +335,12 @@ public final class StandardProcessScheduler implements ProcessScheduler { return; } } catch (final Exception e) { + final Throwable cause = (e instanceof InvocationTargetException) ? e.getCause() : e; final ProcessorLog procLog = new SimpleProcessLogger(procNode.getIdentifier(), procNode.getProcessor()); procLog.error("{} failed to invoke @OnScheduled method due to {}; processor will not be scheduled to run for {}", - new Object[]{procNode.getProcessor(), e.getCause(), administrativeYieldDuration}, e.getCause()); - LOG.error("Failed to invoke @OnScheduled method due to {}", e.getCause().toString(), e.getCause()); + new Object[]{procNode.getProcessor(), cause, administrativeYieldDuration}, cause); + LOG.error("Failed to invoke @OnScheduled method due to {}", cause.toString(), cause); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnUnscheduled.class, procNode.getProcessor(), processContext); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, procNode.getProcessor(), processContext); @@ -535,11 +563,6 @@ public final class StandardProcessScheduler implements ProcessScheduler { } procNode.setScheduledState(ScheduledState.STOPPED); - - try (final NarCloseable x = NarCloseable.withNarLoader()) { - final ProcessorLog processorLog = new SimpleProcessLogger(procNode.getIdentifier(), procNode.getProcessor()); - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnEnabled.class, procNode.getProcessor(), processorLog); - } } @Override @@ -549,11 +572,6 @@ public final class StandardProcessScheduler implements ProcessScheduler { } procNode.setScheduledState(ScheduledState.DISABLED); - - try (final NarCloseable x = NarCloseable.withNarLoader()) { - final ProcessorLog processorLog = new SimpleProcessLogger(procNode.getIdentifier(), procNode.getProcessor()); - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnDisabled.class, procNode.getProcessor(), processorLog); - } } public synchronized void enableReportingTask(final ReportingTaskNode taskNode) { @@ -562,10 +580,6 @@ public final class StandardProcessScheduler implements ProcessScheduler { } taskNode.setScheduledState(ScheduledState.STOPPED); - - try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnEnabled.class, taskNode.getReportingTask()); - } } public synchronized void disableReportingTask(final ReportingTaskNode taskNode) { @@ -574,10 +588,6 @@ public final class StandardProcessScheduler implements ProcessScheduler { } taskNode.setScheduledState(ScheduledState.DISABLED); - - try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnDisabled.class, taskNode.getReportingTask()); - } } @Override @@ -605,4 +615,114 @@ public final class StandardProcessScheduler implements ProcessScheduler { } return scheduleState; } + + @Override + public void enableControllerService(final ControllerServiceNode service) { + service.setState(ControllerServiceState.ENABLING); + final ScheduleState scheduleState = getScheduleState(service); + + final Runnable enableRunnable = new Runnable() { + @Override + public void run() { + try (final NarCloseable x = NarCloseable.withNarLoader()) { + long lastStopTime = scheduleState.getLastStopTime(); + final ConfigurationContext configContext = new StandardConfigurationContext(service, controllerServiceProvider); + + while (true) { + try { + synchronized (scheduleState) { + // if no longer enabled, then we're finished. This can happen, for example, + // if the @OnEnabled method throws an Exception and the user disables the service + // while we're administratively yielded. + // + // we also check if the schedule state's last stop time is equal to what it was before. + // if not, then means that the service has been disabled and enabled again, so we should just + // bail; another thread will be responsible for invoking the @OnEnabled methods. + if (!scheduleState.isScheduled() || scheduleState.getLastStopTime() != lastStopTime) { + return; + } + + ReflectionUtils.invokeMethodsWithAnnotation(OnEnabled.class, service.getControllerServiceImplementation(), configContext); + heartbeater.heartbeat(); + service.setState(ControllerServiceState.ENABLED); + return; + } + } catch (final Exception e) { + final Throwable cause = (e instanceof InvocationTargetException) ? e.getCause() : e; + + final ComponentLog componentLog = new SimpleProcessLogger(service.getIdentifier(), service); + componentLog.error("Failed to invoke @OnEnabled method due to {}", cause); + LOG.error("Failed to invoke @OnEnabled method of {} due to {}", service.getControllerServiceImplementation(), cause.toString()); + if ( LOG.isDebugEnabled() ) { + LOG.error("", cause); + } + + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnDisabled.class, service.getControllerServiceImplementation(), configContext); + Thread.sleep(administrativeYieldMillis); + continue; + } + } + } catch (final Throwable t) { + final Throwable cause = (t instanceof InvocationTargetException) ? t.getCause() : t; + final ComponentLog componentLog = new SimpleProcessLogger(service.getIdentifier(), service); + componentLog.error("Failed to invoke @OnEnabled method due to {}", cause); + + LOG.error("Failed to invoke @OnEnabled method on {} due to {}", service.getControllerServiceImplementation(), cause.toString()); + if ( LOG.isDebugEnabled() ) { + LOG.error("", cause); + } + } + } + }; + + scheduleState.setScheduled(true); + componentLifeCycleThreadPool.execute(enableRunnable); + } + + @Override + public void disableControllerService(final ControllerServiceNode service) { + service.verifyCanDisable(); + + final ScheduleState state = getScheduleState(requireNonNull(service)); + final Runnable disableRunnable = new Runnable() { + @Override + public void run() { + synchronized (state) { + state.setScheduled(false); + } + + try (final NarCloseable x = NarCloseable.withNarLoader()) { + final ConfigurationContext configContext = new StandardConfigurationContext(service, controllerServiceProvider); + + while(true) { + try { + ReflectionUtils.invokeMethodsWithAnnotation(OnDisabled.class, service.getControllerServiceImplementation(), configContext); + heartbeater.heartbeat(); + service.setState(ControllerServiceState.DISABLED); + return; + } catch (final Exception e) { + final Throwable cause = (e instanceof InvocationTargetException) ? e.getCause() : e; + final ComponentLog componentLog = new SimpleProcessLogger(service.getIdentifier(), service); + componentLog.error("Failed to invoke @OnDisabled method due to {}", cause); + + LOG.error("Failed to invoke @OnDisabled method of {} due to {}", service.getControllerServiceImplementation(), cause.toString()); + if ( LOG.isDebugEnabled() ) { + LOG.error("", cause); + } + + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnDisabled.class, service.getControllerServiceImplementation(), configContext); + try { + Thread.sleep(administrativeYieldMillis); + } catch (final InterruptedException ie) {} + + continue; + } + } + } + } + }; + + service.setState(ControllerServiceState.DISABLING); + componentLifeCycleThreadPool.execute(disableRunnable); + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java index db44b5f0d6..1fde670901 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java @@ -17,30 +17,29 @@ package org.apache.nifi.controller.service; import java.io.BufferedInputStream; -import java.io.File; import java.io.IOException; import java.io.InputStream; -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.StandardOpenOption; import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Set; -import javax.xml.XMLConstants; import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilderFactory; import javax.xml.parsers.ParserConfigurationException; -import javax.xml.validation.Schema; -import javax.xml.validation.SchemaFactory; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.apache.nifi.controller.FlowFromDOMFactory; +import org.apache.nifi.encrypt.StringEncryptor; +import org.apache.nifi.reporting.BulletinRepository; import org.apache.nifi.util.DomUtils; -import org.apache.nifi.util.file.FileUtils; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.w3c.dom.Document; import org.w3c.dom.Element; -import org.w3c.dom.NodeList; import org.xml.sax.SAXException; import org.xml.sax.SAXParseException; @@ -49,35 +48,14 @@ import org.xml.sax.SAXParseException; */ public class ControllerServiceLoader { - private static final Log logger = LogFactory.getLog(ControllerServiceLoader.class); + private static final Logger logger = LoggerFactory.getLogger(ControllerServiceLoader.class); - private final Path serviceConfigXmlPath; - public ControllerServiceLoader(final Path serviceConfigXmlPath) throws IOException { - final File serviceConfigXmlFile = serviceConfigXmlPath.toFile(); - if (!serviceConfigXmlFile.exists() || !serviceConfigXmlFile.canRead()) { - throw new IOException(serviceConfigXmlPath + " does not appear to exist or cannot be read. Cannot load configuration."); - } - - this.serviceConfigXmlPath = serviceConfigXmlPath; - } - - public List loadControllerServices(final ControllerServiceProvider provider) throws IOException { - final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI); + public static List loadControllerServices(final ControllerServiceProvider provider, final InputStream serializedStream, final StringEncryptor encryptor, final BulletinRepository bulletinRepo, final boolean autoResumeState) throws IOException { final DocumentBuilderFactory documentBuilderFactory = DocumentBuilderFactory.newInstance(); - InputStream fis = null; - BufferedInputStream bis = null; documentBuilderFactory.setNamespaceAware(true); - final List services = new ArrayList<>(); - - try { - final URL configurationResource = this.getClass().getResource("/ControllerServiceConfiguration.xsd"); - if (configurationResource == null) { - throw new NullPointerException("Unable to load XML Schema for ControllerServiceConfiguration"); - } - final Schema schema = schemaFactory.newSchema(configurationResource); - documentBuilderFactory.setSchema(schema); + try (final InputStream in = new BufferedInputStream(serializedStream)) { final DocumentBuilder builder = documentBuilderFactory.newDocumentBuilder(); builder.setErrorHandler(new org.xml.sax.ErrorHandler() { @@ -109,43 +87,72 @@ public class ControllerServiceLoader { throw err; } }); - - //if controllerService.xml does not exist, create an empty file... - fis = Files.newInputStream(this.serviceConfigXmlPath, StandardOpenOption.READ); - bis = new BufferedInputStream(fis); - if (Files.size(this.serviceConfigXmlPath) > 0) { - final Document document = builder.parse(bis); - final NodeList servicesNodes = document.getElementsByTagName("services"); - final Element servicesElement = (Element) servicesNodes.item(0); - - final List serviceNodes = DomUtils.getChildElementsByTagName(servicesElement, "service"); - for (final Element serviceElement : serviceNodes) { - //get properties for the specific controller task - id, name, class, - //and schedulingPeriod must be set - final String serviceId = DomUtils.getChild(serviceElement, "identifier").getTextContent().trim(); - final String serviceClass = DomUtils.getChild(serviceElement, "class").getTextContent().trim(); - - //set the class to be used for the configured controller task - final ControllerServiceNode serviceNode = provider.createControllerService(serviceClass, serviceId, false); - - //optional task-specific properties - for (final Element optionalProperty : DomUtils.getChildElementsByTagName(serviceElement, "property")) { - final String name = optionalProperty.getAttribute("name").trim(); - final String value = optionalProperty.getTextContent().trim(); - serviceNode.setProperty(name, value); - } - - services.add(serviceNode); - provider.enableControllerService(serviceNode); - } - } + + final Document document = builder.parse(in); + final Element controllerServices = document.getDocumentElement(); + final List serviceElements = DomUtils.getChildElementsByTagName(controllerServices, "controllerService"); + return new ArrayList(loadControllerServices(serviceElements, provider, encryptor, bulletinRepo, autoResumeState)); } catch (SAXException | ParserConfigurationException sxe) { throw new IOException(sxe); - } finally { - FileUtils.closeQuietly(fis); - FileUtils.closeQuietly(bis); } + } + + public static Collection loadControllerServices(final List serviceElements, final ControllerServiceProvider provider, final StringEncryptor encryptor, final BulletinRepository bulletinRepo, final boolean autoResumeState) { + final Map nodeMap = new HashMap<>(); + for ( final Element serviceElement : serviceElements ) { + final ControllerServiceNode serviceNode = createControllerService(provider, serviceElement, encryptor); + // We need to clone the node because it will be used in a separate thread below, and + // Element is not thread-safe. + nodeMap.put(serviceNode, (Element) serviceElement.cloneNode(true)); + } + for ( final Map.Entry entry : nodeMap.entrySet() ) { + configureControllerService(entry.getKey(), entry.getValue(), encryptor); + } + + // Start services + if ( autoResumeState ) { + final Set nodesToEnable = new HashSet<>(); + + for ( final ControllerServiceNode node : nodeMap.keySet() ) { + final Element controllerServiceElement = nodeMap.get(node); - return services; + final ControllerServiceDTO dto; + synchronized (controllerServiceElement.getOwnerDocument()) { + dto = FlowFromDOMFactory.getControllerService(controllerServiceElement, encryptor); + } + + final ControllerServiceState state = ControllerServiceState.valueOf(dto.getState()); + if (state == ControllerServiceState.ENABLED) { + nodesToEnable.add(node); + } + } + + provider.enableControllerServices(nodesToEnable); + } + + return nodeMap.keySet(); + } + + + private static ControllerServiceNode createControllerService(final ControllerServiceProvider provider, final Element controllerServiceElement, final StringEncryptor encryptor) { + final ControllerServiceDTO dto = FlowFromDOMFactory.getControllerService(controllerServiceElement, encryptor); + + final ControllerServiceNode node = provider.createControllerService(dto.getType(), dto.getId(), false); + node.setName(dto.getName()); + node.setComments(dto.getComments()); + return node; + } + + private static void configureControllerService(final ControllerServiceNode node, final Element controllerServiceElement, final StringEncryptor encryptor) { + final ControllerServiceDTO dto = FlowFromDOMFactory.getControllerService(controllerServiceElement, encryptor); + node.setAnnotationData(dto.getAnnotationData()); + + for (final Map.Entry entry : dto.getProperties().entrySet()) { + if (entry.getValue() == null) { + node.removeProperty(entry.getKey()); + } else { + node.setProperty(entry.getKey(), entry.getValue()); + } + } } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInitializationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInitializationContext.java index 8b5f27f2d8..8d46b05c88 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInitializationContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInitializationContext.java @@ -21,14 +21,17 @@ import java.util.Set; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerServiceInitializationContext; import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.logging.ComponentLog; public class StandardControllerServiceInitializationContext implements ControllerServiceInitializationContext, ControllerServiceLookup { private final String id; private final ControllerServiceProvider serviceProvider; + private final ComponentLog logger; - public StandardControllerServiceInitializationContext(final String identifier, final ControllerServiceProvider serviceProvider) { + public StandardControllerServiceInitializationContext(final String identifier, final ComponentLog logger, final ControllerServiceProvider serviceProvider) { this.id = identifier; + this.logger = logger; this.serviceProvider = serviceProvider; } @@ -61,4 +64,19 @@ public class StandardControllerServiceInitializationContext implements Controlle public boolean isControllerServiceEnabled(final ControllerService service) { return serviceProvider.isControllerServiceEnabled(service); } + + @Override + public boolean isControllerServiceEnabling(String serviceIdentifier) { + return serviceProvider.isControllerServiceEnabling(serviceIdentifier); + } + + @Override + public String getControllerServiceName(final String serviceIdentifier) { + return serviceProvider.getControllerServiceName(serviceIdentifier); + } + + @Override + public ComponentLog getLogger() { + return logger; + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java index 741caeca72..e768b9ab18 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java @@ -16,22 +16,23 @@ */ package org.apache.nifi.controller.service; +import java.util.Collection; +import java.util.Collections; import java.util.HashSet; import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.nifi.components.ValidationResult; import org.apache.nifi.controller.AbstractConfiguredComponent; -import org.apache.nifi.controller.Availability; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.controller.ConfiguredComponent; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ValidationContextFactory; import org.apache.nifi.controller.annotation.OnConfigured; -import org.apache.nifi.controller.exception.ProcessorLifeCycleException; +import org.apache.nifi.controller.exception.ComponentLifeCycleException; import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.util.ReflectionUtils; @@ -41,55 +42,24 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i private final ControllerService implementation; private final ControllerServiceProvider serviceProvider; - private final AtomicReference availability = new AtomicReference<>(Availability.NODE_ONLY); - private final AtomicBoolean disabled = new AtomicBoolean(true); + private final AtomicReference stateRef = new AtomicReference<>(ControllerServiceState.DISABLED); private final ReadWriteLock rwLock = new ReentrantReadWriteLock(); private final Lock readLock = rwLock.readLock(); private final Lock writeLock = rwLock.writeLock(); private final Set referencingComponents = new HashSet<>(); + private String comment; public StandardControllerServiceNode(final ControllerService proxiedControllerService, final ControllerService implementation, final String id, final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider) { - super(proxiedControllerService, id, validationContextFactory, serviceProvider); + super(implementation, id, validationContextFactory, serviceProvider); this.proxedControllerService = proxiedControllerService; this.implementation = implementation; this.serviceProvider = serviceProvider; } - @Override - public boolean isDisabled() { - return disabled.get(); - } - - @Override - public void setDisabled(final boolean disabled) { - if (!disabled && !isValid()) { - throw new IllegalStateException("Cannot enable Controller Service " + implementation + " because it is not valid"); - } - - if (disabled) { - // do not allow a Controller Service to be disabled if it's currently being used. - final Set runningRefs = getReferences().getRunningReferences(); - if (!runningRefs.isEmpty()) { - throw new IllegalStateException("Cannot disable Controller Service because it is referenced (either directly or indirectly) by " + runningRefs.size() + " different components that are currently running"); - } - } - - this.disabled.set(disabled); - } - - @Override - public Availability getAvailability() { - return availability.get(); - } - - @Override - public void setAvailability(final Availability availability) { - this.availability.set(availability); - } - + @Override public ControllerService getProxiedControllerService() { return proxedControllerService; @@ -132,7 +102,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i @Override public void verifyModifiable() throws IllegalStateException { - if (!isDisabled()) { + if (getState() != ControllerServiceState.DISABLED) { throw new IllegalStateException("Cannot modify Controller Service configuration because it is currently enabled. Please disable the Controller Service first."); } } @@ -140,7 +110,6 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i @Override public void setProperty(final String name, final String value) { super.setProperty(name, value); - onConfigured(); } @@ -160,37 +129,102 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i final ConfigurationContext configContext = new StandardConfigurationContext(this, serviceProvider); ReflectionUtils.invokeMethodsWithAnnotation(OnConfigured.class, implementation, configContext); } catch (final Exception e) { - throw new ProcessorLifeCycleException("Failed to invoke On-Configured Lifecycle methods of " + implementation, e); + throw new ComponentLifeCycleException("Failed to invoke On-Configured Lifecycle methods of " + implementation, e); } } @Override public void verifyCanDelete() { - if ( !isDisabled() ) { + if ( getState() != ControllerServiceState.DISABLED ) { throw new IllegalStateException(implementation + " cannot be deleted because it is not disabled"); } } @Override public void verifyCanDisable() { + verifyCanDisable(Collections.emptySet()); + } + + @Override + public void verifyCanDisable(final Set ignoreReferences) { + final ControllerServiceState state = getState(); + if ( state != ControllerServiceState.ENABLED && state != ControllerServiceState.ENABLING ) { + throw new IllegalStateException("Cannot disable " + getControllerServiceImplementation() + " because it is not enabled"); + } + final ControllerServiceReference references = getReferences(); - final int numRunning = references.getRunningReferences().size(); - if ( numRunning > 0 ) { - throw new IllegalStateException(implementation + " cannot be disabled because it is referenced by " + numRunning + " components that are currently running"); + + for ( final ConfiguredComponent activeReference : references.getActiveReferences() ) { + if ( !ignoreReferences.contains(activeReference) ) { + throw new IllegalStateException(implementation + " cannot be disabled because it is referenced by at least one component that is currently running"); + } } } @Override public void verifyCanEnable() { - if ( !isDisabled() ) { + if ( getState() != ControllerServiceState.DISABLED ) { throw new IllegalStateException(implementation + " cannot be enabled because it is not disabled"); } + + if ( !isValid() ) { + throw new IllegalStateException(implementation + " cannot be enabled because it is not valid: " + getValidationErrors()); + } + } + + @Override + public void verifyCanEnable(final Set ignoredReferences) { + if (getState() != ControllerServiceState.DISABLED) { + throw new IllegalStateException(implementation + " cannot be enabled because it is not disabled"); + } + + final Set ids = new HashSet<>(); + for ( final ControllerServiceNode node : ignoredReferences ) { + ids.add(node.getIdentifier()); + } + + final Collection validationResults = getValidationErrors(ids); + for ( final ValidationResult result : validationResults ) { + if ( !result.isValid() ) { + throw new IllegalStateException(implementation + " cannot be enabled because it is not valid: " + result); + } + } } @Override public void verifyCanUpdate() { - if ( !isDisabled() ) { + if ( getState() != ControllerServiceState.DISABLED ) { throw new IllegalStateException(implementation + " cannot be updated because it is not disabled"); } } + + @Override + public String getComments() { + readLock.lock(); + try { + return comment; + } finally { + readLock.unlock(); + } + } + + @Override + public void setComments(final String comment) { + writeLock.lock(); + try { + this.comment = comment; + } finally { + writeLock.unlock(); + } + } + + @Override + public ControllerServiceState getState() { + return stateRef.get(); + } + + @Override + public void setState(final ControllerServiceState state) { + this.stateRef.set(state); + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java index a40e43e5e6..d8f1338bc0 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java @@ -23,26 +23,39 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.lang.reflect.Proxy; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import org.apache.nifi.annotation.lifecycle.OnAdded; -import org.apache.nifi.annotation.lifecycle.OnEnabled; -import org.apache.nifi.annotation.lifecycle.OnDisabled; import org.apache.nifi.annotation.lifecycle.OnRemoved; +import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.controller.ConfiguredComponent; import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.controller.ProcessScheduler; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.ValidationContextFactory; -import org.apache.nifi.controller.exception.ControllerServiceAlreadyExistsException; -import org.apache.nifi.controller.exception.ControllerServiceNotFoundException; -import org.apache.nifi.controller.exception.ProcessorLifeCycleException; +import org.apache.nifi.controller.exception.ControllerServiceInstantiationException; +import org.apache.nifi.controller.exception.ComponentLifeCycleException; +import org.apache.nifi.events.BulletinFactory; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.NarCloseable; +import org.apache.nifi.processor.SimpleProcessLogger; import org.apache.nifi.processor.StandardValidationContextFactory; +import org.apache.nifi.reporting.BulletinRepository; +import org.apache.nifi.reporting.Severity; import org.apache.nifi.util.ObjectHolder; import org.apache.nifi.util.ReflectionUtils; import org.slf4j.Logger; @@ -55,8 +68,10 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi private static final Logger logger = LoggerFactory.getLogger(StandardControllerServiceProvider.class); - private final Map controllerServices; + private final ProcessScheduler processScheduler; + private final ConcurrentMap controllerServices; private static final Set validDisabledMethods; + private final BulletinRepository bulletinRepo; static { // methods that are okay to be called when the service is disabled. @@ -70,10 +85,12 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi validDisabledMethods = Collections.unmodifiableSet(validMethods); } - public StandardControllerServiceProvider() { + public StandardControllerServiceProvider(final ProcessScheduler scheduler, final BulletinRepository bulletinRepo) { // the following 2 maps must be updated atomically, but we do not lock around them because they are modified // only in the createControllerService method, and both are modified before the method returns this.controllerServices = new ConcurrentHashMap<>(); + this.processScheduler = scheduler; + this.bulletinRepo = bulletinRepo; } private Class[] getInterfaces(final Class cls) { @@ -95,21 +112,24 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi populateInterfaces(superClass, interfacesDefinedThusFar); } } - + @Override public ControllerServiceNode createControllerService(final String type, final String id, final boolean firstTimeAdded) { if (type == null || id == null) { throw new NullPointerException(); } - if (controllerServices.containsKey(id)) { - throw new ControllerServiceAlreadyExistsException(id); - } - + final ClassLoader currentContextClassLoader = Thread.currentThread().getContextClassLoader(); try { final ClassLoader cl = ExtensionManager.getClassLoader(type); - Thread.currentThread().setContextClassLoader(cl); - final Class rawClass = Class.forName(type, false, cl); + final Class rawClass; + if ( cl == null ) { + rawClass = Class.forName(type); + } else { + Thread.currentThread().setContextClassLoader(cl); + rawClass = Class.forName(type, false, cl); + } + final Class controllerServiceClass = rawClass.asSubclass(ControllerService.class); final ControllerService originalService = controllerServiceClass.newInstance(); @@ -117,8 +137,16 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi final InvocationHandler invocationHandler = new InvocationHandler() { @Override public Object invoke(final Object proxy, final Method method, final Object[] args) throws Throwable { + + final String methodName = method.getName(); + if("initialize".equals(methodName) || "onPropertyModified".equals(methodName)){ + throw new UnsupportedOperationException(method + " may only be invoked by the NiFi framework"); + } + final ControllerServiceNode node = serviceNodeHolder.get(); - if (node.isDisabled() && !validDisabledMethods.contains(method)) { + final ControllerServiceState state = node.getState(); + final boolean disabled = (state != ControllerServiceState.ENABLED); // only allow method call if service state is ENABLED. + if (disabled && !validDisabledMethods.contains(method)) { // Use nar class loader here because we are implicitly calling toString() on the original implementation. try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { throw new IllegalStateException("Cannot invoke method " + method + " on Controller Service " + originalService + " because the Controller Service is disabled"); @@ -137,30 +165,35 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi } }; - final ControllerService proxiedService = (ControllerService) Proxy.newProxyInstance(cl, getInterfaces(controllerServiceClass), invocationHandler); - logger.info("Loaded service {} as configured.", type); + final ControllerService proxiedService; + if ( cl == null ) { + proxiedService = (ControllerService) Proxy.newProxyInstance(getClass().getClassLoader(), getInterfaces(controllerServiceClass), invocationHandler); + } else { + proxiedService = (ControllerService) Proxy.newProxyInstance(cl, getInterfaces(controllerServiceClass), invocationHandler); + } + logger.info("Created Controller Service of type {} with identifier {}", type, id); - originalService.initialize(new StandardControllerServiceInitializationContext(id, this)); + final ComponentLog serviceLogger = new SimpleProcessLogger(id, originalService); + originalService.initialize(new StandardControllerServiceInitializationContext(id, serviceLogger, this)); final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(this); final ControllerServiceNode serviceNode = new StandardControllerServiceNode(proxiedService, originalService, id, validationContextFactory, this); serviceNodeHolder.set(serviceNode); - serviceNode.setAnnotationData(null); - serviceNode.setName(id); + serviceNode.setName(rawClass.getSimpleName()); if ( firstTimeAdded ) { try (final NarCloseable x = NarCloseable.withNarLoader()) { ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, originalService); } catch (final Exception e) { - throw new ProcessorLifeCycleException("Failed to invoke On-Added Lifecycle methods of " + originalService, e); + throw new ComponentLifeCycleException("Failed to invoke On-Added Lifecycle methods of " + originalService, e); } } this.controllerServices.put(id, serviceNode); return serviceNode; } catch (final Throwable t) { - throw new ControllerServiceNotFoundException(t); + throw new ControllerServiceInstantiationException(t); } finally { if (currentContextClassLoader != null) { Thread.currentThread().setContextClassLoader(currentContextClassLoader); @@ -168,29 +201,242 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi } } + + + @Override + public void disableReferencingServices(final ControllerServiceNode serviceNode) { + // Get a list of all Controller Services that need to be disabled, in the order that they need to be + // disabled. + final List toDisable = findRecursiveReferences(serviceNode, ControllerServiceNode.class); + final Set serviceSet = new HashSet<>(toDisable); + + for ( final ControllerServiceNode nodeToDisable : toDisable ) { + final ControllerServiceState state = nodeToDisable.getState(); + + if ( state != ControllerServiceState.DISABLED && state != ControllerServiceState.DISABLING ) { + nodeToDisable.verifyCanDisable(serviceSet); + } + } + + Collections.reverse(toDisable); + for ( final ControllerServiceNode nodeToDisable : toDisable ) { + final ControllerServiceState state = nodeToDisable.getState(); + + if ( state != ControllerServiceState.DISABLED && state != ControllerServiceState.DISABLING ) { + disableControllerService(nodeToDisable); + } + } + } + + + @Override + public void scheduleReferencingComponents(final ControllerServiceNode serviceNode) { + // find all of the schedulable components (processors, reporting tasks) that refer to this Controller Service, + // or a service that references this controller service, etc. + final List processors = findRecursiveReferences(serviceNode, ProcessorNode.class); + final List reportingTasks = findRecursiveReferences(serviceNode, ReportingTaskNode.class); + + // verify that we can start all components (that are not disabled) before doing anything + for ( final ProcessorNode node : processors ) { + if ( node.getScheduledState() != ScheduledState.DISABLED ) { + node.verifyCanStart(); + } + } + for ( final ReportingTaskNode node : reportingTasks ) { + if ( node.getScheduledState() != ScheduledState.DISABLED ) { + node.verifyCanStart(); + } + } + + // start all of the components that are not disabled + for ( final ProcessorNode node : processors ) { + if ( node.getScheduledState() != ScheduledState.DISABLED ) { + node.getProcessGroup().startProcessor(node); + } + } + for ( final ReportingTaskNode node : reportingTasks ) { + if ( node.getScheduledState() != ScheduledState.DISABLED ) { + processScheduler.schedule(node); + } + } + } + + @Override + public void unscheduleReferencingComponents(final ControllerServiceNode serviceNode) { + // find all of the schedulable components (processors, reporting tasks) that refer to this Controller Service, + // or a service that references this controller service, etc. + final List processors = findRecursiveReferences(serviceNode, ProcessorNode.class); + final List reportingTasks = findRecursiveReferences(serviceNode, ReportingTaskNode.class); + + // verify that we can stop all components (that are running) before doing anything + for ( final ProcessorNode node : processors ) { + if ( node.getScheduledState() == ScheduledState.RUNNING ) { + node.verifyCanStop(); + } + } + for ( final ReportingTaskNode node : reportingTasks ) { + if ( node.getScheduledState() == ScheduledState.RUNNING ) { + node.verifyCanStop(); + } + } + + // stop all of the components that are running + for ( final ProcessorNode node : processors ) { + if ( node.getScheduledState() == ScheduledState.RUNNING ) { + node.getProcessGroup().stopProcessor(node); + } + } + for ( final ReportingTaskNode node : reportingTasks ) { + if ( node.getScheduledState() == ScheduledState.RUNNING ) { + processScheduler.unschedule(node); + } + } + } + @Override public void enableControllerService(final ControllerServiceNode serviceNode) { serviceNode.verifyCanEnable(); - - try (final NarCloseable x = NarCloseable.withNarLoader()) { - final ConfigurationContext configContext = new StandardConfigurationContext(serviceNode, this); - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnEnabled.class, serviceNode.getControllerServiceImplementation(), configContext); + processScheduler.enableControllerService(serviceNode); + } + + @Override + public void enableControllerServices(final Collection serviceNodes) { + final Set servicesToEnable = new HashSet<>(); + // Ensure that all nodes are already disabled + for ( final ControllerServiceNode serviceNode : serviceNodes ) { + final ControllerServiceState curState = serviceNode.getState(); + if ( ControllerServiceState.DISABLED.equals(curState) ) { + servicesToEnable.add(serviceNode); + } else { + logger.warn("Cannot enable {} because it is not disabled; current state is {}", serviceNode, curState); + } } - serviceNode.setDisabled(false); + // determine the order to load the services. We have to ensure that if service A references service B, then B + // is enabled first, and so on. + final Map idToNodeMap = new HashMap<>(); + for ( final ControllerServiceNode node : servicesToEnable ) { + idToNodeMap.put(node.getIdentifier(), node); + } + + // We can have many Controller Services dependent on one another. We can have many of these + // disparate lists of Controller Services that are dependent on one another. We refer to each + // of these as a branch. + final List> branches = determineEnablingOrder(idToNodeMap); + + if ( branches.isEmpty() ) { + logger.info("No Controller Services to enable"); + return; + } else { + logger.info("Will enable {} Controller Services", servicesToEnable.size()); + } + + // Mark all services that are configured to be enabled as 'ENABLING'. This allows Processors, reporting tasks + // to be valid so that they can be scheduled. + for ( final List branch : branches ) { + for ( final ControllerServiceNode nodeToEnable : branch ) { + nodeToEnable.setState(ControllerServiceState.ENABLING); + } + } + + final Set enabledNodes = Collections.synchronizedSet(new HashSet()); + final ExecutorService executor = Executors.newFixedThreadPool(Math.min(10, branches.size())); + for ( final List branch : branches ) { + final Runnable enableBranchRunnable = new Runnable() { + @Override + public void run() { + logger.debug("Enabling Controller Service Branch {}", branch); + + for ( final ControllerServiceNode serviceNode : branch ) { + try { + if ( !enabledNodes.contains(serviceNode) ) { + enabledNodes.add(serviceNode); + + logger.info("Enabling {}", serviceNode); + try { + processScheduler.enableControllerService(serviceNode); + } catch (final Exception e) { + logger.error("Failed to enable " + serviceNode + " due to " + e); + if ( logger.isDebugEnabled() ) { + logger.error("", e); + } + + if ( bulletinRepo != null ) { + bulletinRepo.addBulletin(BulletinFactory.createBulletin( + "Controller Service", Severity.ERROR.name(), "Could not start " + serviceNode + " due to " + e)); + } + } + } + + // wait for service to finish enabling. + while ( ControllerServiceState.ENABLING.equals(serviceNode.getState()) ) { + try { + Thread.sleep(100L); + } catch (final InterruptedException ie) {} + } + + logger.info("State for {} is now {}", serviceNode, serviceNode.getState()); + } catch (final Exception e) { + logger.error("Failed to enable {} due to {}", serviceNode, e.toString()); + if ( logger.isDebugEnabled() ) { + logger.error("", e); + } + } + } + } + }; + + executor.submit(enableBranchRunnable); + } + + executor.shutdown(); } + static List> determineEnablingOrder(final Map serviceNodeMap) { + final List> orderedNodeLists = new ArrayList<>(); + + for ( final ControllerServiceNode node : serviceNodeMap.values() ) { + if ( orderedNodeLists.contains(node) ) { + continue; // this node is already in the list. + } + + final List branch = new ArrayList<>(); + determineEnablingOrder(serviceNodeMap, node, branch, new HashSet()); + orderedNodeLists.add(branch); + } + + return orderedNodeLists; + } + + + private static void determineEnablingOrder(final Map serviceNodeMap, final ControllerServiceNode contextNode, final List orderedNodes, final Set visited) { + if ( visited.contains(contextNode) ) { + return; + } + + for ( final Map.Entry entry : contextNode.getProperties().entrySet() ) { + if ( entry.getKey().getControllerServiceDefinition() != null ) { + final String referencedServiceId = entry.getValue(); + if ( referencedServiceId != null ) { + final ControllerServiceNode referencedNode = serviceNodeMap.get(referencedServiceId); + if ( !orderedNodes.contains(referencedNode) ) { + visited.add(contextNode); + determineEnablingOrder(serviceNodeMap, referencedNode, orderedNodes, visited); + } + } + } + } + + if ( !orderedNodes.contains(contextNode) ) { + orderedNodes.add(contextNode); + } + } + + @Override public void disableControllerService(final ControllerServiceNode serviceNode) { serviceNode.verifyCanDisable(); - - // We must set the service to disabled before we invoke the OnDisabled methods because the service node - // can throw Exceptions if we attempt to disable the service while it's known to be in use. - serviceNode.setDisabled(true); - - try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnDisabled.class, serviceNode.getControllerServiceImplementation()); - } + processScheduler.disableControllerService(serviceNode); } @Override @@ -207,9 +453,15 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi @Override public boolean isControllerServiceEnabled(final String serviceIdentifier) { final ControllerServiceNode node = controllerServices.get(serviceIdentifier); - return (node == null) ? false : !node.isDisabled(); + return (node == null) ? false : (ControllerServiceState.ENABLED == node.getState()); } + @Override + public boolean isControllerServiceEnabling(final String serviceIdentifier) { + final ControllerServiceNode node = controllerServices.get(serviceIdentifier); + return (node == null) ? false : (ControllerServiceState.ENABLING == node.getState()); + } + @Override public ControllerServiceNode getControllerServiceNode(final String serviceIdentifier) { return controllerServices.get(serviceIdentifier); @@ -228,6 +480,11 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi } @Override + public String getControllerServiceName(final String serviceIdentifier) { + final ControllerServiceNode node = getControllerServiceNode(serviceIdentifier); + return node == null ? null : node.getName(); + } + public void removeControllerService(final ControllerServiceNode serviceNode) { final ControllerServiceNode existing = controllerServices.get(serviceNode.getIdentifier()); if ( existing == null || existing != serviceNode ) { @@ -241,6 +498,139 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, serviceNode.getControllerServiceImplementation(), configurationContext); } + for ( final Map.Entry entry : serviceNode.getProperties().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 referencedNode = getControllerServiceNode(value); + if ( referencedNode != null ) { + referencedNode.removeReference(serviceNode); + } + } + } + } + controllerServices.remove(serviceNode.getIdentifier()); } + + @Override + public Set getAllControllerServices() { + return new HashSet<>(controllerServices.values()); + } + + + /** + * Returns a List of all components that reference the given referencedNode (either directly or indirectly through + * another service) that are also of the given componentType. The list that is returned is in the order in which they will + * need to be 'activated' (enabled/started). + * @param referencedNode + * @param componentType + * @return + */ + private List findRecursiveReferences(final ControllerServiceNode referencedNode, final Class componentType) { + final List references = new ArrayList<>(); + + for ( final ConfiguredComponent referencingComponent : referencedNode.getReferences().getReferencingComponents() ) { + if ( componentType.isAssignableFrom(referencingComponent.getClass()) ) { + references.add(componentType.cast(referencingComponent)); + } + + if ( referencingComponent instanceof ControllerServiceNode ) { + final ControllerServiceNode referencingNode = (ControllerServiceNode) referencingComponent; + + // find components recursively that depend on referencingNode. + final List recursive = findRecursiveReferences(referencingNode, componentType); + + // For anything that depends on referencing node, we want to add it to the list, but we know + // that it must come after the referencing node, so we first remove any existing occurrence. + references.removeAll(recursive); + references.addAll(recursive); + } + } + + return references; + } + + + @Override + public void enableReferencingServices(final ControllerServiceNode serviceNode) { + final List recursiveReferences = findRecursiveReferences(serviceNode, ControllerServiceNode.class); + enableReferencingServices(serviceNode, recursiveReferences); + } + + private void enableReferencingServices(final ControllerServiceNode serviceNode, final List recursiveReferences) { + if ( serviceNode.getState() != ControllerServiceState.ENABLED && serviceNode.getState() != ControllerServiceState.ENABLING ) { + serviceNode.verifyCanEnable(new HashSet<>(recursiveReferences)); + } + + final Set ifEnabled = new HashSet<>(); + final List toEnable = findRecursiveReferences(serviceNode, ControllerServiceNode.class); + for ( final ControllerServiceNode nodeToEnable : toEnable ) { + final ControllerServiceState state = nodeToEnable.getState(); + if ( state != ControllerServiceState.ENABLED && state != ControllerServiceState.ENABLING ) { + nodeToEnable.verifyCanEnable(ifEnabled); + ifEnabled.add(nodeToEnable); + } + } + + for ( final ControllerServiceNode nodeToEnable : toEnable ) { + final ControllerServiceState state = nodeToEnable.getState(); + if ( state != ControllerServiceState.ENABLED && state != ControllerServiceState.ENABLING ) { + enableControllerService(nodeToEnable); + } + } + } + + @Override + public void verifyCanEnableReferencingServices(final ControllerServiceNode serviceNode) { + final List referencingServices = findRecursiveReferences(serviceNode, ControllerServiceNode.class); + final Set referencingServiceSet = new HashSet<>(referencingServices); + + for ( final ControllerServiceNode referencingService : referencingServices ) { + referencingService.verifyCanEnable(referencingServiceSet); + } + } + + @Override + public void verifyCanScheduleReferencingComponents(final ControllerServiceNode serviceNode) { + final List referencingServices = findRecursiveReferences(serviceNode, ControllerServiceNode.class); + final List referencingReportingTasks = findRecursiveReferences(serviceNode, ReportingTaskNode.class); + final List referencingProcessors = findRecursiveReferences(serviceNode, ProcessorNode.class); + + final Set referencingServiceSet = new HashSet<>(referencingServices); + + for ( final ReportingTaskNode taskNode : referencingReportingTasks ) { + if ( taskNode.getScheduledState() != ScheduledState.DISABLED ) { + taskNode.verifyCanStart(referencingServiceSet); + } + } + + for ( final ProcessorNode procNode : referencingProcessors ) { + if ( procNode.getScheduledState() != ScheduledState.DISABLED ) { + procNode.verifyCanStart(referencingServiceSet); + } + } + } + + @Override + public void verifyCanDisableReferencingServices(final ControllerServiceNode serviceNode) { + // Get a list of all Controller Services that need to be disabled, in the order that they need to be + // disabled. + final List toDisable = findRecursiveReferences(serviceNode, ControllerServiceNode.class); + final Set serviceSet = new HashSet<>(toDisable); + + for ( final ControllerServiceNode nodeToDisable : toDisable ) { + final ControllerServiceState state = nodeToDisable.getState(); + + if ( state != ControllerServiceState.DISABLED && state != ControllerServiceState.DISABLING ) { + nodeToDisable.verifyCanDisable(serviceSet); + } + } + } + + @Override + public void verifyCanStopReferencingComponents(final ControllerServiceNode serviceNode) { + // we can always stop referencing components + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java index a1c4984cc4..c470b99056 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java @@ -58,23 +58,28 @@ public class StandardControllerServiceReference implements ControllerServiceRefe } @Override - public Set getRunningReferences() { - final Set runningReferences = new HashSet<>(); + public Set getActiveReferences() { + final Set activeReferences = new HashSet<>(); final Set serviceNodes = new HashSet<>(); for (final ConfiguredComponent component : components) { if (component instanceof ControllerServiceNode) { serviceNodes.add((ControllerServiceNode) component); + + final ControllerServiceState state = ((ControllerServiceNode) component).getState(); + if ( state != ControllerServiceState.DISABLED ) { + activeReferences.add(component); + } } else if (isRunning(component)) { - runningReferences.add(component); + activeReferences.add(component); } } - runningReferences.addAll(getRunningIndirectReferences(serviceNodes)); - return runningReferences; + activeReferences.addAll(getActiveIndirectReferences(serviceNodes)); + return activeReferences; } - private Set getRunningIndirectReferences(final Set referencingServices) { + private Set getActiveIndirectReferences(final Set referencingServices) { if (referencingServices.isEmpty()) { return Collections.emptySet(); } @@ -92,7 +97,7 @@ public class StandardControllerServiceReference implements ControllerServiceRefe } } - references.addAll(getRunningIndirectReferences(serviceNodes)); + references.addAll(getActiveIndirectReferences(serviceNodes)); } return references; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java index b5b60c9bf8..5ecd22e657 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java @@ -107,6 +107,6 @@ public class ContinuallyRunConnectableTask implements Callable { return true; } - return true; + return false; // do not yield } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java index 9b7058101f..0c472c8fe3 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java @@ -19,15 +19,13 @@ package org.apache.nifi.controller.tasks; import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.scheduling.ScheduleState; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.nar.NarCloseable; +import org.apache.nifi.processor.SimpleProcessLogger; import org.apache.nifi.util.ReflectionUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class ReportingTaskWrapper implements Runnable { - private static final Logger logger = LoggerFactory.getLogger(ReportingTaskWrapper.class); - private final ReportingTaskNode taskNode; private final ScheduleState scheduleState; @@ -43,20 +41,23 @@ public class ReportingTaskWrapper implements Runnable { try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { taskNode.getReportingTask().onTrigger(taskNode.getReportingContext()); } catch (final Throwable t) { - logger.error("Error running task {} due to {}", taskNode.getReportingTask(), t.toString()); - if (logger.isDebugEnabled()) { - logger.error("", t); + final ComponentLog componentLog = new SimpleProcessLogger(taskNode.getIdentifier(), taskNode.getReportingTask()); + componentLog.error("Error running task {} due to {}", new Object[] {taskNode.getReportingTask(), t.toString()}); + if (componentLog.isDebugEnabled()) { + componentLog.error("", t); } } finally { - // if the processor is no longer scheduled to run and this is the last thread, - // invoke the OnStopped methods - if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) { - try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, org.apache.nifi.processor.annotation.OnStopped.class, taskNode.getReportingTask(), taskNode.getReportingContext()); + try { + // if the reporting task is no longer scheduled to run and this is the last thread, + // invoke the OnStopped methods + if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) { + try (final NarCloseable x = NarCloseable.withNarLoader()) { + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, org.apache.nifi.processor.annotation.OnStopped.class, taskNode.getReportingTask(), taskNode.getConfigurationContext()); + } } + } finally { + scheduleState.decrementActiveThreadCount(); } - - scheduleState.decrementActiveThreadCount(); } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java index 85755691f2..9a8ad28270 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java @@ -41,12 +41,14 @@ import javax.xml.validation.SchemaFactory; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.FlowFromDOMFactory; import org.apache.nifi.controller.Template; import org.apache.nifi.controller.exception.ProcessorInstantiationException; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.processor.Processor; import org.apache.nifi.util.DomUtils; import org.apache.nifi.web.api.dto.ConnectionDTO; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.FlowSnippetDTO; import org.apache.nifi.web.api.dto.FunnelDTO; import org.apache.nifi.web.api.dto.LabelDTO; @@ -58,6 +60,7 @@ import org.apache.nifi.web.api.dto.ProcessorDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupContentsDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; import org.apache.nifi.web.api.dto.TemplateDTO; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -250,6 +253,22 @@ public final class FingerprintFactory { // root group final Element rootGroupElem = (Element) DomUtils.getChildNodesByTagName(flowControllerElem, "rootGroup").item(0); addProcessGroupFingerprint(builder, rootGroupElem, controller); + + final Element controllerServicesElem = DomUtils.getChild(flowControllerElem, "controllerServices"); + if ( controllerServicesElem != null ) { + for ( final Element serviceElem : DomUtils.getChildElementsByTagName(controllerServicesElem, "controllerService") ) { + addControllerServiceFingerprint(builder, serviceElem); + } + } + + final Element reportingTasksElem = DomUtils.getChild(flowControllerElem, "reportingTasks"); + if ( reportingTasksElem != null ) { + for ( final Element taskElem : DomUtils.getChildElementsByTagName(reportingTasksElem, "reportingTask") ) { + addReportingTaskFingerprint(builder, taskElem); + } + } + + return builder; } @@ -832,6 +851,66 @@ public final class FingerprintFactory { builder.append(funnel.getId()); return builder; } + + private void addControllerServiceFingerprint(final StringBuilder builder, final Element controllerServiceElem) { + final ControllerServiceDTO dto = FlowFromDOMFactory.getControllerService(controllerServiceElem, encryptor); + addControllerServiceFingerprint(builder, dto); + } + + private void addControllerServiceFingerprint(final StringBuilder builder, final ControllerServiceDTO dto) { + builder.append(dto.getId()); + builder.append(dto.getType()); + builder.append(dto.getName()); + builder.append(dto.getComments()); + builder.append(dto.getAnnotationData()); + + final Map properties = dto.getProperties(); + if (properties == null) { + builder.append("NO_PROPERTIES"); + } else { + final SortedMap sortedProps = new TreeMap<>(properties); + for (final Map.Entry entry : sortedProps.entrySet()) { + final String propName = entry.getKey(); + final String propValue = entry.getValue(); + if (propValue == null) { + continue; + } + + builder.append(propName).append("=").append(propValue); + } + } + } + + private void addReportingTaskFingerprint(final StringBuilder builder, final Element element) { + final ReportingTaskDTO dto = FlowFromDOMFactory.getReportingTask(element, encryptor); + addReportingTaskFingerprint(builder, dto); + } + + private void addReportingTaskFingerprint(final StringBuilder builder, final ReportingTaskDTO dto) { + builder.append(dto.getId()); + builder.append(dto.getType()); + builder.append(dto.getName()); + builder.append(dto.getComments()); + builder.append(dto.getSchedulingPeriod()); + builder.append(dto.getSchedulingStrategy()); + builder.append(dto.getAnnotationData()); + + final Map properties = dto.getProperties(); + if (properties == null) { + builder.append("NO_PROPERTIES"); + } else { + final SortedMap sortedProps = new TreeMap<>(properties); + for (final Map.Entry entry : sortedProps.entrySet()) { + final String propName = entry.getKey(); + final String propValue = entry.getValue(); + if (propValue == null) { + continue; + } + + builder.append(propName).append("=").append(propValue); + } + } + } private Comparator getIdsComparator() { return new Comparator() { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java index 3cd5853265..6a26d09db8 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java @@ -30,8 +30,13 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.nifi.annotation.lifecycle.OnRemoved; import org.apache.nifi.annotation.lifecycle.OnShutdown; +import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; @@ -43,9 +48,11 @@ import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.Snippet; -import org.apache.nifi.controller.exception.ProcessorLifeCycleException; +import org.apache.nifi.controller.exception.ComponentLifeCycleException; import org.apache.nifi.controller.label.Label; +import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.logging.LogRepositoryFactory; import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.processor.StandardProcessContext; @@ -53,11 +60,6 @@ import org.apache.nifi.remote.RemoteGroupPort; import org.apache.nifi.remote.RootGroupPort; import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.ReflectionUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.lang3.builder.HashCodeBuilder; -import org.apache.commons.lang3.builder.ToStringBuilder; -import org.apache.commons.lang3.builder.ToStringStyle; -import org.apache.nifi.encrypt.StringEncryptor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -329,7 +331,8 @@ public final class StandardProcessGroup implements ProcessGroup { private void shutdown(final ProcessGroup procGroup) { for (final ProcessorNode node : procGroup.getProcessors()) { try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, org.apache.nifi.processor.annotation.OnShutdown.class, node.getProcessor()); + final StandardProcessContext processContext = new StandardProcessContext(node, controllerServiceProvider, encryptor); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, org.apache.nifi.processor.annotation.OnShutdown.class, node.getProcessor(), processContext); } } @@ -671,9 +674,22 @@ public final class StandardProcessGroup implements ProcessGroup { final StandardProcessContext processContext = new StandardProcessContext(processor, controllerServiceProvider, encryptor); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, org.apache.nifi.processor.annotation.OnRemoved.class, processor.getProcessor(), processContext); } catch (final Exception e) { - throw new ProcessorLifeCycleException("Failed to invoke 'OnRemoved' methods of " + processor, e); + throw new ComponentLifeCycleException("Failed to invoke 'OnRemoved' methods of " + processor, e); } + for ( final Map.Entry entry : processor.getProperties().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(processor); + } + } + } + } + processors.remove(id); LogRepositoryFactory.getRepository(processor.getIdentifier()).removeAllObservers(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java index 4f3afaf598..8957314f9c 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java @@ -19,15 +19,12 @@ package org.apache.nifi.persistence; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.util.List; import org.apache.nifi.cluster.protocol.DataFlow; import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.FlowSerializationException; import org.apache.nifi.controller.FlowSynchronizationException; -import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.UninheritableFlowException; -import org.apache.nifi.controller.service.ControllerServiceNode; /** * Interface to define service methods for FlowController configuration. @@ -110,26 +107,4 @@ public interface FlowConfigurationDAO { */ void save(FlowController flow, boolean archive) throws IOException; - /** - * Instantiates and schedules all controller tasks from the file used in the - * constructor - * - * @param controller - * @return - * @throws java.io.IOException - * @returns all of the ReportingTasks that were instantiated & scheduled - */ - List loadReportingTasks(FlowController controller) throws IOException; - - /** - * Instantiates all controller services from the file used in the - * constructor - * - * @param controller - * @return - * @throws java.io.IOException - * @returns all of the ReportingTasks that were instantiated & scheduled - */ - List loadControllerServices(FlowController controller) throws IOException; - } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java index c11aa72c78..b93ae8a678 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java @@ -21,72 +21,36 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.net.URL; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardOpenOption; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; import java.util.zip.GZIPInputStream; import java.util.zip.GZIPOutputStream; -import javax.xml.XMLConstants; -import javax.xml.parsers.DocumentBuilder; -import javax.xml.parsers.DocumentBuilderFactory; -import javax.xml.parsers.ParserConfigurationException; -import javax.xml.transform.dom.DOMSource; -import javax.xml.validation.Schema; -import javax.xml.validation.SchemaFactory; -import javax.xml.validation.Validator; - import org.apache.nifi.cluster.protocol.DataFlow; -import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.FlowSerializationException; import org.apache.nifi.controller.FlowSynchronizationException; import org.apache.nifi.controller.FlowSynchronizer; -import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.StandardFlowSerializer; import org.apache.nifi.controller.StandardFlowSynchronizer; import org.apache.nifi.controller.UninheritableFlowException; -import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException; -import org.apache.nifi.controller.reporting.StandardReportingInitializationContext; -import org.apache.nifi.controller.service.ControllerServiceLoader; -import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.encrypt.StringEncryptor; -import org.apache.nifi.util.file.FileUtils; -import org.apache.nifi.nar.NarCloseable; -import org.apache.nifi.reporting.InitializationException; -import org.apache.nifi.reporting.ReportingInitializationContext; -import org.apache.nifi.reporting.ReportingTask; -import org.apache.nifi.scheduling.SchedulingStrategy; -import org.apache.nifi.util.DomUtils; import org.apache.nifi.util.NiFiProperties; - +import org.apache.nifi.util.file.FileUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.w3c.dom.DOMException; -import org.w3c.dom.Document; -import org.w3c.dom.Element; -import org.w3c.dom.NodeList; -import org.xml.sax.SAXException; -import org.xml.sax.SAXParseException; public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationDAO { public static final String CONFIGURATION_ARCHIVE_DIR_KEY = "nifi.flow.configuration.archive.dir"; private final Path flowXmlPath; - private final Path taskConfigXmlPath; - private final ControllerServiceLoader servicerLoader; private final StringEncryptor encryptor; private static final Logger LOG = LoggerFactory.getLogger(StandardXMLFlowConfigurationDAO.class); - public StandardXMLFlowConfigurationDAO(final Path flowXml, final Path taskConfigXml, final Path serviceConfigXml, final StringEncryptor encryptor) throws IOException { + public StandardXMLFlowConfigurationDAO(final Path flowXml, final StringEncryptor encryptor) throws IOException { final File flowXmlFile = flowXml.toFile(); if (!flowXmlFile.exists()) { Files.createDirectories(flowXml.getParent()); @@ -96,14 +60,7 @@ public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationD throw new IOException(flowXml + " exists but you have insufficient read/write privileges"); } - final File taskConfigXmlFile = Objects.requireNonNull(taskConfigXml).toFile(); - if ((!taskConfigXmlFile.exists() || !taskConfigXmlFile.canRead())) { - throw new IOException(taskConfigXml + " does not appear to exist or cannot be read. Cannot load configuration."); - } - this.flowXmlPath = flowXml; - this.taskConfigXmlPath = taskConfigXml; - this.servicerLoader = new ControllerServiceLoader(serviceConfigXml); this.encryptor = encryptor; } @@ -198,148 +155,4 @@ public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationD } } - @Override - public List loadReportingTasks(final FlowController controller) { - final List tasks = new ArrayList<>(); - if (taskConfigXmlPath == null) { - LOG.info("No reporting tasks to start"); - return tasks; - } - - try { - final URL schemaUrl = getClass().getResource("/ReportingTaskConfiguration.xsd"); - final Document document = parse(taskConfigXmlPath.toFile(), schemaUrl); - - final NodeList tasksNodes = document.getElementsByTagName("tasks"); - final Element tasksElement = (Element) tasksNodes.item(0); - - for (final Element taskElement : DomUtils.getChildElementsByTagName(tasksElement, "task")) { - //add global properties common to all tasks - Map properties = new HashMap<>(); - - //get properties for the specific reporting task - id, name, class, - //and schedulingPeriod must be set - final String taskId = DomUtils.getChild(taskElement, "id").getTextContent().trim(); - final String taskName = DomUtils.getChild(taskElement, "name").getTextContent().trim(); - - final List schedulingStrategyNodeList = DomUtils.getChildElementsByTagName(taskElement, "schedulingStrategy"); - String schedulingStrategyValue = SchedulingStrategy.TIMER_DRIVEN.name(); - if (schedulingStrategyNodeList.size() == 1) { - final String specifiedValue = schedulingStrategyNodeList.get(0).getTextContent(); - - try { - schedulingStrategyValue = SchedulingStrategy.valueOf(specifiedValue).name(); - } catch (final Exception e) { - throw new RuntimeException("Cannot start Reporting Task with id " + taskId + " because its Scheduling Strategy does not have a valid value", e); - } - } - - final SchedulingStrategy schedulingStrategy = SchedulingStrategy.valueOf(schedulingStrategyValue); - final String taskSchedulingPeriod = DomUtils.getChild(taskElement, "schedulingPeriod").getTextContent().trim(); - final String taskClass = DomUtils.getChild(taskElement, "class").getTextContent().trim(); - - //optional task-specific properties - for (final Element optionalProperty : DomUtils.getChildElementsByTagName(taskElement, "property")) { - final String name = optionalProperty.getAttribute("name"); - final String value = optionalProperty.getTextContent().trim(); - properties.put(name, value); - } - - //set the class to be used for the configured reporting task - final ReportingTaskNode reportingTaskNode; - try { - reportingTaskNode = controller.createReportingTask(taskClass, taskId); - } catch (final ReportingTaskInstantiationException e) { - LOG.error("Unable to load reporting task {} due to {}", new Object[]{taskId, e}); - if (LOG.isDebugEnabled()) { - LOG.error("", e); - } - continue; - } - - reportingTaskNode.setName(taskName); - reportingTaskNode.setScheduldingPeriod(taskSchedulingPeriod); - reportingTaskNode.setSchedulingStrategy(schedulingStrategy); - - final ReportingTask reportingTask = reportingTaskNode.getReportingTask(); - - final ReportingInitializationContext config = new StandardReportingInitializationContext(taskId, taskName, schedulingStrategy, taskSchedulingPeriod, controller); - reportingTask.initialize(config); - - final Map resolvedProps; - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { - resolvedProps = new HashMap<>(); - for (final Map.Entry entry : properties.entrySet()) { - final PropertyDescriptor descriptor = reportingTask.getPropertyDescriptor(entry.getKey()); - resolvedProps.put(descriptor, entry.getValue()); - } - } - - for (final Map.Entry entry : resolvedProps.entrySet()) { - reportingTaskNode.setProperty(entry.getKey().getName(), entry.getValue()); - } - - tasks.add(reportingTaskNode); - controller.startReportingTask(reportingTaskNode); - } - } catch (final SAXException | ParserConfigurationException | IOException | DOMException | NumberFormatException | InitializationException t) { - LOG.error("Unable to load reporting tasks from {} due to {}", new Object[]{taskConfigXmlPath, t}); - if (LOG.isDebugEnabled()) { - LOG.error("", t); - } - } - - return tasks; - } - - @Override - public List loadControllerServices(final FlowController controller) throws IOException { - return servicerLoader.loadControllerServices(controller); - } - - private Document parse(final File xmlFile, final URL schemaUrl) throws SAXException, ParserConfigurationException, IOException { - final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI); - final Schema schema = schemaFactory.newSchema(schemaUrl); - final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance(); - docFactory.setSchema(schema); - final DocumentBuilder builder = docFactory.newDocumentBuilder(); - - builder.setErrorHandler(new org.xml.sax.ErrorHandler() { - @Override - public void fatalError(final SAXParseException err) throws SAXException { - LOG.error("Config file line " + err.getLineNumber() + ", col " + err.getColumnNumber() + ", uri " + err.getSystemId() + " :message: " + err.getMessage()); - if (LOG.isDebugEnabled()) { - LOG.error("Error Stack Dump", err); - } - throw err; - } - - @Override - public void error(final SAXParseException err) throws SAXParseException { - LOG.error("Config file line " + err.getLineNumber() + ", col " + err.getColumnNumber() + ", uri " + err.getSystemId() + " :message: " + err.getMessage()); - if (LOG.isDebugEnabled()) { - LOG.error("Error Stack Dump", err); - } - throw err; - } - - @Override - public void warning(final SAXParseException err) throws SAXParseException { - LOG.warn(" Config file line " + err.getLineNumber() + ", uri " + err.getSystemId() + " : message : " + err.getMessage()); - if (LOG.isDebugEnabled()) { - LOG.warn("Warning stack dump", err); - } - throw err; - } - }); - - // build the docuemnt - final Document document = builder.parse(xmlFile); - - // ensure schema compliance - final Validator validator = schema.newValidator(); - validator.validate(new DOMSource(document)); - - return document; - } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java index 88f1790c22..0a345a095f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java @@ -28,16 +28,16 @@ public class SimpleProcessLogger implements ProcessorLog { private final Logger logger; private final LogRepository logRepository; - private final Processor processor; + private final Object component; - public SimpleProcessLogger(final String processorId, final Processor processor) { - this.logger = LoggerFactory.getLogger(processor.getClass()); - this.logRepository = LogRepositoryFactory.getRepository(processorId); - this.processor = processor; + public SimpleProcessLogger(final String componentId, final Object component) { + this.logger = LoggerFactory.getLogger(component.getClass()); + this.logRepository = LogRepositoryFactory.getRepository(componentId); + this.component = component; } private Object[] addProcessor(final Object[] originalArgs) { - return prependToArgs(originalArgs, processor); + return prependToArgs(originalArgs, component); } private Object[] prependToArgs(final Object[] originalArgs, final Object... toAdd) { @@ -63,7 +63,7 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void warn(final String msg, final Throwable t) { - warn("{} " + msg, new Object[]{processor}, t); + warn("{} " + msg, new Object[]{component}, t); } @Override @@ -93,15 +93,15 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void warn(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; - logger.warn(msg, processor); + final Object[] os = {component}; + logger.warn(msg, component); logRepository.addLogMessage(LogLevel.WARN, msg, os); } @Override public void trace(String msg, Throwable t) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.trace(msg, os, t); logRepository.addLogMessage(LogLevel.TRACE, msg, os, t); } @@ -117,7 +117,7 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void trace(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.trace(msg, os); logRepository.addLogMessage(LogLevel.TRACE, msg, os); } @@ -160,7 +160,7 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void info(String msg, Throwable t) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.info(msg, os); if (logger.isDebugEnabled()) { @@ -181,7 +181,7 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void info(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.info(msg, os); logRepository.addLogMessage(LogLevel.INFO, msg, os); @@ -207,7 +207,7 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void error(String msg, Throwable t) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.error(msg, os, t); if (logger.isDebugEnabled()) { @@ -231,7 +231,7 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void error(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.error(msg, os); logRepository.addLogMessage(LogLevel.ERROR, msg, os); @@ -239,11 +239,11 @@ public class SimpleProcessLogger implements ProcessorLog { private Object[] addProcessorAndThrowable(final Object[] os, final Throwable t) { final Object[] modifiedArgs = new Object[os.length + 2]; - modifiedArgs[0] = processor.toString(); + modifiedArgs[0] = component.toString(); for (int i = 0; i < os.length; i++) { modifiedArgs[i + 1] = os[i]; } - modifiedArgs[modifiedArgs.length - 1] = t.toString(); + modifiedArgs[modifiedArgs.length - 1] = (t == null) ? "" : t.toString(); return modifiedArgs; } @@ -263,7 +263,7 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void debug(String msg, Throwable t) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.debug(msg, os, t); logRepository.addLogMessage(LogLevel.DEBUG, msg, os, t); @@ -298,7 +298,7 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void debug(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.debug(msg, os); logRepository.addLogMessage(LogLevel.DEBUG, msg, os); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java index cd0d31c465..d14a459099 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java @@ -141,6 +141,11 @@ public class StandardProcessContext implements ProcessContext, ControllerService return controllerServiceProvider.isControllerServiceEnabled(serviceIdentifier); } + @Override + public boolean isControllerServiceEnabling(final String serviceIdentifier) { + return controllerServiceProvider.isControllerServiceEnabling(serviceIdentifier); + } + @Override public ControllerServiceLookup getControllerServiceLookup() { return this; @@ -170,4 +175,9 @@ public class StandardProcessContext implements ProcessContext, ControllerService return set; } + @Override + public String getControllerServiceName(final String serviceIdentifier) { + return controllerServiceProvider.getControllerServiceName(serviceIdentifier); + } + } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java index ac585044cd..c37a80d6bb 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java @@ -25,6 +25,7 @@ import org.apache.nifi.controller.ControllerServiceLookup; import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.controller.service.ControllerServiceState; public class StandardSchedulingContext implements SchedulingContext { @@ -45,8 +46,8 @@ public class StandardSchedulingContext implements SchedulingContext { throw new IllegalArgumentException("Cannot lease Controller Service because no Controller Service exists with identifier " + identifier); } - if (serviceNode.isDisabled()) { - throw new IllegalStateException("Cannot lease Controller Service because Controller Service " + serviceNode.getProxiedControllerService() + " is currently disabled"); + if ( serviceNode.getState() != ControllerServiceState.ENABLED ) { + throw new IllegalStateException("Cannot lease Controller Service because Controller Service " + serviceNode.getProxiedControllerService() + " is not currently enabled"); } if (!serviceNode.isValid()) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java index 99322be30c..c51cb9aa25 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java @@ -20,6 +20,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.nifi.attribute.expression.language.PreparedQuery; import org.apache.nifi.attribute.expression.language.Query; @@ -41,11 +42,17 @@ public class StandardValidationContext implements ValidationContext { private final Map preparedQueries; private final Map expressionLanguageSupported; private final String annotationData; + private final Set serviceIdentifiersToNotValidate; public StandardValidationContext(final ControllerServiceProvider controllerServiceProvider, final Map properties, final String annotationData) { + this(controllerServiceProvider, Collections.emptySet(), properties, annotationData); + } + + public StandardValidationContext(final ControllerServiceProvider controllerServiceProvider, final Set serviceIdentifiersToNotValidate, final Map properties, final String annotationData) { this.controllerServiceProvider = controllerServiceProvider; this.properties = new HashMap<>(properties); this.annotationData = annotationData; + this.serviceIdentifiersToNotValidate = serviceIdentifiersToNotValidate; preparedQueries = new HashMap<>(properties.size()); for (final Map.Entry entry : properties.entrySet()) { @@ -101,6 +108,11 @@ public class StandardValidationContext implements ValidationContext { public ControllerServiceLookup getControllerServiceLookup() { return controllerServiceProvider; } + + @Override + public boolean isValidationRequired(final ControllerService service) { + return !serviceIdentifiersToNotValidate.contains(service.getIdentifier()); + } @Override public boolean isExpressionLanguagePresent(final String value) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContextFactory.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContextFactory.java index e172f935ea..c3df987032 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContextFactory.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContextFactory.java @@ -17,6 +17,7 @@ package org.apache.nifi.processor; import java.util.Map; +import java.util.Set; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; @@ -36,4 +37,8 @@ public class StandardValidationContextFactory implements ValidationContextFactor return new StandardValidationContext(serviceProvider, properties, annotationData); } + @Override + public ValidationContext newValidationContext(final Set serviceIdentifiersToNotValidate, final Map properties, final String annotationData) { + return new StandardValidationContext(serviceProvider, serviceIdentifiersToNotValidate, properties, annotationData); + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/DomUtils.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/DomUtils.java index c8b65731de..da4f04deb6 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/DomUtils.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/DomUtils.java @@ -25,6 +25,16 @@ import org.w3c.dom.NodeList; public class DomUtils { + public static String getChildText(final Element element, final String tagName) { + final Element childElement = getChild(element, tagName); + if ( childElement == null ) { + return null; + } + + final String text = childElement.getTextContent(); + return (text == null) ? null : text.trim(); + } + public static Element getChild(final Element element, final String tagName) { final List children = getChildElementsByTagName(element, tagName); if (children.isEmpty()) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ControllerServiceConfiguration.xsd b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ControllerServiceConfiguration.xsd deleted file mode 100644 index d3efed19b6..0000000000 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ControllerServiceConfiguration.xsd +++ /dev/null @@ -1,61 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd index 1e6c25c8a1..00c71ac4ad 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd @@ -28,6 +28,10 @@ + + + + @@ -58,11 +62,6 @@ IFF schedulingStrategy is EVENT_DRIVEN --> - @@ -332,4 +331,44 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ReportingTaskConfiguration.xsd b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ReportingTaskConfiguration.xsd deleted file mode 100644 index dcf10904a0..0000000000 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ReportingTaskConfiguration.xsd +++ /dev/null @@ -1,87 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java new file mode 100644 index 0000000000..7fef7065ff --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java @@ -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.controller.service; + +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.controller.StandardFlowServiceTest; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.nar.NarClassLoaders; +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.util.NiFiProperties; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +public class StandardControllerServiceProviderTest { + + private ControllerService proxied; + private ControllerService implementation; + + @BeforeClass + public static void setupSuite() throws Exception { + System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, StandardFlowServiceTest.class.getResource("/conf/nifi.properties").getFile()); + NiFiProperties properties = NiFiProperties.getInstance(); + NarClassLoaders.load(properties); + ExtensionManager.discoverExtensions(); + } + + @Before + public void setup() throws Exception { + String id = "id"; + String clazz = "org.apache.nifi.controller.service.util.TestControllerService"; + ControllerServiceProvider provider = new StandardControllerServiceProvider(null, null); + ControllerServiceNode node = provider.createControllerService(clazz,id,true); + proxied = node.getProxiedControllerService(); + implementation = node.getControllerServiceImplementation(); + } + + @Test (expected=UnsupportedOperationException.class) + public void testCallProxiedOnPropertyModified() { + proxied.onPropertyModified(null, "oldValue", "newValue"); + } + + @Test + public void testCallImplementationOnPropertyModified() { + implementation.onPropertyModified(null, "oldValue", "newValue"); + } + + @Test (expected=UnsupportedOperationException.class) + public void testCallProxiedInitialized() throws InitializationException { + proxied.initialize(null); + } + + @Test + public void testCallImplementationInitialized() throws InitializationException { + implementation.initialize(null); + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java new file mode 100644 index 0000000000..3dc1752c9a --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java @@ -0,0 +1,385 @@ +/* + * 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.service; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +import org.apache.nifi.controller.ProcessScheduler; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.StandardProcessorNode; +import org.apache.nifi.controller.service.mock.DummyProcessor; +import org.apache.nifi.controller.service.mock.ServiceA; +import org.apache.nifi.controller.service.mock.ServiceB; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.processor.StandardProcessorInitializationContext; +import org.apache.nifi.processor.StandardValidationContextFactory; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +public class TestStandardControllerServiceProvider { + + private ProcessScheduler createScheduler() { + final ProcessScheduler scheduler = Mockito.mock(ProcessScheduler.class); + Mockito.doAnswer(new Answer() { + @Override + public Object answer(final InvocationOnMock invocation) throws Throwable { + final ControllerServiceNode node = (ControllerServiceNode) invocation.getArguments()[0]; + node.verifyCanEnable(); + node.setState(ControllerServiceState.ENABLED); + return null; + } + }).when(scheduler).enableControllerService(Mockito.any(ControllerServiceNode.class)); + + Mockito.doAnswer(new Answer() { + @Override + public Object answer(final InvocationOnMock invocation) throws Throwable { + final ControllerServiceNode node = (ControllerServiceNode) invocation.getArguments()[0]; + node.verifyCanDisable(); + node.setState(ControllerServiceState.DISABLED); + return null; + } + }).when(scheduler).disableControllerService(Mockito.any(ControllerServiceNode.class)); + + return scheduler; + } + + @Test + public void testDisableControllerService() { + final ProcessScheduler scheduler = createScheduler(); + final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + + final ControllerServiceNode serviceNode = provider.createControllerService(ServiceB.class.getName(), "B", false); + provider.enableControllerService(serviceNode); + provider.disableControllerService(serviceNode); + } + + @Test + public void testEnableDisableWithReference() { + final ProcessScheduler scheduler = createScheduler(); + final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + + final ControllerServiceNode serviceNodeB = provider.createControllerService(ServiceB.class.getName(), "B", false); + final ControllerServiceNode serviceNodeA = provider.createControllerService(ServiceA.class.getName(), "A", false); + + serviceNodeA.setProperty(ServiceA.OTHER_SERVICE.getName(), "B"); + + try { + provider.enableControllerService(serviceNodeA); + Assert.fail("Was able to enable Service A but Service B is disabled."); + } catch (final IllegalStateException expected) { + } + + provider.enableControllerService(serviceNodeB); + provider.enableControllerService(serviceNodeA); + + try { + provider.disableControllerService(serviceNodeB); + Assert.fail("Was able to disable Service B but Service A is enabled and references B"); + } catch (final IllegalStateException expected) { + } + + provider.disableControllerService(serviceNodeA); + provider.disableControllerService(serviceNodeB); + } + + + @Test + public void testEnableReferencingServicesGraph() { + final ProcessScheduler scheduler = createScheduler(); + final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + + // build a graph of controller services with dependencies as such: + // + // A -> B -> D + // C ---^----^ + // + // In other words, A references B, which references D. + // AND + // C references B and D. + // + // So we have to verify that if D is enabled, when we enable its referencing services, + // we enable C and B, even if we attempt to enable C before B... i.e., if we try to enable C, we cannot do so + // until B is first enabled so ensure that we enable B first. + + final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1", false); + final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceA.class.getName(), "2", false); + final ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3", false); + final ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4", false); + + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + serviceNode2.setProperty(ServiceA.OTHER_SERVICE.getName(), "4"); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE_2.getName(), "4"); + + provider.enableControllerService(serviceNode4); + provider.enableReferencingServices(serviceNode4); + + assertEquals(ControllerServiceState.ENABLED, serviceNode3.getState()); + assertEquals(ControllerServiceState.ENABLED, serviceNode2.getState()); + assertEquals(ControllerServiceState.ENABLED, serviceNode1.getState()); + } + + + @Test + public void testStartStopReferencingComponents() { + final ProcessScheduler scheduler = createScheduler(); + final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + + // build a graph of reporting tasks and controller services with dependencies as such: + // + // Processor P1 -> A -> B -> D + // Processor P2 -> C ---^----^ + // + // In other words, Processor P1 references Controller Service A, which references B, which references D. + // AND + // Processor P2 references Controller Service C, which references B and D. + // + // So we have to verify that if D is enabled, when we enable its referencing services, + // we enable C and B, even if we attempt to enable C before B... i.e., if we try to enable C, we cannot do so + // until B is first enabled so ensure that we enable B first. + + final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1", false); + final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceA.class.getName(), "2", false); + final ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3", false); + final ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4", false); + + final ProcessGroup mockProcessGroup = Mockito.mock(ProcessGroup.class); + Mockito.doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + final ProcessorNode procNode = (ProcessorNode) invocation.getArguments()[0]; + procNode.verifyCanStart(); + procNode.setScheduledState(ScheduledState.RUNNING); + return null; + } + }).when(mockProcessGroup).startProcessor(Mockito.any(ProcessorNode.class)); + + Mockito.doAnswer(new Answer() { + @Override + public Object answer(final InvocationOnMock invocation) throws Throwable { + final ProcessorNode procNode = (ProcessorNode) invocation.getArguments()[0]; + procNode.verifyCanStop(); + procNode.setScheduledState(ScheduledState.STOPPED); + return null; + } + }).when(mockProcessGroup).stopProcessor(Mockito.any(ProcessorNode.class)); + + final String id1 = UUID.randomUUID().toString(); + final ProcessorNode procNodeA = new StandardProcessorNode(new DummyProcessor(), id1, + new StandardValidationContextFactory(provider), scheduler, provider); + procNodeA.getProcessor().initialize(new StandardProcessorInitializationContext(id1, null, provider)); + procNodeA.setProperty(DummyProcessor.SERVICE.getName(), "1"); + procNodeA.setProcessGroup(mockProcessGroup); + + final String id2 = UUID.randomUUID().toString(); + final ProcessorNode procNodeB = new StandardProcessorNode(new DummyProcessor(),id2, + new StandardValidationContextFactory(provider), scheduler, provider); + procNodeB.getProcessor().initialize(new StandardProcessorInitializationContext(id2, null, provider)); + procNodeB.setProperty(DummyProcessor.SERVICE.getName(), "3"); + procNodeB.setProcessGroup(mockProcessGroup); + + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + serviceNode2.setProperty(ServiceA.OTHER_SERVICE.getName(), "4"); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE_2.getName(), "4"); + + provider.enableControllerService(serviceNode4); + provider.enableReferencingServices(serviceNode4); + provider.scheduleReferencingComponents(serviceNode4); + + assertEquals(ControllerServiceState.ENABLED, serviceNode3.getState()); + assertEquals(ControllerServiceState.ENABLED, serviceNode2.getState()); + assertEquals(ControllerServiceState.ENABLED, serviceNode1.getState()); + assertTrue(procNodeA.isRunning()); + assertTrue(procNodeB.isRunning()); + + // stop processors and verify results. + provider.unscheduleReferencingComponents(serviceNode4); + assertFalse(procNodeA.isRunning()); + assertFalse(procNodeB.isRunning()); + assertEquals(ControllerServiceState.ENABLED, serviceNode3.getState()); + assertEquals(ControllerServiceState.ENABLED, serviceNode2.getState()); + assertEquals(ControllerServiceState.ENABLED, serviceNode1.getState()); + + provider.disableReferencingServices(serviceNode4); + assertEquals(ControllerServiceState.DISABLED, serviceNode3.getState()); + assertEquals(ControllerServiceState.DISABLED, serviceNode2.getState()); + assertEquals(ControllerServiceState.DISABLED, serviceNode1.getState()); + assertEquals(ControllerServiceState.ENABLED, serviceNode4.getState()); + + provider.disableControllerService(serviceNode4); + assertEquals(ControllerServiceState.DISABLED, serviceNode4.getState()); + } + + + @Test + public void testOrderingOfServices() { + final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(null, null); + final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1", false); + final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceB.class.getName(), "2", false); + + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + + final Map nodeMap = new LinkedHashMap<>(); + nodeMap.put("1", serviceNode1); + nodeMap.put("2", serviceNode2); + + List> branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap); + assertEquals(2, branches.size()); + List ordered = branches.get(0); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode2); + assertTrue(ordered.get(1) == serviceNode1); + assertEquals(1, branches.get(1).size()); + assertTrue(branches.get(1).get(0) == serviceNode2); + + nodeMap.clear(); + nodeMap.put("2", serviceNode2); + nodeMap.put("1", serviceNode1); + + branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap); + assertEquals(2, branches.size()); + ordered = branches.get(1); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode2); + assertTrue(ordered.get(1) == serviceNode1); + assertEquals(1, branches.get(0).size()); + assertTrue(branches.get(0).get(0) == serviceNode2); + + // add circular dependency on self. + nodeMap.clear(); + serviceNode1.setProperty(ServiceA.OTHER_SERVICE_2.getName(), "1"); + nodeMap.put("1", serviceNode1); + nodeMap.put("2", serviceNode2); + + branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap); + assertEquals(2, branches.size()); + ordered = branches.get(0); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode2); + assertTrue(ordered.get(1) == serviceNode1); + + nodeMap.clear(); + nodeMap.put("2", serviceNode2); + nodeMap.put("1", serviceNode1); + branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap); + assertEquals(2, branches.size()); + ordered = branches.get(1); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode2); + assertTrue(ordered.get(1) == serviceNode1); + + // add circular dependency once removed. In this case, we won't actually be able to enable these because of the + // circular dependency because they will never be valid because they will always depend on a disabled service. + // But we want to ensure that the method returns successfully without throwing a StackOverflowException or anything + // like that. + nodeMap.clear(); + final ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3", false); + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "3"); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "1"); + nodeMap.put("1", serviceNode1); + nodeMap.put("3", serviceNode3); + branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap); + assertEquals(2, branches.size()); + ordered = branches.get(0); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode3); + assertTrue(ordered.get(1) == serviceNode1); + + nodeMap.clear(); + nodeMap.put("3", serviceNode3); + nodeMap.put("1", serviceNode1); + branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap); + assertEquals(2, branches.size()); + ordered = branches.get(1); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode3); + assertTrue(ordered.get(1) == serviceNode1); + + + // Add multiple completely disparate branches. + nodeMap.clear(); + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + final ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4", false); + final ControllerServiceNode serviceNode5 = provider.createControllerService(ServiceB.class.getName(), "5", false); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "4"); + nodeMap.put("1", serviceNode1); + nodeMap.put("2", serviceNode2); + nodeMap.put("3", serviceNode3); + nodeMap.put("4", serviceNode4); + nodeMap.put("5", serviceNode5); + + branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap); + assertEquals(5, branches.size()); + + ordered = branches.get(0); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode2); + assertTrue(ordered.get(1) == serviceNode1); + + assertEquals(1, branches.get(1).size()); + assertTrue(branches.get(1).get(0) == serviceNode2); + + ordered = branches.get(2); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode4); + assertTrue(ordered.get(1) == serviceNode3); + + assertEquals(1, branches.get(3).size()); + assertTrue(branches.get(3).get(0) == serviceNode4); + + assertEquals(1, branches.get(4).size()); + assertTrue(branches.get(4).get(0) == serviceNode5); + + // create 2 branches both dependent on the same service + nodeMap.clear(); + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + nodeMap.put("1", serviceNode1); + nodeMap.put("2", serviceNode2); + nodeMap.put("3", serviceNode3); + + branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap); + assertEquals(3, branches.size()); + + ordered = branches.get(0); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode2); + assertTrue(ordered.get(1) == serviceNode1); + + ordered = branches.get(1); + assertEquals(1, ordered.size()); + assertTrue(ordered.get(0) == serviceNode2); + + ordered = branches.get(2); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode2); + assertTrue(ordered.get(1) == serviceNode3); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/DummyProcessor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/DummyProcessor.java new file mode 100644 index 0000000000..615e172ebc --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/DummyProcessor.java @@ -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.service.mock; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; + +public class DummyProcessor extends AbstractProcessor { + + public static final PropertyDescriptor SERVICE = new PropertyDescriptor.Builder() + .name("Controller Service") + .identifiesControllerService(ControllerService.class) + .required(true) + .build(); + + + @Override + protected List getSupportedPropertyDescriptors() { + final List descriptors = new ArrayList<>(); + descriptors.add(SERVICE); + return descriptors; + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceA.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceA.java new file mode 100644 index 0000000000..4918468ea7 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceA.java @@ -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.service.mock; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.controller.ControllerService; + +public class ServiceA extends AbstractControllerService { + + public static final PropertyDescriptor OTHER_SERVICE = new PropertyDescriptor.Builder() + .name("Other Service") + .identifiesControllerService(ControllerService.class) + .required(true) + .build(); + + public static final PropertyDescriptor OTHER_SERVICE_2 = new PropertyDescriptor.Builder() + .name("Other Service 2") + .identifiesControllerService(ControllerService.class) + .required(false) + .build(); + + + @Override + protected List getSupportedPropertyDescriptors() { + final List descriptors = new ArrayList<>(); + descriptors.add(OTHER_SERVICE); + descriptors.add(OTHER_SERVICE_2); + return descriptors; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceB.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceB.java new file mode 100644 index 0000000000..070b156e3a --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceB.java @@ -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.controller.service.mock; + +import org.apache.nifi.controller.AbstractControllerService; + +public class ServiceB extends AbstractControllerService { + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/util/TestControllerService.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/util/TestControllerService.java new file mode 100644 index 0000000000..95200a0eeb --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/util/TestControllerService.java @@ -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.controller.service.util; + +import java.util.Collection; +import java.util.List; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.controller.ControllerServiceInitializationContext; +import org.apache.nifi.reporting.InitializationException; + +public class TestControllerService implements ControllerService { + + @Override + public Collection validate(ValidationContext context) { + return null; + } + + @Override + public PropertyDescriptor getPropertyDescriptor(String name) { + return null; + } + + @Override + public void onPropertyModified(PropertyDescriptor descriptor, + String oldValue, String newValue) { + } + + @Override + public List getPropertyDescriptors() { + return null; + } + + @Override + public String getIdentifier() { + return null; + } + + @Override + public void initialize(ControllerServiceInitializationContext context) + throws InitializationException { + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/processor/TestStandardPropertyValue.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/processor/TestStandardPropertyValue.java index 7390098dac..a0bf30d79c 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/processor/TestStandardPropertyValue.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/processor/TestStandardPropertyValue.java @@ -17,6 +17,7 @@ package org.apache.nifi.processor; import org.apache.nifi.processor.StandardPropertyValue; + import static org.junit.Assert.assertEquals; import java.util.Calendar; @@ -29,7 +30,6 @@ import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerServiceLookup; import org.apache.nifi.controller.repository.StandardFlowFileRecord; import org.apache.nifi.flowfile.FlowFile; - import org.junit.Test; public class TestStandardPropertyValue { @@ -163,5 +163,14 @@ public class TestStandardPropertyValue { return true; } + @Override + public String getControllerServiceName(String serviceIdentifier) { + return null; + } + + @Override + public boolean isControllerServiceEnabling(String serviceIdentifier) { + return false; + } } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.controller.ControllerService new file mode 100644 index 0000000000..bd6a4f3935 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.controller.ControllerService @@ -0,0 +1,15 @@ +# 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. +org.apache.nifi.controller.service.util.TestControllerService diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/.gitignore b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/.gitignore index ea8c4bf7f3..29546b567b 100755 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/.gitignore +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/.gitignore @@ -1 +1,2 @@ /target +/target/ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties index fd16cb5363..90b3cdd07f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties @@ -25,8 +25,6 @@ nifi.administrative.yield.duration=${nifi.administrative.yield.duration} nifi.bored.yield.duration=${nifi.bored.yield.duration} nifi.authority.provider.configuration.file=${nifi.authority.provider.configuration.file} -nifi.reporting.task.configuration.file=${nifi.reporting.task.configuration.file} -nifi.controller.service.configuration.file=${nifi.controller.service.configuration.file} nifi.templates.directory=${nifi.templates.directory} nifi.ui.banner.text=${nifi.ui.banner.text} nifi.ui.autorefresh.interval=${nifi.ui.autorefresh.interval} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/reporting-tasks.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/reporting-tasks.xml deleted file mode 100644 index 3f60b937ec..0000000000 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/reporting-tasks.xml +++ /dev/null @@ -1,49 +0,0 @@ - - - - NiFi - - status-report - Controller Status Report - org.apache.nifi.controller.ControllerStatusReportingTask - 3 mins - - - - - - - - \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java index 740e405b87..eec6ed58b5 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java @@ -171,6 +171,7 @@ public class StandardRemoteGroupPort extends RemoteGroupPort { this.targetRunning.set(false); final String message = String.format("%s failed to communicate with %s because the remote instance indicates that the port is not in a valid state", this, url); logger.error(message); + session.rollback(); remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message); return; } catch (final UnknownPortException e) { @@ -178,21 +179,24 @@ public class StandardRemoteGroupPort extends RemoteGroupPort { this.targetExists.set(false); final String message = String.format("%s failed to communicate with %s because the remote instance indicates that the port no longer exists", this, url); logger.error(message); + session.rollback(); remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message); return; } catch (final IOException e) { + context.yield(); final String message = String.format("%s failed to communicate with %s due to %s", this, url, e.toString()); logger.error(message); if ( logger.isDebugEnabled() ) { logger.error("", e); } + session.rollback(); remoteGroup.getEventReporter().reportEvent(Severity.ERROR, CATEGORY, message); - session.rollback(); return; } if ( transaction == null ) { logger.debug("{} Unable to create transaction to communicate with; all peers must be penalized, so yielding context", this); + session.rollback(); context.yield(); return; } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Component.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Component.java index 62dd68f6ac..7f62c92bd9 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Component.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Component.java @@ -27,5 +27,7 @@ public enum Component { ProcessGroup, RemoteProcessGroup, Funnel, - Connection; + Connection, + ControllerService, + ReportingTask; } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ProcessorDetails.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ExtensionDetails.java similarity index 94% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ProcessorDetails.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ExtensionDetails.java index ef0af7c29f..e557548e84 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ProcessorDetails.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ExtensionDetails.java @@ -19,7 +19,7 @@ package org.apache.nifi.action.component.details; /** * */ -public class ProcessorDetails extends ComponentDetails { +public class ExtensionDetails extends ComponentDetails { private String type; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletConfigurationRequestContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletConfigurationRequestContext.java new file mode 100644 index 0000000000..986ce4cbc4 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletConfigurationRequestContext.java @@ -0,0 +1,56 @@ +/* + * 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; + +import javax.servlet.http.HttpServletRequest; + +/** + * An implementation of the ConfigurationRequestContext that retrieves configuration + * from a HttpServletRequest instance. + */ +public class HttpServletConfigurationRequestContext extends HttpServletRequestContext implements NiFiWebConfigurationRequestContext { + + private static final String CLIENT_ID_PARAM = "clientId"; + private static final String REVISION_PARAM = "revision"; + + private final HttpServletRequest request; + + public HttpServletConfigurationRequestContext(final UiExtensionType extensionType, final HttpServletRequest request) { + super(extensionType, request); + this.request = request; + } + + /** + * @return the revision retrieved from the request parameters with keys + * equal to "clientId" and "revision". + */ + @Override + public Revision getRevision() { + final String revisionParamVal = request.getParameter(REVISION_PARAM); + Long revision; + try { + revision = Long.parseLong(revisionParamVal); + } catch (final Exception ex) { + revision = null; + } + + final String clientId = request.getParameter(CLIENT_ID_PARAM); + + return new Revision(revision, clientId); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletRequestContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletRequestContext.java new file mode 100644 index 0000000000..311fbc7fa4 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletRequestContext.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web; + +import java.security.cert.X509Certificate; +import javax.servlet.http.HttpServletRequest; + +/** + * An implementation of the ConfigurationRequestContext that retrieves configuration + * from a HttpServletRequest instance. + */ +public class HttpServletRequestContext implements NiFiWebRequestContext { + + private static final String ID_PARAM = "id"; + + private final UiExtensionType extensionType; + private final HttpServletRequest request; + + public HttpServletRequestContext(final UiExtensionType extensionType, final HttpServletRequest request) { + this.extensionType = extensionType; + this.request = request; + } + + @Override + public UiExtensionType getExtensionType() { + return extensionType; + } + + @Override + public String getProxiedEntitiesChain() { + String xProxiedEntitiesChain = request.getHeader("X-ProxiedEntitiesChain"); + final X509Certificate cert = extractClientCertificate(request); + if (cert != null) { + final String extractedPrincipal = extractPrincipal(cert); + final String formattedPrincipal = formatProxyDn(extractedPrincipal); + if (xProxiedEntitiesChain == null || xProxiedEntitiesChain.trim().isEmpty()) { + xProxiedEntitiesChain = formattedPrincipal; + } else { + xProxiedEntitiesChain += formattedPrincipal; + } + } + + return xProxiedEntitiesChain; + } + + /** + * @return the protocol scheme of the HttpServletRequest instance. + */ + @Override + public String getScheme() { + return request.getScheme(); + } + + /** + * @return the ID retrieved from the request parameter with key + * equal to "id". + */ + @Override + public String getId() { + return request.getParameter(ID_PARAM); + } + + /** + * Utility methods that have been copied into this class to reduce the + * dependency footprint of this artifact. These utility methods typically + * live in web-utilities but that would pull in spring, jersey, jackson, + * etc. + */ + private X509Certificate extractClientCertificate(HttpServletRequest request) { + X509Certificate[] certs = (X509Certificate[]) request.getAttribute("javax.servlet.request.X509Certificate"); + + if (certs != null && certs.length > 0) { + return certs[0]; + } + + return null; + } + + private String extractPrincipal(X509Certificate cert) { + return cert.getSubjectDN().getName().trim(); + } + + private String formatProxyDn(String dn) { + return "<" + dn + ">"; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletRequestContextConfig.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletRequestContextConfig.java index e39ebccfb5..e376ab62b7 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletRequestContextConfig.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletRequestContextConfig.java @@ -23,6 +23,7 @@ import javax.servlet.http.HttpServletRequest; * An implementation of the NiFiWebContextConfig that retrieves configuration * from a HttpServletRequest instance. */ +@Deprecated public class HttpServletRequestContextConfig implements NiFiWebContextConfig { public static final String PROCESSOR_ID_PARAM = "processorId"; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml index 9f1cf99dee..4db963753e 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml @@ -123,6 +123,11 @@ nifi-custom-ui-utilities compile + + org.apache.nifi + nifi-ui-extension + compile + org.apache.nifi nifi-web-security diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java index 54111a1df5..1134c774e3 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java @@ -53,6 +53,10 @@ import org.apache.nifi.web.NiFiWebContext; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.web.ContentAccess; +import org.apache.nifi.ui.extension.UiExtension; +import org.apache.nifi.ui.extension.UiExtensionMapping; +import org.apache.nifi.web.NiFiWebConfigurationContext; +import org.apache.nifi.web.UiExtensionType; import org.eclipse.jetty.server.Connector; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.HttpConfiguration; @@ -97,14 +101,23 @@ public class JettyServer implements NiFiServer { }; private final Server server; + private final NiFiProperties props; + private ExtensionMapping extensionMapping; private WebAppContext webApiContext; private WebAppContext webDocsContext; + + // content viewer and mime type specific extensions private WebAppContext webContentViewerContext; - private Collection customUiWebContexts; private Collection contentViewerWebContexts; - private final NiFiProperties props; + // component (processor, controller service, reporting task) ui extensions + private UiExtensionMapping componentUiExtensions; + private Collection componentUiExtensionWebContexts; + + @Deprecated + private Collection customUiWebContexts; + /** * Creates and configures a new Jetty instance. * @@ -200,23 +213,34 @@ public class JettyServer implements NiFiServer { // handlers for each war and init params for the web api final HandlerCollection handlers = new HandlerCollection(); - final Map customUiMappings = new HashMap<>(); - final Map mimeTypeMappings = new HashMap<>(); + final Map mimeMappings = new HashMap<>(); final ClassLoader frameworkClassLoader = getClass().getClassLoader(); final ClassLoader jettyClassLoader = frameworkClassLoader.getParent(); + @Deprecated + final Map customUiMappings = new HashMap<>(); + // deploy the other wars if (CollectionUtils.isNotEmpty(otherWars)) { + // hold onto to the web contexts for all ui extensions customUiWebContexts = new ArrayList<>(); + componentUiExtensionWebContexts = new ArrayList<>(); contentViewerWebContexts = new ArrayList<>(); + // ui extension organized by component type + final Map> componentUiExtensionsByType = new HashMap<>(); for (File war : otherWars) { // see if this war is a custom processor ui + @Deprecated List customUiProcessorTypes = getWarExtensions(war, "META-INF/nifi-processor"); - List contentViewerMimeTypes = getWarExtensions(war, "META-INF/nifi-content-viewer"); - // only include wars that are for extensions - if (!customUiProcessorTypes.isEmpty() || !contentViewerMimeTypes.isEmpty()) { + // identify all known extension types in the war + final Map> uiExtensionInWar = new HashMap<>(); + identifyUiExtensionsForComponents(uiExtensionInWar, war); + + // only include wars that are for custom processor ui's + if (!customUiProcessorTypes.isEmpty() || !uiExtensionInWar.isEmpty()) { + // get the context path String warName = StringUtils.substringBeforeLast(war.getName(), "."); String warContextPath = String.format("/%s", warName); @@ -234,24 +258,62 @@ public class JettyServer implements NiFiServer { // also store it by type so we can populate the appropriate initialization parameters if (!customUiProcessorTypes.isEmpty()) { customUiWebContexts.add(extensionUiContext); + + // @Deprecated - supported custom uis as init params to the web api + for (String customUiProcessorType : customUiProcessorTypes) { + // map the processor type to the custom ui path + customUiMappings.put(customUiProcessorType, warContextPath); + } } else { - // record the mime type to web app mapping (need to handle type collision) - contentViewerWebContexts.add(extensionUiContext); + // create the ui extensions + for (final Map.Entry> entry : uiExtensionInWar.entrySet()) { + final UiExtensionType extensionType = entry.getKey(); + final List types = entry.getValue(); + + if (UiExtensionType.ContentViewer.equals(extensionType)) { + // consider each content type identified + for (final String contentType : types) { + // map the content type to the context path + mimeMappings.put(contentType, warContextPath); + } + + // this ui extension provides a content viewer + contentViewerWebContexts.add(extensionUiContext); + } else { + // consider each component type identified + for (final String componentType : types) { + logger.info(String.format("Loading UI extension [%s, %s] for %s", extensionType, warContextPath, types)); + + // record the extension definition + final UiExtension uiExtension = new UiExtension(extensionType, warContextPath); + + // create if this is the first extension for this component type + List componentUiExtensionsForType = componentUiExtensionsByType.get(componentType); + if (componentUiExtensionsForType == null) { + componentUiExtensionsForType = new ArrayList<>(); + componentUiExtensionsByType.put(componentType, componentUiExtensionsForType); + } + + // record this extension + componentUiExtensionsForType.add(uiExtension); + } + + // this ui extension provides a component custom ui + componentUiExtensionWebContexts.add(extensionUiContext); + } + } } // include custom ui web context in the handlers handlers.addHandler(extensionUiContext); - - // add the initialization paramters - for (String customUiProcessorType : customUiProcessorTypes) { - // map the processor type to the custom ui path - customUiMappings.put(customUiProcessorType, warContextPath); - } - for (final String contentViewerMimeType : contentViewerMimeTypes) { - mimeTypeMappings.put(contentViewerMimeType, warContextPath); - } } + } + + // record all ui extensions to give to the web api + componentUiExtensions = new UiExtensionMapping(componentUiExtensionsByType); + } else { + componentUiExtensions = new UiExtensionMapping(Collections.EMPTY_MAP); } // load the web ui app @@ -264,7 +326,7 @@ public class JettyServer implements NiFiServer { // load the content viewer app webContentViewerContext = loadWar(webContentViewerWar, "/nifi-content-viewer", frameworkClassLoader); - webContentViewerContext.getInitParams().putAll(mimeTypeMappings); + webContentViewerContext.getInitParams().putAll(mimeMappings); handlers.addHandler(webContentViewerContext); // create a web app for the docs @@ -315,6 +377,67 @@ public class JettyServer implements NiFiServer { return wars; } + private void readUiExtensions(final Map> uiExtensions, final UiExtensionType uiExtensionType, final JarFile jarFile, final JarEntry jarEntry) throws IOException { + if (jarEntry == null) { + return; + } + + // get an input stream for the nifi-processor configuration file + BufferedReader in = new BufferedReader(new InputStreamReader(jarFile.getInputStream(jarEntry))); + + // read in each configured type + String rawComponentType; + while ((rawComponentType = in.readLine()) != null) { + // extract the component type + final String componentType = extractComponentType(rawComponentType); + if (componentType != null) { + List extensions = uiExtensions.get(uiExtensionType); + + // if there are currently no extensions for this type create it + if (extensions == null) { + extensions = new ArrayList<>(); + uiExtensions.put(uiExtensionType, extensions); + } + + // add the specified type + extensions.add(componentType); + } + } + } + + /** + * Identifies all known UI extensions and stores them in the specified map. + * + * @param uiExtensions + * @param warFile + */ + private void identifyUiExtensionsForComponents(final Map> uiExtensions, final File warFile) { + try (final JarFile jarFile = new JarFile(warFile)) { + // locate the ui extensions + readUiExtensions(uiExtensions, UiExtensionType.ContentViewer, jarFile, jarFile.getJarEntry("META-INF/nifi-content-viewer")); + readUiExtensions(uiExtensions, UiExtensionType.ProcessorConfiguration, jarFile, jarFile.getJarEntry("META-INF/nifi-processor-configuration")); + readUiExtensions(uiExtensions, UiExtensionType.ControllerServiceConfiguration, jarFile, jarFile.getJarEntry("META-INF/nifi-controller-service-configuration")); + readUiExtensions(uiExtensions, UiExtensionType.ReportingTaskConfiguration, jarFile, jarFile.getJarEntry("META-INF/nifi-reporting-task-configuration")); + } catch (IOException ioe) { + logger.warn(String.format("Unable to inspect %s for a UI extensions.", warFile)); + } + } + + /** + * Extracts the component type. Trims the line and considers comments. Returns null if no type was found. + * + * @param line + * @return + */ + private String extractComponentType(final String line) { + final String trimmedLine = line.trim(); + if (!trimmedLine.isEmpty() && !trimmedLine.startsWith("#")) { + final int indexOfPound = trimmedLine.indexOf("#"); + return (indexOfPound > 0) ? trimmedLine.substring(0, indexOfPound) : trimmedLine; + } + return null; + } + /** * Returns the extension in the specified WAR using the specified path. * @@ -335,10 +458,11 @@ public class JettyServer implements NiFiServer { BufferedReader in = new BufferedReader(new InputStreamReader(jarFile.getInputStream(jarEntry))); // read in each configured type - String processorType; - while ((processorType = in.readLine()) != null) { - // ensure the line isn't blank - if (StringUtils.isNotBlank(processorType)) { + String rawProcessorType; + while ((rawProcessorType = in.readLine()) != null) { + // extract the processor type + final String processorType = extractComponentType(rawProcessorType); + if (processorType != null) { processorTypes.add(processorType); } } @@ -558,13 +682,18 @@ public class JettyServer implements NiFiServer { } } } - + // ensure the appropriate wars deployed successfully before injecting the NiFi context and security filters - // this must be done after starting the server (and ensuring there were no start up failures) if (webApiContext != null) { + // give the web api the component ui extensions final ServletContext webApiServletContext = webApiContext.getServletHandler().getServletContext(); + webApiServletContext.setAttribute("nifi-ui-extensions", componentUiExtensions); + + // get the application context final WebApplicationContext webApplicationContext = WebApplicationContextUtils.getRequiredWebApplicationContext(webApiServletContext); + // @Deprecated if (CollectionUtils.isNotEmpty(customUiWebContexts)) { final NiFiWebContext niFiWebContext = webApplicationContext.getBean("nifiWebContext", NiFiWebContext.class); @@ -576,22 +705,40 @@ public class JettyServer implements NiFiServer { // add the security filter to any custom ui wars final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter("springSecurityFilterChain"); if (securityFilter != null) { - customUiContext.addFilter(securityFilter, "/*", EnumSet.of(DispatcherType.REQUEST)); + customUiContext.addFilter(securityFilter, "/*", EnumSet.allOf(DispatcherType.class)); } } } + // component ui extensions + if (CollectionUtils.isNotEmpty(componentUiExtensionWebContexts)) { + final NiFiWebConfigurationContext configurationContext = webApplicationContext.getBean("nifiWebConfigurationContext", NiFiWebConfigurationContext.class); + + for (final WebAppContext customUiContext : componentUiExtensionWebContexts) { + // set the NiFi context in each custom ui servlet context + final ServletContext customUiServletContext = customUiContext.getServletHandler().getServletContext(); + customUiServletContext.setAttribute("nifi-web-configuration-context", configurationContext); + + // add the security filter to any ui extensions wars + final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter("springSecurityFilterChain"); + if (securityFilter != null) { + customUiContext.addFilter(securityFilter, "/*", EnumSet.allOf(DispatcherType.class)); + } + } + } + + // content viewer extensions if (CollectionUtils.isNotEmpty(contentViewerWebContexts)) { for (final WebAppContext contentViewerContext : contentViewerWebContexts) { // add the security filter to any content viewer wars final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter("springSecurityFilterChain"); if (securityFilter != null) { - contentViewerContext.addFilter(securityFilter, "/*", EnumSet.of(DispatcherType.REQUEST, DispatcherType.FORWARD, DispatcherType.INCLUDE)); + contentViewerContext.addFilter(securityFilter, "/*", EnumSet.allOf(DispatcherType.class)); } } } - // ensure the web content viewer war was loaded + // content viewer controller if (webContentViewerContext != null) { final ContentAccess contentAccess = webApplicationContext.getBean("contentAccess", ContentAccess.class); @@ -599,14 +746,13 @@ public class JettyServer implements NiFiServer { final ServletContext webContentViewerServletContext = webContentViewerContext.getServletHandler().getServletContext(); webContentViewerServletContext.setAttribute("nifi-content-access", contentAccess); - // add the security filter to the content viewer controller final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter("springSecurityFilterChain"); if (securityFilter != null) { - webContentViewerContext.addFilter(securityFilter, "/*", EnumSet.of(DispatcherType.REQUEST, DispatcherType.FORWARD, DispatcherType.INCLUDE)); + webContentViewerContext.addFilter(securityFilter, "/*", EnumSet.allOf(DispatcherType.class)); } } } - + // ensure the web document war was loaded and provide the extension mapping if (webDocsContext != null) { final ServletContext webDocsServletContext = webDocsContext.getServletHandler().getServletContext(); @@ -735,4 +881,4 @@ public class JettyServer implements NiFiServer { logger.warn("Failed to stop web server", ex); } } -} +} \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/pom.xml new file mode 100644 index 0000000000..22388b398e --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/pom.xml @@ -0,0 +1,21 @@ + + + 4.0.0 + + org.apache.nifi + nifi-web + 0.1.0-incubating-SNAPSHOT + + nifi-ui-extension + jar + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-user-actions + + + \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtension.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtension.java new file mode 100644 index 0000000000..e5b9b3e1ea --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtension.java @@ -0,0 +1,52 @@ +/* + * 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.ui.extension; + +import org.apache.nifi.web.UiExtensionType; + +/** + * Information about a UI extension required to be invoked. + */ +public class UiExtension { + + private final UiExtensionType extensionType; + private final String contextPath; + + public UiExtension(final UiExtensionType extensionType, final String contextPath) { + this.extensionType = extensionType; + this.contextPath = contextPath; + } + + /** + * The type of this UI extension. + * + * @return + */ + public UiExtensionType getExtensionType() { + return extensionType; + } + + /** + * The context path of this UI extenion. + * + * @return + */ + public String getContextPath() { + return contextPath; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtensionMapping.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtensionMapping.java new file mode 100644 index 0000000000..16bffd0b43 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtensionMapping.java @@ -0,0 +1,52 @@ +/* + * 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.ui.extension; + +import java.util.List; +import java.util.Map; + +/** + * Mapping of all discovered UI extensions. + */ +public class UiExtensionMapping { + + private final Map> uiExtensions; + + public UiExtensionMapping(Map> uiExtensions) { + this.uiExtensions = uiExtensions; + } + + /** + * Returns whether there are any UI extensions for the specified component type. + * + * @param type + * @return + */ + public boolean hasUiExtension(final String type) { + return uiExtensions.containsKey(type); + } + + /** + * Gets the listing of all discovered UI extensions for the specified component type. + * @param type + * @return + */ + public List getUiExtension(final String type) { + return uiExtensions.get(type); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/pom.xml index 6a51838923..5a87ff8030 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/pom.xml @@ -157,6 +157,11 @@ nifi-web-optimistic-locking provided + + org.apache.nifi + nifi-ui-extension + provided + org.apache.nifi nifi-administration diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java new file mode 100644 index 0000000000..ea3af14905 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java @@ -0,0 +1,475 @@ +/* + * 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.audit; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Date; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import org.apache.nifi.action.Action; +import org.apache.nifi.action.Component; +import org.apache.nifi.action.Operation; +import org.apache.nifi.action.component.details.ExtensionDetails; +import org.apache.nifi.action.details.ActionDetails; +import org.apache.nifi.action.details.ConfigureDetails; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.ConfiguredComponent; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.web.security.user.NiFiUserUtils; +import org.apache.nifi.user.NiFiUser; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceReference; +import org.apache.nifi.controller.service.ControllerServiceState; +import org.apache.nifi.reporting.ReportingTask; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; +import org.apache.nifi.web.dao.ControllerServiceDAO; +import org.aspectj.lang.ProceedingJoinPoint; +import org.aspectj.lang.annotation.Around; +import org.aspectj.lang.annotation.Aspect; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Audits controller service creation/removal and configuration changes. + */ +@Aspect +public class ControllerServiceAuditor extends NiFiAuditor { + + private static final Logger logger = LoggerFactory.getLogger(ControllerServiceAuditor.class); + + private static final String COMMENTS = "Comments"; + private static final String NAME = "Name"; + private static final String ANNOTATION_DATA = "Annotation Data"; + + /** + * Audits the creation of controller service via createControllerService(). + * + * This method only needs to be run 'after returning'. However, in Java 7 + * the order in which these methods are returned from + * Class.getDeclaredMethods (even though there is no order guaranteed) seems + * to differ from Java 6. SpringAOP depends on this ordering to determine + * advice precedence. By normalizing all advice into Around advice we can + * alleviate this issue. + * + * @param proceedingJoinPoint + * @return + * @throws java.lang.Throwable + */ + @Around("within(org.apache.nifi.web.dao.ControllerServiceDAO+) && " + + "execution(org.apache.nifi.controller.service.ControllerServiceNode createControllerService(org.apache.nifi.web.api.dto.ControllerServiceDTO))") + public ControllerServiceNode createControllerServiceAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + // update the controller service state + ControllerServiceNode controllerService = (ControllerServiceNode) proceedingJoinPoint.proceed(); + + // if no exceptions were thrown, add the controller service action... + final Action action = generateAuditRecord(controllerService, Operation.Add); + + // save the actions + if (action != null) { + saveAction(action, logger); + } + + return controllerService; + } + + /** + * Audits the configuration of a single controller service. + * + * @param proceedingJoinPoint + * @param controllerServiceDTO + * @param controllerServiceDAO + * @return + * @throws Throwable + */ + @Around("within(org.apache.nifi.web.dao.ControllerServiceDAO+) && " + + "execution(org.apache.nifi.controller.service.ControllerServiceNode updateControllerService(org.apache.nifi.web.api.dto.ControllerServiceDTO)) && " + + "args(controllerServiceDTO) && " + + "target(controllerServiceDAO)") + public Object updateControllerServiceAdvice(ProceedingJoinPoint proceedingJoinPoint, ControllerServiceDTO controllerServiceDTO, ControllerServiceDAO controllerServiceDAO) throws Throwable { + // determine the initial values for each property/setting thats changing + ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(controllerServiceDTO.getId()); + final Map values = extractConfiguredPropertyValues(controllerService, controllerServiceDTO); + final boolean isDisabled = isDisabled(controllerService); + + // update the controller service state + final ControllerServiceNode updatedControllerService = (ControllerServiceNode) proceedingJoinPoint.proceed(); + + // if no exceptions were thrown, add the controller service action... + controllerService = controllerServiceDAO.getControllerService(updatedControllerService.getIdentifier()); + + // get the current user + NiFiUser user = NiFiUserUtils.getNiFiUser(); + + // ensure the user was found + if (user != null) { + // determine the updated values + Map updatedValues = extractConfiguredPropertyValues(controllerService, controllerServiceDTO); + + // create the controller service details + ExtensionDetails serviceDetails = new ExtensionDetails(); + serviceDetails.setType(controllerService.getControllerServiceImplementation().getClass().getSimpleName()); + + // create a controller service action + Date actionTimestamp = new Date(); + Collection actions = new ArrayList<>(); + + // go through each updated value + for (String property : updatedValues.keySet()) { + String newValue = updatedValues.get(property); + String oldValue = values.get(property); + Operation operation = null; + + // determine the type of operation + if (oldValue == null || newValue == null || !newValue.equals(oldValue)) { + operation = Operation.Configure; + } + + // create a configuration action accordingly + if (operation != null) { + // clear the value if this property is sensitive + final PropertyDescriptor propertyDescriptor = controllerService.getControllerServiceImplementation().getPropertyDescriptor(property); + if (propertyDescriptor != null && propertyDescriptor.isSensitive()) { + if (newValue != null) { + newValue = "********"; + } + if (oldValue != null) { + oldValue = "********"; + } + } else if (ANNOTATION_DATA.equals(property)) { + if (newValue != null) { + newValue = ""; + } + if (oldValue != null) { + oldValue = ""; + } + } + + final ConfigureDetails actionDetails = new ConfigureDetails(); + actionDetails.setName(property); + actionDetails.setValue(newValue); + actionDetails.setPreviousValue(oldValue); + + // create a configuration action + Action configurationAction = new Action(); + configurationAction.setUserDn(user.getDn()); + configurationAction.setUserName(user.getUserName()); + configurationAction.setOperation(operation); + configurationAction.setTimestamp(actionTimestamp); + configurationAction.setSourceId(controllerService.getIdentifier()); + configurationAction.setSourceName(controllerService.getName()); + configurationAction.setSourceType(Component.ControllerService); + configurationAction.setComponentDetails(serviceDetails); + configurationAction.setActionDetails(actionDetails); + actions.add(configurationAction); + } + } + + // determine the new executing state + final boolean updateIsDisabled = isDisabled(updatedControllerService); + + // determine if the running state has changed and its not disabled + if (isDisabled != updateIsDisabled) { + // create a controller service action + Action serviceAction = new Action(); + serviceAction.setUserDn(user.getDn()); + serviceAction.setUserName(user.getUserName()); + serviceAction.setTimestamp(new Date()); + serviceAction.setSourceId(controllerService.getIdentifier()); + serviceAction.setSourceName(controllerService.getName()); + serviceAction.setSourceType(Component.ControllerService); + serviceAction.setComponentDetails(serviceDetails); + + // set the operation accordingly + if (updateIsDisabled) { + serviceAction.setOperation(Operation.Disable); + } else { + serviceAction.setOperation(Operation.Enable); + } + actions.add(serviceAction); + } + + // ensure there are actions to record + if (!actions.isEmpty()) { + // save the actions + saveActions(actions, logger); + } + } + + return updatedControllerService; + } + + /** + * Audits the update of a component referencing a controller service. + * + * @param proceedingJoinPoint + * @param controllerServiceId + * @return + * @throws Throwable + */ + @Around("within(org.apache.nifi.web.dao.ControllerServiceDAO+) && " + + "execution(org.apache.nifi.controller.service.ControllerServiceReference updateControllerServiceReferencingComponents(java.lang.String, org.apache.nifi.controller.ScheduledState, org.apache.nifi.controller.service.ControllerServiceState))") + public Object updateControllerServiceReferenceAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + // update the controller service references + final ControllerServiceReference controllerServiceReference = (ControllerServiceReference) proceedingJoinPoint.proceed(); + + // get the current user + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + + if (user != null) { + final Collection actions = new ArrayList<>(); + final Collection visitedServices = new ArrayList<>(); + visitedServices.add(controllerServiceReference.getReferencedComponent().getIdentifier()); + + // get all applicable actions + getUpdateActionsForReferencingComponents(user, actions, visitedServices, controllerServiceReference.getReferencingComponents()); + + // ensure there are actions to record + if (!actions.isEmpty()) { + // save the actions + saveActions(actions, logger); + } + } + + return controllerServiceReference; + } + + /** + * Gets the update actions for all specified referencing components. + * + * @param user + * @param actions + * @param visitedServices + * @param referencingComponents + */ + private void getUpdateActionsForReferencingComponents(final NiFiUser user, final Collection actions, final Collection visitedServices, final Set referencingComponents) { + // consider each component updates + for (final ConfiguredComponent component : referencingComponents) { + if (component instanceof ProcessorNode) { + final ProcessorNode processor = ((ProcessorNode) component); + + // create the processor details + ExtensionDetails processorDetails = new ExtensionDetails(); + processorDetails.setType(processor.getProcessor().getClass().getSimpleName()); + + // create a processor action + Action processorAction = new Action(); + processorAction.setUserDn(user.getDn()); + processorAction.setUserName(user.getUserName()); + processorAction.setTimestamp(new Date()); + processorAction.setSourceId(processor.getIdentifier()); + processorAction.setSourceName(processor.getName()); + processorAction.setSourceType(Component.Processor); + processorAction.setComponentDetails(processorDetails); + processorAction.setOperation(ScheduledState.RUNNING.equals(processor.getScheduledState()) ? Operation.Start : Operation.Stop); + actions.add(processorAction); + } else if (component instanceof ReportingTask) { + final ReportingTaskNode reportingTask = ((ReportingTaskNode) component); + + // create the reporting task details + ExtensionDetails processorDetails = new ExtensionDetails(); + processorDetails.setType(reportingTask.getReportingTask().getClass().getSimpleName()); + + // create a reporting task action + Action reportingTaskAction = new Action(); + reportingTaskAction.setUserDn(user.getDn()); + reportingTaskAction.setUserName(user.getUserName()); + reportingTaskAction.setTimestamp(new Date()); + reportingTaskAction.setSourceId(reportingTask.getIdentifier()); + reportingTaskAction.setSourceName(reportingTask.getName()); + reportingTaskAction.setSourceType(Component.ReportingTask); + reportingTaskAction.setComponentDetails(processorDetails); + reportingTaskAction.setOperation(ScheduledState.RUNNING.equals(reportingTask.getScheduledState()) ? Operation.Start : Operation.Stop); + actions.add(reportingTaskAction); + } else if (component instanceof ControllerServiceNode) { + final ControllerServiceNode controllerService = ((ControllerServiceNode) component); + + // create the controller service details + ExtensionDetails serviceDetails = new ExtensionDetails(); + serviceDetails.setType(controllerService.getControllerServiceImplementation().getClass().getSimpleName()); + + // create a controller service action + Action serviceAction = new Action(); + serviceAction.setUserDn(user.getDn()); + serviceAction.setUserName(user.getUserName()); + serviceAction.setTimestamp(new Date()); + serviceAction.setSourceId(controllerService.getIdentifier()); + serviceAction.setSourceName(controllerService.getName()); + serviceAction.setSourceType(Component.ControllerService); + serviceAction.setComponentDetails(serviceDetails); + serviceAction.setOperation(isDisabled(controllerService) ? Operation.Disable : Operation.Enable); + actions.add(serviceAction); + + // need to consider components referencing this controller service (transitive) + if (!visitedServices.contains(controllerService.getIdentifier())) { + getUpdateActionsForReferencingComponents(user, actions, visitedServices, controllerService.getReferences().getReferencingComponents()); + } + } + } + } + + /** + * Audits the removal of a controller service via deleteControllerService(). + * + * @param proceedingJoinPoint + * @param controllerServiceId + * @param controllerServiceDAO + * @throws Throwable + */ + @Around("within(org.apache.nifi.web.dao.ControllerServiceDAO+) && " + + "execution(void deleteControllerService(java.lang.String)) && " + + "args(controllerServiceId) && " + + "target(controllerServiceDAO)") + public void removeControllerServiceAdvice(ProceedingJoinPoint proceedingJoinPoint, String controllerServiceId, ControllerServiceDAO controllerServiceDAO) throws Throwable { + // get the controller service before removing it + ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(controllerServiceId); + + // remove the controller service + proceedingJoinPoint.proceed(); + + // if no exceptions were thrown, add removal actions... + // audit the controller service removal + final Action action = generateAuditRecord(controllerService, Operation.Remove); + + // save the actions + if (action != null) { + saveAction(action, logger); + } + } + + /** + * Generates an audit record for the creation of a controller service. + * + * @param controllerService + * @param operation + * @return + */ + private Action generateAuditRecord(ControllerServiceNode controllerService, Operation operation) { + return generateAuditRecord(controllerService, operation, null); + } + + /** + * Generates an audit record for the creation of a controller service. + * + * @param controllerService + * @param operation + * @param actionDetails + * @return + */ + private Action generateAuditRecord(ControllerServiceNode controllerService, Operation operation, ActionDetails actionDetails) { + Action action = null; + + // get the current user + NiFiUser user = NiFiUserUtils.getNiFiUser(); + + // ensure the user was found + if (user != null) { + // create the controller service details + ExtensionDetails serviceDetails = new ExtensionDetails(); + serviceDetails.setType(controllerService.getControllerServiceImplementation().getClass().getSimpleName()); + + // create the controller service action for adding this controller service + action = new Action(); + action.setUserDn(user.getDn()); + action.setUserName(user.getUserName()); + action.setOperation(operation); + action.setTimestamp(new Date()); + action.setSourceId(controllerService.getIdentifier()); + action.setSourceName(controllerService.getName()); + action.setSourceType(Component.ControllerService); + action.setComponentDetails(serviceDetails); + + if (actionDetails != null) { + action.setActionDetails(actionDetails); + } + } + + return action; + } + + /** + * Extracts the values for the configured properties from the specified + * ControllerService. + * + * @param controllerService + * @param controllerServiceDTO + * @return + */ + private Map extractConfiguredPropertyValues(ControllerServiceNode controllerService, ControllerServiceDTO controllerServiceDTO) { + Map values = new HashMap<>(); + + if (controllerServiceDTO.getName() != null) { + values.put(NAME, controllerService.getName()); + } + if (controllerServiceDTO.getAnnotationData() != null) { + values.put(ANNOTATION_DATA, controllerService.getAnnotationData()); + } + if (controllerServiceDTO.getProperties() != null) { + // for each property specified, extract its configured value + Map properties = controllerServiceDTO.getProperties(); + Map configuredProperties = controllerService.getProperties(); + for (String propertyName : properties.keySet()) { + // build a descriptor for getting the configured value + PropertyDescriptor propertyDescriptor = new PropertyDescriptor.Builder().name(propertyName).build(); + String configuredPropertyValue = configuredProperties.get(propertyDescriptor); + + // if the configured value couldn't be found, use the default value from the actual descriptor + if (configuredPropertyValue == null) { + propertyDescriptor = locatePropertyDescriptor(configuredProperties.keySet(), propertyDescriptor); + configuredPropertyValue = propertyDescriptor.getDefaultValue(); + } + values.put(propertyName, configuredPropertyValue); + } + } + if (controllerServiceDTO.getComments() != null) { + values.put(COMMENTS, controllerService.getComments()); + } + + return values; + } + + /** + * Locates the actual property descriptor for the given spec property + * descriptor. + * + * @param propertyDescriptors + * @param specDescriptor + * @return + */ + private PropertyDescriptor locatePropertyDescriptor(Set propertyDescriptors, PropertyDescriptor specDescriptor) { + for (PropertyDescriptor propertyDescriptor : propertyDescriptors) { + if (propertyDescriptor.equals(specDescriptor)) { + return propertyDescriptor; + } + } + return specDescriptor; + } + + /** + * Returns whether the specified controller service is disabled (or disabling). + * + * @param controllerService + * @return + */ + private boolean isDisabled(final ControllerServiceNode controllerService) { + return ControllerServiceState.DISABLED.equals(controllerService.getState()) || ControllerServiceState.DISABLING.equals(controllerService.getState()); + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java index ef66dc6975..b079da893b 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java @@ -50,7 +50,7 @@ public class FunnelAuditor extends NiFiAuditor { */ @Around("within(org.apache.nifi.web.dao.FunnelDAO+) && " + "execution(org.apache.nifi.connectable.Funnel createFunnel(java.lang.String, org.apache.nifi.web.api.dto.FunnelDTO))") - public Object createFunnelAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + public Funnel createFunnelAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { // perform the underlying operation Funnel funnel = (Funnel) proceedingJoinPoint.proceed(); @@ -69,6 +69,7 @@ public class FunnelAuditor extends NiFiAuditor { * Audits the removal of a funnel. * * @param proceedingJoinPoint + * @param groupId * @param funnelId * @param funnelDAO * @throws Throwable @@ -97,6 +98,8 @@ public class FunnelAuditor extends NiFiAuditor { * Generates an audit record for the creation of the specified funnel. * * @param funnel + * @param operation + * @return */ public Action generateAuditRecord(Funnel funnel, Operation operation) { return generateAuditRecord(funnel, operation, null); @@ -106,6 +109,9 @@ public class FunnelAuditor extends NiFiAuditor { * Generates an audit record for the creation of the specified funnel. * * @param funnel + * @param operation + * @param actionDetails + * @return */ public Action generateAuditRecord(Funnel funnel, Operation operation, ActionDetails actionDetails) { Action action = null; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/NiFiAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/NiFiAuditor.java index 046d5ff7d1..adff9d1d4c 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/NiFiAuditor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/NiFiAuditor.java @@ -54,14 +54,13 @@ public abstract class NiFiAuditor { * Records the actions. * * @param actions + * @param logger */ protected void saveActions(Collection actions, Logger logger) { - /* - * if we're a clustered node, then set actions on threadlocal - */ - if (serviceFacade.isClustered()) { - // if we're a connected node, then put audit actions on threadlocal to propagate back to manager - ClusterContext ctx = ClusterContextThreadLocal.getContext(); + ClusterContext ctx = ClusterContextThreadLocal.getContext(); + + // if we're a connected node, then put audit actions on threadlocal to propagate back to manager + if (ctx != null) { ctx.getActions().addAll(actions); } else { // if we're the cluster manager, or a disconnected node, or running standalone, then audit actions diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java index 58dcc39503..af4b5bd06b 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java @@ -60,7 +60,7 @@ public class PortAuditor extends NiFiAuditor { */ @Around("within(org.apache.nifi.web.dao.PortDAO+) && " + "execution(org.apache.nifi.connectable.Port createPort(java.lang.String, org.apache.nifi.web.api.dto.PortDTO))") - public Object createPortAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + public Port createPortAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { // perform the underlying operation Port port = (Port) proceedingJoinPoint.proceed(); @@ -79,6 +79,9 @@ public class PortAuditor extends NiFiAuditor { * Audits the update of a port. * * @param proceedingJoinPoint + * @param groupId + * @param portDTO + * @param portDAO * @return * @throws Throwable */ @@ -86,7 +89,7 @@ public class PortAuditor extends NiFiAuditor { + "execution(org.apache.nifi.connectable.Port updatePort(java.lang.String, org.apache.nifi.web.api.dto.PortDTO)) && " + "args(groupId, portDTO) && " + "target(portDAO)") - public Object updatePortAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, PortDTO portDTO, PortDAO portDAO) throws Throwable { + public Port updatePortAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, PortDTO portDTO, PortDAO portDAO) throws Throwable { final Port port = portDAO.getPort(groupId, portDTO.getId()); final ScheduledState scheduledState = port.getScheduledState(); final String name = port.getName(); @@ -261,8 +264,9 @@ public class PortAuditor extends NiFiAuditor { * Audits the removal of a processor via deleteProcessor(). * * @param proceedingJoinPoint - * @param processorId - * @param processorDAO + * @param groupId + * @param portId + * @param portDAO * @throws Throwable */ @Around("within(org.apache.nifi.web.dao.PortDAO+) && " @@ -290,6 +294,8 @@ public class PortAuditor extends NiFiAuditor { * Generates an audit record for the creation of the specified port. * * @param port + * @param operation + * @return */ public Action generateAuditRecord(Port port, Operation operation) { return generateAuditRecord(port, operation, null); @@ -299,6 +305,9 @@ public class PortAuditor extends NiFiAuditor { * Generates an audit record for the creation of the specified port. * * @param port + * @param operation + * @param actionDetails + * @return */ public Action generateAuditRecord(Port port, Operation operation, ActionDetails actionDetails) { Action action = null; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java index 1b2af7d999..43918812cf 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java @@ -55,15 +55,12 @@ public class ProcessGroupAuditor extends NiFiAuditor { * alleviate this issue. * * @param proceedingJoinPoint + * @return + * @throws java.lang.Throwable */ -// @AfterReturning( -// pointcut="within(org.apache.nifi.web.dao.ProcessGroupDAO+) && " -// + "execution(org.apache.nifi.web.api.dto.ProcessGroupDTO createProcessGroup(java.lang.String, org.apache.nifi.web.api.dto.ProcessGroupDTO))", -// returning="processGroup" -// ) @Around("within(org.apache.nifi.web.dao.ProcessGroupDAO+) && " + "execution(org.apache.nifi.groups.ProcessGroup createProcessGroup(java.lang.String, org.apache.nifi.web.api.dto.ProcessGroupDTO))") - public Object createProcessGroupAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + public ProcessGroup createProcessGroupAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { // create the process group ProcessGroup processGroup = (ProcessGroup) proceedingJoinPoint.proceed(); @@ -83,13 +80,14 @@ public class ProcessGroupAuditor extends NiFiAuditor { * Audits the update of process group configuration. * * @param proceedingJoinPoint + * @param processGroupDTO * @return * @throws Throwable */ @Around("within(org.apache.nifi.web.dao.ProcessGroupDAO+) && " + "execution(org.apache.nifi.groups.ProcessGroup updateProcessGroup(org.apache.nifi.web.api.dto.ProcessGroupDTO)) && " + "args(processGroupDTO)") - public Object updateProcessGroupAdvice(ProceedingJoinPoint proceedingJoinPoint, ProcessGroupDTO processGroupDTO) throws Throwable { + public ProcessGroup updateProcessGroupAdvice(ProceedingJoinPoint proceedingJoinPoint, ProcessGroupDTO processGroupDTO) throws Throwable { ProcessGroupDAO processGroupDAO = getProcessGroupDAO(); ProcessGroup processGroup = processGroupDAO.getProcessGroup(processGroupDTO.getId()); @@ -170,7 +168,7 @@ public class ProcessGroupAuditor extends NiFiAuditor { processGroupAction.setTimestamp(new Date()); // determine the running state - if (processGroupDTO.isRunning().booleanValue()) { + if (processGroupDTO.isRunning()) { processGroupAction.setOperation(Operation.Start); } else { processGroupAction.setOperation(Operation.Stop); @@ -194,7 +192,6 @@ public class ProcessGroupAuditor extends NiFiAuditor { * * @param proceedingJoinPoint * @param groupId - * @param processGroupDAO * @throws Throwable */ @Around("within(org.apache.nifi.web.dao.ProcessGroupDAO+) && " @@ -222,6 +219,7 @@ public class ProcessGroupAuditor extends NiFiAuditor { * Generates an audit record for the creation of a process group. * * @param processGroup + * @param operation * @return */ public Action generateAuditRecord(ProcessGroup processGroup, Operation operation) { @@ -232,6 +230,8 @@ public class ProcessGroupAuditor extends NiFiAuditor { * Generates an audit record for the creation of a process group. * * @param processGroup + * @param operation + * @param actionDetails * @return */ public Action generateAuditRecord(ProcessGroup processGroup, Operation operation, ActionDetails actionDetails) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java index c3c70c4780..ff377a3413 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java @@ -30,7 +30,7 @@ import java.util.Set; import org.apache.nifi.action.Action; import org.apache.nifi.action.Component; import org.apache.nifi.action.Operation; -import org.apache.nifi.action.component.details.ProcessorDetails; +import org.apache.nifi.action.component.details.ExtensionDetails; import org.apache.nifi.action.details.ActionDetails; import org.apache.nifi.action.details.ConfigureDetails; import org.apache.nifi.components.PropertyDescriptor; @@ -78,13 +78,10 @@ public class ProcessorAuditor extends NiFiAuditor { * advice precedence. By normalizing all advice into Around advice we can * alleviate this issue. * - * @param processor + * @param proceedingJoinPoint + * @return + * @throws java.lang.Throwable */ -// @AfterReturning( -// pointcut="within(org.apache.nifi.web.dao.ProcessorDAO+) && " -// + "execution(org.apache.nifi.web.api.dto.ProcessorDTO createProcessor(org.apache.nifi.web.api.dto.ProcessorDTO))", -// returning="processor" -// ) @Around("within(org.apache.nifi.web.dao.ProcessorDAO+) && " + "execution(org.apache.nifi.controller.ProcessorNode createProcessor(java.lang.String, org.apache.nifi.web.api.dto.ProcessorDTO))") public ProcessorNode createProcessorAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { @@ -106,6 +103,7 @@ public class ProcessorAuditor extends NiFiAuditor { * Audits the configuration of a single processor. * * @param proceedingJoinPoint + * @param groupId * @param processorDTO * @param processorDAO * @return @@ -115,7 +113,7 @@ public class ProcessorAuditor extends NiFiAuditor { + "execution(org.apache.nifi.controller.ProcessorNode updateProcessor(java.lang.String, org.apache.nifi.web.api.dto.ProcessorDTO)) && " + "args(groupId, processorDTO) && " + "target(processorDAO)") - public Object updateProcessorAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ProcessorDTO processorDTO, ProcessorDAO processorDAO) throws Throwable { + public ProcessorNode updateProcessorAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ProcessorDTO processorDTO, ProcessorDAO processorDAO) throws Throwable { // determine the initial values for each property/setting thats changing ProcessorNode processor = processorDAO.getProcessor(groupId, processorDTO.getId()); final Map values = extractConfiguredPropertyValues(processor, processorDTO); @@ -137,7 +135,7 @@ public class ProcessorAuditor extends NiFiAuditor { Map updatedValues = extractConfiguredPropertyValues(processor, processorDTO); // create the processor details - ProcessorDetails processorDetails = new ProcessorDetails(); + ExtensionDetails processorDetails = new ExtensionDetails(); processorDetails.setType(processor.getProcessor().getClass().getSimpleName()); // create a processor action @@ -240,6 +238,7 @@ public class ProcessorAuditor extends NiFiAuditor { * Audits the removal of a processor via deleteProcessor(). * * @param proceedingJoinPoint + * @param groupId * @param processorId * @param processorDAO * @throws Throwable @@ -281,6 +280,7 @@ public class ProcessorAuditor extends NiFiAuditor { * * @param processor * @param operation + * @param actionDetails * @return */ public Action generateAuditRecord(ProcessorNode processor, Operation operation, ActionDetails actionDetails) { @@ -292,7 +292,7 @@ public class ProcessorAuditor extends NiFiAuditor { // ensure the user was found if (user != null) { // create the processor details - ProcessorDetails processorDetails = new ProcessorDetails(); + ExtensionDetails processorDetails = new ExtensionDetails(); processorDetails.setType(processor.getProcessor().getClass().getSimpleName()); // create the processor action for adding this processor diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java index 9cc694bcf1..ba7814194b 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java @@ -76,16 +76,13 @@ public class RelationshipAuditor extends NiFiAuditor { * advice precedence. By normalizing all advice into Around advice we can * alleviate this issue. * - * @param connection + * @param proceedingJoinPoint + * @return + * @throws java.lang.Throwable */ -// @AfterReturning( -// pointcut="within(org.apache.nifi.view.dao.ConnectionDAO+) && " -// + "execution(org.apache.nifi.view.model.Connection createConnection(org.apache.nifi.api.dto.ConnectionDTO))", -// returning="connection" -// ) @Around("within(org.apache.nifi.web.dao.ConnectionDAO+) && " + "execution(org.apache.nifi.connectable.Connection createConnection(java.lang.String, org.apache.nifi.web.api.dto.ConnectionDTO))") - public Object createConnectionAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + public Connection createConnectionAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { // perform the underlying operation Connection connection = (Connection) proceedingJoinPoint.proceed(); @@ -105,6 +102,7 @@ public class RelationshipAuditor extends NiFiAuditor { * Audits the creation and removal of relationships via updateConnection(). * * @param proceedingJoinPoint + * @param groupId * @param connectionDTO * @param connectionDAO * @return @@ -114,7 +112,7 @@ public class RelationshipAuditor extends NiFiAuditor { + "execution(org.apache.nifi.connectable.Connection updateConnection(java.lang.String, org.apache.nifi.web.api.dto.ConnectionDTO)) && " + "args(groupId, connectionDTO) && " + "target(connectionDAO)") - public Object updateConnectionAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ConnectionDTO connectionDTO, ConnectionDAO connectionDAO) throws Throwable { + public Connection updateConnectionAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ConnectionDTO connectionDTO, ConnectionDAO connectionDAO) throws Throwable { // get the previous configuration Connection connection = connectionDAO.getConnection(groupId, connectionDTO.getId()); Connectable previousDestination = connection.getDestination(); @@ -218,6 +216,7 @@ public class RelationshipAuditor extends NiFiAuditor { * Audits the removal of relationships via deleteConnection(). * * @param proceedingJoinPoint + * @param groupId * @param id * @param connectionDAO * @throws Throwable @@ -251,7 +250,9 @@ public class RelationshipAuditor extends NiFiAuditor { * Creates action details for connect/disconnect actions. * * @param connection + * @param source * @param relationships + * @param destination * @return */ public ConnectDetails createConnectDetails(final Connection connection, final Connectable source, final Collection relationships, final Connectable destination) { @@ -327,6 +328,7 @@ public class RelationshipAuditor extends NiFiAuditor { * * @param connection * @param operation + * @param actionDetails * @return */ public Action generateAuditRecordForConnection(Connection connection, Operation operation, ActionDetails actionDetails) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java index a10495d6c2..d0836d4adb 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java @@ -60,16 +60,13 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor { * advice precedence. By normalizing all advice into Around advice we can * alleviate this issue. * - * @param processor + * @param proceedingJoinPoint + * @return + * @throws java.lang.Throwable */ -// @AfterReturning( -// pointcut="within(org.apache.nifi.web.dao.RemoteProcessGroupDAO+) && " -// + "execution(org.apache.nifi.web.api.dto.RemoteProcessGroupDTO createRemoteProcessGroup(java.lang.String, org.apache.nifi.web.api.dto.RemoteProcessGroupDTO))", -// returning="remoteProcessGroup" -// ) @Around("within(org.apache.nifi.web.dao.RemoteProcessGroupDAO+) && " + "execution(org.apache.nifi.groups.RemoteProcessGroup createRemoteProcessGroup(java.lang.String, org.apache.nifi.web.api.dto.RemoteProcessGroupDTO))") - public Object createRemoteProcessGroupAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + public RemoteProcessGroup createRemoteProcessGroupAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { // create the remote process group RemoteProcessGroup remoteProcessGroup = (RemoteProcessGroup) proceedingJoinPoint.proceed(); @@ -99,7 +96,7 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor { + "execution(org.apache.nifi.groups.RemoteProcessGroup updateRemoteProcessGroup(java.lang.String, org.apache.nifi.web.api.dto.RemoteProcessGroupDTO)) && " + "args(groupId, remoteProcessGroupDTO) && " + "target(remoteProcessGroupDAO)") - public Object auditUpdateProcessGroupConfiguration(ProceedingJoinPoint proceedingJoinPoint, String groupId, RemoteProcessGroupDTO remoteProcessGroupDTO, RemoteProcessGroupDAO remoteProcessGroupDAO) throws Throwable { + public RemoteProcessGroup auditUpdateProcessGroupConfiguration(ProceedingJoinPoint proceedingJoinPoint, String groupId, RemoteProcessGroupDTO remoteProcessGroupDTO, RemoteProcessGroupDAO remoteProcessGroupDAO) throws Throwable { final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.getRemoteProcessGroup(groupId, remoteProcessGroupDTO.getId()); // record the current value of this remoteProcessGroups configuration for comparisons later @@ -332,7 +329,9 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor { /** * Generates an audit record for the specified remote process group. * - * @param port + * @param remoteProcessGroup + * @param operation + * @return */ public Action generateAuditRecord(RemoteProcessGroup remoteProcessGroup, Operation operation) { return generateAuditRecord(remoteProcessGroup, operation, null); @@ -341,7 +340,10 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor { /** * Generates an audit record for the specified remote process group. * - * @param port + * @param remoteProcessGroup + * @param operation + * @param actionDetails + * @return */ public Action generateAuditRecord(RemoteProcessGroup remoteProcessGroup, Operation operation, ActionDetails actionDetails) { Action action = null; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java new file mode 100644 index 0000000000..ba2cdfb4c5 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java @@ -0,0 +1,353 @@ +/* + * 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.audit; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Date; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import org.apache.nifi.action.Action; +import org.apache.nifi.action.Component; +import org.apache.nifi.action.Operation; +import org.apache.nifi.action.component.details.ExtensionDetails; +import org.apache.nifi.action.details.ActionDetails; +import org.apache.nifi.action.details.ConfigureDetails; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.web.security.user.NiFiUserUtils; +import org.apache.nifi.user.NiFiUser; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; +import org.apache.nifi.web.dao.ReportingTaskDAO; +import org.aspectj.lang.ProceedingJoinPoint; +import org.aspectj.lang.annotation.Around; +import org.aspectj.lang.annotation.Aspect; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Audits reporting creation/removal and configuration changes. + */ +@Aspect +public class ReportingTaskAuditor extends NiFiAuditor { + + private static final Logger logger = LoggerFactory.getLogger(ReportingTaskAuditor.class); + + private static final String NAME = "Name"; + private static final String ANNOTATION_DATA = "Annotation Data"; + + /** + * Audits the creation of reporting task via createReportingTask(). + * + * This method only needs to be run 'after returning'. However, in Java 7 + * the order in which these methods are returned from + * Class.getDeclaredMethods (even though there is no order guaranteed) seems + * to differ from Java 6. SpringAOP depends on this ordering to determine + * advice precedence. By normalizing all advice into Around advice we can + * alleviate this issue. + * + * @param proceedingJoinPoint + * @return + * @throws java.lang.Throwable + */ + @Around("within(org.apache.nifi.web.dao.ReportingTaskDAO+) && " + + "execution(org.apache.nifi.controller.ReportingTaskNode createReportingTask(org.apache.nifi.web.api.dto.ReportingTaskDTO))") + public ReportingTaskNode createReportingTaskAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + // update the reporting task state + ReportingTaskNode reportingTask = (ReportingTaskNode) proceedingJoinPoint.proceed(); + + // if no exceptions were thrown, add the reporting task action... + final Action action = generateAuditRecord(reportingTask, Operation.Add); + + // save the actions + if (action != null) { + saveAction(action, logger); + } + + return reportingTask; + } + + /** + * Audits the configuration of a reporting task. + * + * @param proceedingJoinPoint + * @param reportingTaskDTO + * @param reportingTaskDAO + * @return + * @throws Throwable + */ + @Around("within(org.apache.nifi.web.dao.ReportingTaskDAO+) && " + + "execution(org.apache.nifi.controller.ReportingTaskNode updateReportingTask(org.apache.nifi.web.api.dto.ReportingTaskDTO)) && " + + "args(reportingTaskDTO) && " + + "target(reportingTaskDAO)") + public Object updateReportingTaskAdvice(ProceedingJoinPoint proceedingJoinPoint, ReportingTaskDTO reportingTaskDTO, ReportingTaskDAO reportingTaskDAO) throws Throwable { + // determine the initial values for each property/setting thats changing + ReportingTaskNode reportingTask = reportingTaskDAO.getReportingTask(reportingTaskDTO.getId()); + final Map values = extractConfiguredPropertyValues(reportingTask, reportingTaskDTO); + final ScheduledState scheduledState = reportingTask.getScheduledState(); + + // update the reporting task state + final ReportingTaskNode updatedReportingTask = (ReportingTaskNode) proceedingJoinPoint.proceed(); + + // if no exceptions were thrown, add the reporting task action... + reportingTask = reportingTaskDAO.getReportingTask(updatedReportingTask.getIdentifier()); + + // get the current user + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + + // ensure the user was found + if (user != null) { + // determine the updated values + Map updatedValues = extractConfiguredPropertyValues(reportingTask, reportingTaskDTO); + + // create the reporting task details + ExtensionDetails taskDetails = new ExtensionDetails(); + taskDetails.setType(reportingTask.getReportingTask().getClass().getSimpleName()); + + // create a reporting task action + Date actionTimestamp = new Date(); + Collection actions = new ArrayList<>(); + + // go through each updated value + for (String property : updatedValues.keySet()) { + String newValue = updatedValues.get(property); + String oldValue = values.get(property); + Operation operation = null; + + // determine the type of operation + if (oldValue == null || newValue == null || !newValue.equals(oldValue)) { + operation = Operation.Configure; + } + + // create a configuration action accordingly + if (operation != null) { + // clear the value if this property is sensitive + final PropertyDescriptor propertyDescriptor = reportingTask.getReportingTask().getPropertyDescriptor(property); + if (propertyDescriptor != null && propertyDescriptor.isSensitive()) { + if (newValue != null) { + newValue = "********"; + } + if (oldValue != null) { + oldValue = "********"; + } + } else if (ANNOTATION_DATA.equals(property)) { + if (newValue != null) { + newValue = ""; + } + if (oldValue != null) { + oldValue = ""; + } + } + + final ConfigureDetails actionDetails = new ConfigureDetails(); + actionDetails.setName(property); + actionDetails.setValue(newValue); + actionDetails.setPreviousValue(oldValue); + + // create a configuration action + Action configurationAction = new Action(); + configurationAction.setUserDn(user.getDn()); + configurationAction.setUserName(user.getUserName()); + configurationAction.setOperation(operation); + configurationAction.setTimestamp(actionTimestamp); + configurationAction.setSourceId(reportingTask.getIdentifier()); + configurationAction.setSourceName(reportingTask.getName()); + configurationAction.setSourceType(Component.ReportingTask); + configurationAction.setComponentDetails(taskDetails); + configurationAction.setActionDetails(actionDetails); + actions.add(configurationAction); + } + } + + // determine the new executing state + final ScheduledState updatedScheduledState = reportingTask.getScheduledState(); + + // determine if the running state has changed and its not disabled + if (scheduledState != updatedScheduledState) { + // create a reporting task action + Action taskAction = new Action(); + taskAction.setUserDn(user.getDn()); + taskAction.setUserName(user.getUserName()); + taskAction.setTimestamp(new Date()); + taskAction.setSourceId(reportingTask.getIdentifier()); + taskAction.setSourceName(reportingTask.getName()); + taskAction.setSourceType(Component.ReportingTask); + taskAction.setComponentDetails(taskDetails); + + // set the operation accordingly + if (ScheduledState.RUNNING.equals(updatedScheduledState)) { + taskAction.setOperation(Operation.Start); + } else if (ScheduledState.DISABLED.equals(updatedScheduledState)) { + taskAction.setOperation(Operation.Disable); + } else { + // state is now stopped... consider the previous state + if (ScheduledState.RUNNING.equals(scheduledState)) { + taskAction.setOperation(Operation.Stop); + } else if (ScheduledState.DISABLED.equals(scheduledState)) { + taskAction.setOperation(Operation.Enable); + } + } + actions.add(taskAction); + } + + // ensure there are actions to record + if (!actions.isEmpty()) { + // save the actions + saveActions(actions, logger); + } + } + + return updatedReportingTask; + } + + /** + * Audits the removal of a reporting task via deleteReportingTask(). + * + * @param proceedingJoinPoint + * @param reportingTaskId + * @param reportingTaskDAO + * @throws Throwable + */ + @Around("within(org.apache.nifi.web.dao.ReportingTaskDAO+) && " + + "execution(void deleteReportingTask(java.lang.String)) && " + + "args(reportingTaskId) && " + + "target(reportingTaskDAO)") + public void removeReportingTaskAdvice(ProceedingJoinPoint proceedingJoinPoint, String reportingTaskId, ReportingTaskDAO reportingTaskDAO) throws Throwable { + // get the reporting task before removing it + ReportingTaskNode reportingTask = reportingTaskDAO.getReportingTask(reportingTaskId); + + // remove the reporting task + proceedingJoinPoint.proceed(); + + // if no exceptions were thrown, add removal actions... + // audit the reporting task removal + final Action action = generateAuditRecord(reportingTask, Operation.Remove); + + // save the actions + if (action != null) { + saveAction(action, logger); + } + } + + /** + * Generates an audit record for the creation of a reporting task. + * + * @param reportingTask + * @param operation + * @return + */ + public Action generateAuditRecord(ReportingTaskNode reportingTask, Operation operation) { + return generateAuditRecord(reportingTask, operation, null); + } + + /** + * Generates an audit record for the creation of a reporting task. + * + * @param reportingTask + * @param operation + * @param actionDetails + * @return + */ + public Action generateAuditRecord(ReportingTaskNode reportingTask, Operation operation, ActionDetails actionDetails) { + Action action = null; + + // get the current user + NiFiUser user = NiFiUserUtils.getNiFiUser(); + + // ensure the user was found + if (user != null) { + // create the reporting task details + ExtensionDetails taskDetails = new ExtensionDetails(); + taskDetails.setType(reportingTask.getReportingTask().getClass().getSimpleName()); + + // create the reporting task action for adding this reporting task + action = new Action(); + action.setUserDn(user.getDn()); + action.setUserName(user.getUserName()); + action.setOperation(operation); + action.setTimestamp(new Date()); + action.setSourceId(reportingTask.getIdentifier()); + action.setSourceName(reportingTask.getName()); + action.setSourceType(Component.ReportingTask); + action.setComponentDetails(taskDetails); + + if (actionDetails != null) { + action.setActionDetails(actionDetails); + } + } + + return action; + } + + /** + * Extracts the values for the configured properties from the specified ReportingTask. + * + * @param reportingTask + * @param reportingTaskDTO + * @return + */ + private Map extractConfiguredPropertyValues(ReportingTaskNode reportingTask, ReportingTaskDTO reportingTaskDTO) { + Map values = new HashMap<>(); + + if (reportingTaskDTO.getName() != null) { + values.put(NAME, reportingTask.getName()); + } + if (reportingTaskDTO.getAnnotationData() != null) { + values.put(ANNOTATION_DATA, reportingTask.getAnnotationData()); + } + if (reportingTaskDTO.getProperties() != null) { + // for each property specified, extract its configured value + Map properties = reportingTaskDTO.getProperties(); + Map configuredProperties = reportingTask.getProperties(); + for (String propertyName : properties.keySet()) { + // build a descriptor for getting the configured value + PropertyDescriptor propertyDescriptor = new PropertyDescriptor.Builder().name(propertyName).build(); + String configuredPropertyValue = configuredProperties.get(propertyDescriptor); + + // if the configured value couldn't be found, use the default value from the actual descriptor + if (configuredPropertyValue == null) { + propertyDescriptor = locatePropertyDescriptor(configuredProperties.keySet(), propertyDescriptor); + configuredPropertyValue = propertyDescriptor.getDefaultValue(); + } + values.put(propertyName, configuredPropertyValue); + } + } + + return values; + } + + /** + * Locates the actual property descriptor for the given spec property + * descriptor. + * + * @param propertyDescriptors + * @param specDescriptor + * @return + */ + private PropertyDescriptor locatePropertyDescriptor(Set propertyDescriptors, PropertyDescriptor specDescriptor) { + for (PropertyDescriptor propertyDescriptor : propertyDescriptors) { + if (propertyDescriptor.equals(specDescriptor)) { + return propertyDescriptor; + } + } + return specDescriptor; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java index 9c075e518a..1f8942b4f2 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java @@ -25,7 +25,7 @@ import java.util.Set; import org.apache.nifi.action.Action; import org.apache.nifi.action.Component; import org.apache.nifi.action.Operation; -import org.apache.nifi.action.component.details.ProcessorDetails; +import org.apache.nifi.action.component.details.ExtensionDetails; import org.apache.nifi.action.component.details.RemoteProcessGroupDetails; import org.apache.nifi.action.details.ConnectDetails; import org.apache.nifi.connectable.ConnectableType; @@ -94,7 +94,7 @@ public class SnippetAuditor extends NiFiAuditor { */ @Around("within(org.apache.nifi.web.dao.SnippetDAO+) && " + "execution(org.apache.nifi.web.api.dto.FlowSnippetDTO copySnippet(java.lang.String, java.lang.String, java.lang.Double, java.lang.Double))") - public Object copySnippetAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + public FlowSnippetDTO copySnippetAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { // perform the underlying operation FlowSnippetDTO snippet = (FlowSnippetDTO) proceedingJoinPoint.proceed(); auditSnippet(snippet); @@ -110,7 +110,7 @@ public class SnippetAuditor extends NiFiAuditor { */ @Around("within(org.apache.nifi.web.dao.TemplateDAO+) && " + "execution(org.apache.nifi.web.api.dto.FlowSnippetDTO instantiateTemplate(java.lang.String, java.lang.Double, java.lang.Double, java.lang.String))") - public Object instantiateTemplateAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + public FlowSnippetDTO instantiateTemplateAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { // perform the underlying operation FlowSnippetDTO snippet = (FlowSnippetDTO) proceedingJoinPoint.proceed(); auditSnippet(snippet); @@ -153,7 +153,7 @@ public class SnippetAuditor extends NiFiAuditor { // processors for (final ProcessorDTO processor : snippet.getProcessors()) { - final ProcessorDetails processorDetails = new ProcessorDetails(); + final ExtensionDetails processorDetails = new ExtensionDetails(); processorDetails.setType(StringUtils.substringAfterLast(processor.getType(), ".")); final Action action = generateAuditRecord(processor.getId(), processor.getName(), Component.Processor, Operation.Add, timestamp); @@ -256,15 +256,16 @@ public class SnippetAuditor extends NiFiAuditor { * Audits a bulk move. * * @param proceedingJoinPoint - * @param snippetId + * @param snippetDTO * @param snippetDAO + * @return * @throws Throwable */ @Around("within(org.apache.nifi.web.dao.SnippetDAO+) && " + "execution(org.apache.nifi.controller.Snippet updateSnippet(org.apache.nifi.web.api.dto.SnippetDTO)) && " + "args(snippetDTO) && " + "target(snippetDAO)") - public Object updateSnippetAdvice(ProceedingJoinPoint proceedingJoinPoint, SnippetDTO snippetDTO, SnippetDAO snippetDAO) throws Throwable { + public Snippet updateSnippetAdvice(ProceedingJoinPoint proceedingJoinPoint, SnippetDTO snippetDTO, SnippetDAO snippetDAO) throws Throwable { // get the snippet before removing it Snippet snippet = snippetDAO.getSnippet(snippetDTO.getId()); final String previousGroupId = snippet.getParentGroupId(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java index 63d302c1cb..8d9dade7b9 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java @@ -19,14 +19,17 @@ package org.apache.nifi.web; import java.util.Collection; import java.util.Date; import java.util.Set; +import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.repository.claim.ContentDirection; +import org.apache.nifi.controller.service.ControllerServiceState; import org.apache.nifi.web.api.dto.BulletinBoardDTO; import org.apache.nifi.web.api.dto.BulletinQueryDTO; import org.apache.nifi.web.api.dto.ClusterDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; import org.apache.nifi.web.api.dto.ControllerConfigurationDTO; import org.apache.nifi.web.api.dto.ControllerDTO; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.CounterDTO; import org.apache.nifi.web.api.dto.CountersDTO; import org.apache.nifi.web.api.dto.DocumentedTypeDTO; @@ -38,9 +41,12 @@ import org.apache.nifi.web.api.dto.NodeSystemDiagnosticsDTO; import org.apache.nifi.web.api.dto.PortDTO; import org.apache.nifi.web.api.dto.ProcessGroupDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; -import org.apache.nifi.web.api.dto.ProcessorHistoryDTO; +import org.apache.nifi.web.api.dto.ComponentHistoryDTO; +import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; import org.apache.nifi.web.api.dto.RevisionDTO; import org.apache.nifi.web.api.dto.SnippetDTO; import org.apache.nifi.web.api.dto.SystemDiagnosticsDTO; @@ -242,7 +248,22 @@ public interface NiFiServiceFacade { * @return The list of available processor types */ Set getProcessorTypes(); - + + /** + * Returns the list of controller service types. + * + * @param serviceType Filters only service types that implement this type + * @return The list of available controller types + */ + Set getControllerServiceTypes(String serviceType); + + /** + * Returns the list of reporting task types. + * + * @return The list of available reporting task types + */ + Set getReportingTaskTypes(); + /** * Returns the list of prioritizer types. * @@ -359,6 +380,16 @@ public interface NiFiServiceFacade { */ StatusHistoryDTO getProcessorStatusHistory(String groupId, String id); + /** + * Get the descriptor for the specified property of the specified processor. + * + * @param groupId + * @param id + * @param property + * @return + */ + PropertyDescriptorDTO getProcessorPropertyDescriptor(String groupId, String id, String property); + /** * Gets all the Processor transfer objects for this controller. * @@ -923,6 +954,172 @@ public interface NiFiServiceFacade { */ ConfigurationSnapshot deleteLabel(Revision revision, String groupId, String labelId); + // ---------------------------------------- + // Controller Services methods + // ---------------------------------------- + + /** + * Creates a controller service. + * + * @param revision Revision to compare with current base revision + * @param controllerServiceDTO The controller service DTO + * @return The controller service DTO + */ + ConfigurationSnapshot createControllerService(Revision revision, ControllerServiceDTO controllerServiceDTO); + + /** + * Gets all controller services. + * + * @return + */ + Set getControllerServices(); + + /** + * Gets the specified controller service. + * + * @param controllerServiceId + * @return + */ + ControllerServiceDTO getControllerService(String controllerServiceId); + + /** + * Get the descriptor for the specified property of the specified controller service. + * + * @param id + * @param property + * @return + */ + PropertyDescriptorDTO getControllerServicePropertyDescriptor(String id, String property); + + /** + * Gets the references for specified controller service. + * + * @param controllerServiceId + * @return + */ + Set getControllerServiceReferencingComponents(String controllerServiceId); + + /** + * Updates the referencing components for the specified controller service. + * + * @param revision + * @param controllerServiceId + * @param scheduledState + * @param controllerServiceState the value of state + * @return The referencing component dtos + */ + ConfigurationSnapshot> updateControllerServiceReferencingComponents(Revision revision, String controllerServiceId, ScheduledState scheduledState, ControllerServiceState controllerServiceState); + + /** + * Updates the specified label. + * + * @param revision Revision to compare with current base revision + * @param controllerServiceDTO The controller service DTO + * @return The controller service DTO + */ + ConfigurationSnapshot updateControllerService(Revision revision, ControllerServiceDTO controllerServiceDTO); + + /** + * Deletes the specified label. + * + * @param revision Revision to compare with current base revision + * @param controllerServiceId The controller service id + * @return + */ + ConfigurationSnapshot deleteControllerService(Revision revision, String controllerServiceId); + + /** + * Verifies the specified controller service can be updated. + * + * @param controllerServiceDTO + */ + void verifyUpdateControllerService(ControllerServiceDTO controllerServiceDTO); + + /** + * Verifies the referencing components of the specified controller service can be updated. + * + * @param controllerServiceId + * @param scheduledState + * @param controllerServiceState + */ + void verifyUpdateControllerServiceReferencingComponents(String controllerServiceId, ScheduledState scheduledState, ControllerServiceState controllerServiceState); + + /** + * Verifies the specified controller service can be removed. + * + * @param controllerServiceId + */ + void verifyDeleteControllerService(String controllerServiceId); + + // ---------------------------------------- + // Reporting Task methods + // ---------------------------------------- + + /** + * Creates a reporting task. + * + * @param revision Revision to compare with current base revision + * @param reportingTaskDTO The reporting task DTO + * @return The reporting task DTO + */ + ConfigurationSnapshot createReportingTask(Revision revision, ReportingTaskDTO reportingTaskDTO); + + /** + * Gets all reporting tasks. + * + * @return + */ + Set getReportingTasks(); + + /** + * Gets the specified reporting task. + * + * @param reportingTaskId + * @return + */ + ReportingTaskDTO getReportingTask(String reportingTaskId); + + /** + * Get the descriptor for the specified property of the specified reporting task. + * + * @param id + * @param property + * @return + */ + PropertyDescriptorDTO getReportingTaskPropertyDescriptor(String id, String property); + + /** + * Updates the specified reporting task. + * + * @param revision Revision to compare with current base revision + * @param reportingTaskDTO The reporting task DTO + * @return The reporting task DTO + */ + ConfigurationSnapshot updateReportingTask(Revision revision, ReportingTaskDTO reportingTaskDTO); + + /** + * Deletes the specified reporting task. + * + * @param revision Revision to compare with current base revision + * @param reportingTaskId The reporting task id + * @return + */ + ConfigurationSnapshot deleteReportingTask(Revision revision, String reportingTaskId); + + /** + * Verifies the specified reporting task can be updated. + * + * @param reportingTaskDTO + */ + void verifyUpdateReportingTask(ReportingTaskDTO reportingTaskDTO); + + /** + * Verifies the specified reporting task can be removed. + * + * @param reportingTaskId + */ + void verifyDeleteReportingTask(String reportingTaskId); + // ---------------------------------------- // History methods // ---------------------------------------- @@ -950,12 +1147,12 @@ public interface NiFiServiceFacade { void deleteActions(Date endDate); /** - * Gets the history for the specified property for the specified processor. + * Gets the history for the specified property for the specified component. * - * @param processorId + * @param componentId * @return */ - ProcessorHistoryDTO getProcessorHistory(String processorId); + ComponentHistoryDTO getComponentHistory(String componentId); // ---------------------------------------- // Snippet methods diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiContentAccess.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiContentAccess.java index 70aad94417..ed581433c4 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiContentAccess.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiContentAccess.java @@ -121,8 +121,14 @@ public class StandardNiFiContentAccess implements ContentAccess { final String rawDirection = StringUtils.substringAfterLast(eventDetails, "/content/"); // get the content type - final Long eventId = Long.parseLong(rawEventId); - final ContentDirection direction = ContentDirection.valueOf(rawDirection.toUpperCase()); + final Long eventId; + final ContentDirection direction; + try { + eventId = Long.parseLong(rawEventId); + direction = ContentDirection.valueOf(rawDirection.toUpperCase()); + } catch (final IllegalArgumentException iae) { + throw new IllegalArgumentException("The specified data reference URI is not valid."); + } return serviceFacade.getContent(eventId, request.getDataUri(), direction); } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java index 14640d8443..88637b4e03 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java @@ -106,7 +106,7 @@ import org.apache.nifi.web.api.dto.PreviousValueDTO; import org.apache.nifi.web.api.dto.ProcessGroupDTO; import org.apache.nifi.web.api.dto.ProcessorConfigDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; -import org.apache.nifi.web.api.dto.ProcessorHistoryDTO; +import org.apache.nifi.web.api.dto.ComponentHistoryDTO; import org.apache.nifi.web.api.dto.PropertyHistoryDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO; @@ -152,6 +152,19 @@ import org.apache.nifi.web.util.SnippetUtils; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.Validator; +import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceReference; +import org.apache.nifi.controller.service.ControllerServiceState; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; +import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; +import org.apache.nifi.web.dao.ControllerServiceDAO; +import org.apache.nifi.web.dao.ReportingTaskDAO; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.security.access.AccessDeniedException; @@ -162,8 +175,6 @@ import org.springframework.security.access.AccessDeniedException; public class StandardNiFiServiceFacade implements NiFiServiceFacade { private static final Logger logger = LoggerFactory.getLogger(StandardNiFiServiceFacade.class); - private static final String INVALID_REVISION_ERROR = "Given revision %s does not match current revision %s."; - private static final String SYNC_ERROR = "This NiFi instance has been updated by '%s'. Please refresh to synchronize the view."; // nifi core components private ControllerFacade controllerFacade; @@ -182,6 +193,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { private PortDAO inputPortDAO; private PortDAO outputPortDAO; private ConnectionDAO connectionDAO; + private ControllerServiceDAO controllerServiceDAO; + private ReportingTaskDAO reportingTaskDAO; private TemplateDAO templateDAO; // administrative services @@ -195,54 +208,10 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { private NiFiProperties properties; private DtoFactory dtoFactory; - /** - * Checks the specified revision against the current revision. - * - * @param revision The revision to check - * @param clientId The client id - * @return Whether or not the request should proceed - * @throws NiFiCoreException If the specified revision is not current - */ - private void checkRevision(Revision revision) { - - boolean approved = optimisticLockingManager.isCurrent(revision); - - if (!approved) { - Revision currentRevision = optimisticLockingManager.getRevision(); - logger.debug("Revision check failed because current revision is " + currentRevision + " but supplied revision is " + revision); - - if (StringUtils.isBlank(currentRevision.getClientId()) || currentRevision.getVersion() == null) { - throw new InvalidRevisionException(String.format(INVALID_REVISION_ERROR, revision, currentRevision)); - } else { - throw new InvalidRevisionException(String.format(SYNC_ERROR, optimisticLockingManager.getLastModifier())); - } - } - } - - /** - * Increments the revision and updates the last modifier. - * - * @param revision - * @return - */ - private Revision updateRevision(Revision revision) { - // update the client id and modifier - final Revision updatedRevision = optimisticLockingManager.incrementRevision(revision.getClientId()); - - // get the nifi user to extract the username - NiFiUser user = NiFiUserUtils.getNiFiUser(); - if (user == null) { - optimisticLockingManager.setLastModifier("unknown"); - } else { - optimisticLockingManager.setLastModifier(user.getUserName()); - } - - return updatedRevision; - } - // ----------------------------------------- // Verification Operations // ----------------------------------------- + @Override public void verifyCreateConnection(String groupId, ConnectionDTO connectionDTO) { connectionDAO.verifyCreate(groupId, connectionDTO); @@ -360,100 +329,123 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { remoteProcessGroupDAO.verifyDelete(groupId, remoteProcessGroupId); } + @Override + public void verifyUpdateControllerService(ControllerServiceDTO controllerServiceDTO) { + // if service does not exist, then the update request is likely creating it + // so we don't verify since it will fail + if (controllerServiceDAO.hasControllerService(controllerServiceDTO.getId())) { + controllerServiceDAO.verifyUpdate(controllerServiceDTO); + } + } + + @Override + public void verifyUpdateControllerServiceReferencingComponents(String controllerServiceId, ScheduledState scheduledState, ControllerServiceState controllerServiceState) { + controllerServiceDAO.verifyUpdateReferencingComponents(controllerServiceId, scheduledState, controllerServiceState); + } + + @Override + public void verifyDeleteControllerService(String controllerServiceId) { + controllerServiceDAO.verifyDelete(controllerServiceId); + } + + @Override + public void verifyUpdateReportingTask(ReportingTaskDTO reportingTaskDTO) { + // if tasks does not exist, then the update request is likely creating it + // so we don't verify since it will fail + if (reportingTaskDAO.hasReportingTask(reportingTaskDTO.getId())) { + reportingTaskDAO.verifyUpdate(reportingTaskDTO); + } + } + + @Override + public void verifyDeleteReportingTask(String reportingTaskId) { + reportingTaskDAO.verifyDelete(reportingTaskId); + } + // ----------------------------------------- // Write Operations // ----------------------------------------- + @Override - public ConfigurationSnapshot updateConnection(Revision revision, String groupId, ConnectionDTO connectionDTO) { - - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateConnection(final Revision revision, final String groupId, final ConnectionDTO connectionDTO) { // if connection does not exist, then create new connection if (connectionDAO.hasConnection(groupId, connectionDTO.getId()) == false) { return createConnection(revision, groupId, connectionDTO); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ConnectionDTO execute() { + final Connection connection = connectionDAO.updateConnection(groupId, connectionDTO); - final Connection connection = connectionDAO.updateConnection(groupId, connectionDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createConnectionDto(connection)); - - // save the flow - controllerFacade.save(); - - return response; + controllerFacade.save(); + + return dtoFactory.createConnectionDto(connection); + } + }); } @Override - public ConfigurationSnapshot updateProcessor(Revision revision, String groupId, ProcessorDTO processorDTO) { - - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateProcessor(final Revision revision, final String groupId, final ProcessorDTO processorDTO) { // if processor does not exist, then create new processor if (processorDAO.hasProcessor(groupId, processorDTO.getId()) == false) { return createProcessor(revision, groupId, processorDTO); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ProcessorDTO execute() { + // update the processor + ProcessorNode processor = processorDAO.updateProcessor(groupId, processorDTO); - // update the processor - ProcessorNode processor = processorDAO.updateProcessor(groupId, processorDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createProcessorDto(processor)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createProcessorDto(processor); + } + }); } @Override - public ConfigurationSnapshot updateLabel(Revision revision, String groupId, LabelDTO labelDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateLabel(final Revision revision, final String groupId, final LabelDTO labelDTO) { // if label does not exist, then create new label if (labelDAO.hasLabel(groupId, labelDTO.getId()) == false) { return createLabel(revision, groupId, labelDTO); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public LabelDTO execute() { + // update the existing label + final Label label = labelDAO.updateLabel(groupId, labelDTO); - // update the existing label - final Label label = labelDAO.updateLabel(groupId, labelDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createLabelDto(label)); - - // save updated controller - controllerFacade.save(); - - return response; + // save updated controller + controllerFacade.save(); + + return dtoFactory.createLabelDto(label); + } + }); } @Override - public ConfigurationSnapshot updateFunnel(Revision revision, String groupId, FunnelDTO funnelDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateFunnel(final Revision revision, final String groupId, final FunnelDTO funnelDTO) { // if label does not exist, then create new label if (funnelDAO.hasFunnel(groupId, funnelDTO.getId()) == false) { return createFunnel(revision, groupId, funnelDTO); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public FunnelDTO execute() { + // update the existing label + final Funnel funnel = funnelDAO.updateFunnel(groupId, funnelDTO); - // update the existing label - final Funnel funnel = funnelDAO.updateFunnel(groupId, funnelDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createFunnelDto(funnel)); - - // save updated controller - controllerFacade.save(); - - return response; + // save updated controller + controllerFacade.save(); + + return dtoFactory.createFunnelDto(funnel); + } + }); } @Override @@ -466,141 +458,126 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { } @Override - public ConfigurationSnapshot updateSnippet(Revision revision, SnippetDTO snippetDto) { - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateSnippet(final Revision revision, final SnippetDTO snippetDto) { // if label does not exist, then create new label if (snippetDAO.hasSnippet(snippetDto.getId()) == false) { return createSnippet(revision, snippetDto); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public SnippetDTO execute() { + // update the snippet + final Snippet snippet = snippetDAO.updateSnippet(snippetDto); - // update the snippet - final Snippet snippet = snippetDAO.updateSnippet(snippetDto); + // build the snippet dto + final SnippetDTO responseSnippetDto = dtoFactory.createSnippetDto(snippet); + responseSnippetDto.setContents(snippetUtils.populateFlowSnippet(snippet, false)); - // build the snippet dto - final SnippetDTO responseSnippetDto = dtoFactory.createSnippetDto(snippet); - responseSnippetDto.setContents(snippetUtils.populateFlowSnippet(snippet, false)); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), responseSnippetDto); - - // save updated controller if applicable - if (snippetDto.getParentGroupId() != null && snippet.isLinked()) { - controllerFacade.save(); - } - - return response; + // save updated controller if applicable + if (snippetDto.getParentGroupId() != null && snippet.isLinked()) { + controllerFacade.save(); + } + + return responseSnippetDto; + } + }); } @Override - public ConfigurationSnapshot updateInputPort(Revision revision, String groupId, PortDTO inputPortDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateInputPort(final Revision revision, final String groupId, final PortDTO inputPortDTO) { // if input port does not exist, then create new input port if (inputPortDAO.hasPort(groupId, inputPortDTO.getId()) == false) { return createInputPort(revision, groupId, inputPortDTO); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public PortDTO execute() { + final Port inputPort = inputPortDAO.updatePort(groupId, inputPortDTO); - final Port inputPort = inputPortDAO.updatePort(groupId, inputPortDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createPortDto(inputPort)); - - // save updated controller - controllerFacade.save(); - - return response; + // save updated controller + controllerFacade.save(); + + return dtoFactory.createPortDto(inputPort); + } + }); } @Override - public ConfigurationSnapshot updateOutputPort(Revision revision, String groupId, PortDTO outputPortDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateOutputPort(final Revision revision, final String groupId, final PortDTO outputPortDTO) { // if output port does not exist, then create new output port if (outputPortDAO.hasPort(groupId, outputPortDTO.getId()) == false) { return createOutputPort(revision, groupId, outputPortDTO); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public PortDTO execute() { + final Port outputPort = outputPortDAO.updatePort(groupId, outputPortDTO); - final Port outputPort = outputPortDAO.updatePort(groupId, outputPortDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createPortDto(outputPort)); - - // save updated controller - controllerFacade.save(); - - return response; + // save updated controller + controllerFacade.save(); + + return dtoFactory.createPortDto(outputPort); + } + }); } @Override - public ConfigurationSnapshot updateRemoteProcessGroup(Revision revision, String groupId, RemoteProcessGroupDTO remoteProcessGroupDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateRemoteProcessGroup(final Revision revision, final String groupId, final RemoteProcessGroupDTO remoteProcessGroupDTO) { // if controller reference does not exist, then create new controller reference if (remoteProcessGroupDAO.hasRemoteProcessGroup(groupId, remoteProcessGroupDTO.getId()) == false) { return createRemoteProcessGroup(revision, groupId, remoteProcessGroupDTO); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public RemoteProcessGroupDTO execute() { + RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.updateRemoteProcessGroup(groupId, remoteProcessGroupDTO); - RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.updateRemoteProcessGroup(groupId, remoteProcessGroupDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createRemoteProcessGroupDto(remoteProcessGroup)); - - // save updated controller - controllerFacade.save(); - - return response; + // save updated controller + controllerFacade.save(); + + return dtoFactory.createRemoteProcessGroupDto(remoteProcessGroup); + } + }); } @Override - public ConfigurationSnapshot updateRemoteProcessGroupInputPort(Revision revision, String groupId, String remoteProcessGroupId, RemoteProcessGroupPortDTO remoteProcessGroupPortDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot updateRemoteProcessGroupInputPort(final Revision revision, final String groupId, final String remoteProcessGroupId, final RemoteProcessGroupPortDTO remoteProcessGroupPortDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public RemoteProcessGroupPortDTO execute() { + // update the remote port + RemoteGroupPort remoteGroupPort = remoteProcessGroupDAO.updateRemoteProcessGroupInputPort(groupId, remoteProcessGroupId, remoteProcessGroupPortDTO); - // update the remote port - RemoteGroupPort remoteGroupPort = remoteProcessGroupDAO.updateRemoteProcessGroupInputPort(groupId, remoteProcessGroupId, remoteProcessGroupPortDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createRemoteProcessGroupPortDto(remoteGroupPort)); - - // save updated controller - controllerFacade.save(); - - return response; + // save updated controller + controllerFacade.save(); + + return dtoFactory.createRemoteProcessGroupPortDto(remoteGroupPort); + } + }); } @Override - public ConfigurationSnapshot updateRemoteProcessGroupOutputPort(Revision revision, String groupId, String remoteProcessGroupId, RemoteProcessGroupPortDTO remoteProcessGroupPortDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot updateRemoteProcessGroupOutputPort(final Revision revision, final String groupId, final String remoteProcessGroupId, final RemoteProcessGroupPortDTO remoteProcessGroupPortDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public RemoteProcessGroupPortDTO execute() { + // update the remote port + RemoteGroupPort remoteGroupPort = remoteProcessGroupDAO.updateRemoteProcessGroupOutputPort(groupId, remoteProcessGroupId, remoteProcessGroupPortDTO); - // update the remote port - RemoteGroupPort remoteGroupPort = remoteProcessGroupDAO.updateRemoteProcessGroupOutputPort(groupId, remoteProcessGroupId, remoteProcessGroupPortDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createRemoteProcessGroupPortDto(remoteGroupPort)); - - // save updated controller - controllerFacade.save(); - - return response; + // save updated controller + controllerFacade.save(); + + return dtoFactory.createRemoteProcessGroupPortDto(remoteGroupPort); + } + }); } @Override - public ConfigurationSnapshot updateProcessGroup(Revision revision, String parentGroupId, ProcessGroupDTO processGroupDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateProcessGroup(final Revision revision, final String parentGroupId, final ProcessGroupDTO processGroupDTO) { // if process group does not exist, then create new process group if (processGroupDAO.hasProcessGroup(processGroupDTO.getId()) == false) { if (parentGroupId == null) { @@ -609,50 +586,49 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { return createProcessGroup(parentGroupId, revision, processGroupDTO); } } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ProcessGroupDTO execute() { + // update the process group + ProcessGroup processGroup = processGroupDAO.updateProcessGroup(processGroupDTO); - // update the process group - ProcessGroup processGroup = processGroupDAO.updateProcessGroup(processGroupDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createProcessGroupDto(processGroup)); - - // save updated controller - controllerFacade.save(); - - return response; + // save updated controller + controllerFacade.save(); + + return dtoFactory.createProcessGroupDto(processGroup); + } + }); } @Override - public ConfigurationSnapshot updateControllerConfiguration(Revision revision, ControllerConfigurationDTO controllerConfigurationDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot updateControllerConfiguration(final Revision revision, final ControllerConfigurationDTO controllerConfigurationDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ControllerConfigurationDTO execute() { + // update the controller configuration through the proxy + if (controllerConfigurationDTO.getName() != null) { + controllerFacade.setName(controllerConfigurationDTO.getName()); + } + if (controllerConfigurationDTO.getComments() != null) { + controllerFacade.setComments(controllerConfigurationDTO.getComments()); + } + if (controllerConfigurationDTO.getMaxTimerDrivenThreadCount() != null) { + controllerFacade.setMaxTimerDrivenThreadCount(controllerConfigurationDTO.getMaxTimerDrivenThreadCount()); + } + if (controllerConfigurationDTO.getMaxEventDrivenThreadCount() != null) { + controllerFacade.setMaxEventDrivenThreadCount(controllerConfigurationDTO.getMaxEventDrivenThreadCount()); + } - // update the controller configuration through the proxy - if (controllerConfigurationDTO.getName() != null) { - controllerFacade.setName(controllerConfigurationDTO.getName()); - } - if (controllerConfigurationDTO.getComments() != null) { - controllerFacade.setComments(controllerConfigurationDTO.getComments()); - } - if (controllerConfigurationDTO.getMaxTimerDrivenThreadCount() != null) { - controllerFacade.setMaxTimerDrivenThreadCount(controllerConfigurationDTO.getMaxTimerDrivenThreadCount()); - } - if (controllerConfigurationDTO.getMaxEventDrivenThreadCount() != null) { - controllerFacade.setMaxEventDrivenThreadCount(controllerConfigurationDTO.getMaxEventDrivenThreadCount()); - } + // create the controller configuration dto + ControllerConfigurationDTO controllerConfig = getControllerConfiguration(); - // create the controller configuration dto - ControllerConfigurationDTO controllerConfig = getControllerConfiguration(); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), controllerConfig); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return controllerConfig; + } + }); } @Override @@ -685,74 +661,66 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { } @Override - public ConfigurationSnapshot deleteConnection(Revision revision, String groupId, String connectionId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteConnection(final Revision revision, final String groupId, final String connectionId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest(){ + @Override + public Void execute() { + connectionDAO.deleteConnection(groupId, connectionId); - connectionDAO.deleteConnection(groupId, connectionId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return null; + } + }); } @Override - public ConfigurationSnapshot deleteProcessor(Revision revision, String groupId, String processorId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteProcessor(final Revision revision, final String groupId, final String processorId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + // delete the processor and synchronize the connection state + processorDAO.deleteProcessor(groupId, processorId); - // delete the processor and synchronize the connection state - processorDAO.deleteProcessor(groupId, processorId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return null; + } + }); } @Override - public ConfigurationSnapshot deleteLabel(Revision revision, String groupId, String labelId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteLabel(final Revision revision, final String groupId, final String labelId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + // delete the label + labelDAO.deleteLabel(groupId, labelId); - // delete the label - labelDAO.deleteLabel(groupId, labelId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return null; + } + }); } @Override - public ConfigurationSnapshot deleteFunnel(Revision revision, String groupId, String funnelId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteFunnel(final Revision revision, final String groupId, final String funnelId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + // delete the label + funnelDAO.deleteFunnel(groupId, funnelId); - // delete the label - funnelDAO.deleteFunnel(groupId, funnelId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return null; + } + }); } @Override @@ -761,95 +729,85 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { } @Override - public ConfigurationSnapshot deleteSnippet(Revision revision, String snippetId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteSnippet(final Revision revision, final String snippetId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + // determine if this snippet was linked to the data flow + Snippet snippet = snippetDAO.getSnippet(snippetId); + boolean linked = snippet.isLinked(); - // determine if this snippet was linked to the data flow - Snippet snippet = snippetDAO.getSnippet(snippetId); - boolean linked = snippet.isLinked(); + // delete the snippet + snippetDAO.deleteSnippet(snippetId); - // delete the snippet - snippetDAO.deleteSnippet(snippetId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow if necessary - if (linked) { - controllerFacade.save(); - } - - return response; + // save the flow if necessary + if (linked) { + controllerFacade.save(); + } + + return null; + } + }); } @Override - public ConfigurationSnapshot deleteInputPort(Revision revision, String groupId, String inputPortId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteInputPort(final Revision revision, final String groupId, final String inputPortId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + inputPortDAO.deletePort(groupId, inputPortId); - inputPortDAO.deletePort(groupId, inputPortId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return null; + } + }); } @Override - public ConfigurationSnapshot deleteOutputPort(Revision revision, String groupId, String outputPortId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteOutputPort(final Revision revision, final String groupId, final String outputPortId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + outputPortDAO.deletePort(groupId, outputPortId); - outputPortDAO.deletePort(groupId, outputPortId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return null; + } + }); } @Override - public ConfigurationSnapshot deleteProcessGroup(Revision revision, String groupId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteProcessGroup(final Revision revision, final String groupId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + processGroupDAO.deleteProcessGroup(groupId); - processGroupDAO.deleteProcessGroup(groupId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return null; + } + }); } @Override - public ConfigurationSnapshot deleteRemoteProcessGroup(Revision revision, String groupId, String remoteProcessGroupId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteRemoteProcessGroup(final Revision revision, final String groupId, final String remoteProcessGroupId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + remoteProcessGroupDAO.deleteRemoteProcessGroup(groupId, remoteProcessGroupId); - remoteProcessGroupDAO.deleteRemoteProcessGroup(groupId, remoteProcessGroupId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return null; + } + }); } @Override @@ -859,97 +817,86 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { } @Override - public ConfigurationSnapshot createConnection(Revision revision, String groupId, ConnectionDTO connectionDTO) { + public ConfigurationSnapshot createConnection(final Revision revision, final String groupId, final ConnectionDTO connectionDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ConnectionDTO execute() { + // ensure id is set + if (StringUtils.isBlank(connectionDTO.getId())) { + connectionDTO.setId(UUID.randomUUID().toString()); + } - // ensure the proper revision before performing the update - checkRevision(revision); + final Connection connection = connectionDAO.createConnection(groupId, connectionDTO); - // ensure id is set - if (StringUtils.isBlank(connectionDTO.getId())) { - connectionDTO.setId(UUID.randomUUID().toString()); - } - - final Connection connection = connectionDAO.createConnection(groupId, connectionDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createConnectionDto(connection)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createConnectionDto(connection); + } + }); } @Override - public ConfigurationSnapshot createProcessor(Revision revision, String groupId, ProcessorDTO processorDTO) { + public ConfigurationSnapshot createProcessor(final Revision revision, final String groupId, final ProcessorDTO processorDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ProcessorDTO execute() { + // ensure id is set + if (StringUtils.isBlank(processorDTO.getId())) { + processorDTO.setId(UUID.randomUUID().toString()); + } - // ensure the proper revision before performing the update - checkRevision(revision); + // create the processor + final ProcessorNode processor = processorDAO.createProcessor(groupId, processorDTO); - // ensure id is set - if (StringUtils.isBlank(processorDTO.getId())) { - processorDTO.setId(UUID.randomUUID().toString()); - } - - // create the processor - final ProcessorNode processor = processorDAO.createProcessor(groupId, processorDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createProcessorDto(processor)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createProcessorDto(processor); + } + }); } @Override - public ConfigurationSnapshot createLabel(Revision revision, String groupId, LabelDTO labelDTO) { + public ConfigurationSnapshot createLabel(final Revision revision, final String groupId, final LabelDTO labelDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public LabelDTO execute() { + // ensure id is set + if (StringUtils.isBlank(labelDTO.getId())) { + labelDTO.setId(UUID.randomUUID().toString()); + } - // ensure the proper revision before performing the update - checkRevision(revision); + // add the label + final Label label = labelDAO.createLabel(groupId, labelDTO); - // ensure id is set - if (StringUtils.isBlank(labelDTO.getId())) { - labelDTO.setId(UUID.randomUUID().toString()); - } - - // add the label - final Label label = labelDAO.createLabel(groupId, labelDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createLabelDto(label)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createLabelDto(label); + } + }); } @Override - public ConfigurationSnapshot createFunnel(Revision revision, String groupId, FunnelDTO funnelDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot createFunnel(final Revision revision, final String groupId, final FunnelDTO funnelDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public FunnelDTO execute() { + // ensure id is set + if (StringUtils.isBlank(funnelDTO.getId())) { + funnelDTO.setId(UUID.randomUUID().toString()); + } - // ensure id is set - if (StringUtils.isBlank(funnelDTO.getId())) { - funnelDTO.setId(UUID.randomUUID().toString()); - } + // add the label + final Funnel funnel = funnelDAO.createFunnel(groupId, funnelDTO); - // add the label - final Funnel funnel = funnelDAO.createFunnel(groupId, funnelDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createFunnelDto(funnel)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createFunnelDto(funnel); + } + }); } private void validateSnippetContents(final FlowSnippetDTO flowSnippet, final String groupId) { @@ -1008,139 +955,129 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { } @Override - public ConfigurationSnapshot copySnippet(Revision revision, String groupId, String snippetId, Double originX, Double originY) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot copySnippet(final Revision revision, final String groupId, final String snippetId, final Double originX, final Double originY) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public FlowSnippetDTO execute() { + String id = snippetId; + + // ensure id is set + if (StringUtils.isBlank(id)) { + id = UUID.randomUUID().toString(); + } - // ensure id is set - if (StringUtils.isBlank(snippetId)) { - snippetId = UUID.randomUUID().toString(); - } + // create the new snippet + FlowSnippetDTO flowSnippet = snippetDAO.copySnippet(groupId, id, originX, originY); - // create the new snippet - FlowSnippetDTO flowSnippet = snippetDAO.copySnippet(groupId, snippetId, originX, originY); + // validate the new snippet + validateSnippetContents(flowSnippet, groupId); - // validate the new snippet - validateSnippetContents(flowSnippet, groupId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), flowSnippet); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return flowSnippet; + } + }); } @Override public ConfigurationSnapshot createSnippet(final Revision revision, final SnippetDTO snippetDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public SnippetDTO execute() { + // ensure id is set + if (StringUtils.isBlank(snippetDTO.getId())) { + snippetDTO.setId(UUID.randomUUID().toString()); + } - // ensure id is set - if (StringUtils.isBlank(snippetDTO.getId())) { - snippetDTO.setId(UUID.randomUUID().toString()); - } - - // add the snippet - final Snippet snippet = snippetDAO.createSnippet(snippetDTO); - final SnippetDTO responseSnippetDTO = dtoFactory.createSnippetDto(snippet); - responseSnippetDTO.setContents(snippetUtils.populateFlowSnippet(snippet, false)); - - // create the response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), responseSnippetDTO); - - return response; + // add the snippet + final Snippet snippet = snippetDAO.createSnippet(snippetDTO); + final SnippetDTO responseSnippetDTO = dtoFactory.createSnippetDto(snippet); + responseSnippetDTO.setContents(snippetUtils.populateFlowSnippet(snippet, false)); + + return responseSnippetDTO; + } + }); } @Override - public ConfigurationSnapshot createInputPort(Revision revision, String groupId, PortDTO inputPortDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot createInputPort(final Revision revision, final String groupId, final PortDTO inputPortDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public PortDTO execute() { + // ensure id is set + if (StringUtils.isBlank(inputPortDTO.getId())) { + inputPortDTO.setId(UUID.randomUUID().toString()); + } - // ensure id is set - if (StringUtils.isBlank(inputPortDTO.getId())) { - inputPortDTO.setId(UUID.randomUUID().toString()); - } + final Port inputPort = inputPortDAO.createPort(groupId, inputPortDTO); - final Port inputPort = inputPortDAO.createPort(groupId, inputPortDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createPortDto(inputPort)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createPortDto(inputPort); + } + }); } @Override - public ConfigurationSnapshot createOutputPort(Revision revision, String groupId, PortDTO outputPortDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot createOutputPort(final Revision revision, final String groupId, final PortDTO outputPortDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public PortDTO execute() { + // ensure id is set + if (StringUtils.isBlank(outputPortDTO.getId())) { + outputPortDTO.setId(UUID.randomUUID().toString()); + } - // ensure id is set - if (StringUtils.isBlank(outputPortDTO.getId())) { - outputPortDTO.setId(UUID.randomUUID().toString()); - } + final Port outputPort = outputPortDAO.createPort(groupId, outputPortDTO); - final Port outputPort = outputPortDAO.createPort(groupId, outputPortDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createPortDto(outputPort)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createPortDto(outputPort); + } + }); } @Override - public ConfigurationSnapshot createProcessGroup(String parentGroupId, Revision revision, ProcessGroupDTO processGroupDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot createProcessGroup(final String parentGroupId, final Revision revision, final ProcessGroupDTO processGroupDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ProcessGroupDTO execute() { + // ensure id is set + if (StringUtils.isBlank(processGroupDTO.getId())) { + processGroupDTO.setId(UUID.randomUUID().toString()); + } - // ensure id is set - if (StringUtils.isBlank(processGroupDTO.getId())) { - processGroupDTO.setId(UUID.randomUUID().toString()); - } + final ProcessGroup processGroup = processGroupDAO.createProcessGroup(parentGroupId, processGroupDTO); - final ProcessGroup processGroup = processGroupDAO.createProcessGroup(parentGroupId, processGroupDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createProcessGroupDto(processGroup)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createProcessGroupDto(processGroup); + } + }); } @Override - public ConfigurationSnapshot createRemoteProcessGroup(Revision revision, String groupId, RemoteProcessGroupDTO remoteProcessGroupDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot createRemoteProcessGroup(final Revision revision, final String groupId, final RemoteProcessGroupDTO remoteProcessGroupDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public RemoteProcessGroupDTO execute() { + // ensure id is set + if (StringUtils.isBlank(remoteProcessGroupDTO.getId())) { + remoteProcessGroupDTO.setId(UUID.randomUUID().toString()); + } - // ensure id is set - if (StringUtils.isBlank(remoteProcessGroupDTO.getId())) { - remoteProcessGroupDTO.setId(UUID.randomUUID().toString()); - } + final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.createRemoteProcessGroup(groupId, remoteProcessGroupDTO); - final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.createRemoteProcessGroup(groupId, remoteProcessGroupDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createRemoteProcessGroupDto(remoteProcessGroup)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createRemoteProcessGroupDto(remoteProcessGroup); + } + }); } @Override @@ -1186,74 +1123,217 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { } @Override - public ConfigurationSnapshot createTemplateInstance(Revision revision, String groupId, Double originX, Double originY, String templateId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot createTemplateInstance(final Revision revision, final String groupId, final Double originX, final Double originY, final String templateId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public FlowSnippetDTO execute() { + // instantiate the template - there is no need to make another copy of the flow snippet since the actual template + // was copied and this dto is only used to instantiate it's components (which as already completed) + FlowSnippetDTO flowSnippet = templateDAO.instantiateTemplate(groupId, originX, originY, templateId); - // instantiate the template - there is no need to make another copy of the flow snippet since the actual template - // was copied and this dto is only used to instantiate it's components (which as already completed) - FlowSnippetDTO flowSnippet = templateDAO.instantiateTemplate(groupId, originX, originY, templateId); + // validate the new snippet + validateSnippetContents(flowSnippet, groupId); - // validate the new snippet - validateSnippetContents(flowSnippet, groupId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), flowSnippet); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return flowSnippet; + } + }); } @Override - public ConfigurationSnapshot createArchive(Revision revision) { - // ensure the proper revision before performing the update - checkRevision(revision); - - // create the archive - controllerFacade.createArchive(); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - return response; + public ConfigurationSnapshot createArchive(final Revision revision) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + // create the archive + controllerFacade.createArchive(); + return null; + } + }); } @Override - public ConfigurationSnapshot setProcessorAnnotationData(Revision revision, String processorId, String annotationData) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot setProcessorAnnotationData(final Revision revision, final String processorId, final String annotationData) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ProcessorDTO execute() { + // create the processor config + final ProcessorConfigDTO config = new ProcessorConfigDTO(); + config.setAnnotationData(annotationData); - // create the processor config - final ProcessorConfigDTO config = new ProcessorConfigDTO(); - config.setAnnotationData(annotationData); + // create the processor dto + final ProcessorDTO processorDTO = new ProcessorDTO(); + processorDTO.setId(processorId); + processorDTO.setConfig(config); - // create the processor dto - final ProcessorDTO processorDTO = new ProcessorDTO(); - processorDTO.setId(processorId); - processorDTO.setConfig(config); + // get the parent group id for the specified processor + String groupId = controllerFacade.findProcessGroupIdForProcessor(processorId); - // get the parent group id for the specified processor - String groupId = controllerFacade.findProcessGroupIdForProcessor(processorId); + // ensure the parent group id was found + if (groupId == null) { + throw new ResourceNotFoundException(String.format("Unable to locate Processor with id '%s'.", processorId)); + } - // ensure the parent group id was found - if (groupId == null) { - throw new ResourceNotFoundException(String.format("Unable to locate Processor with id '%s'.", processorId)); + // update the processor configuration + ProcessorNode processor = processorDAO.updateProcessor(groupId, processorDTO); + + // save the flow + controllerFacade.save(); + + return dtoFactory.createProcessorDto(processor); + } + }); + } + + @Override + public ConfigurationSnapshot createControllerService(final Revision revision, final ControllerServiceDTO controllerServiceDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ControllerServiceDTO execute() { + // ensure id is set + if (StringUtils.isBlank(controllerServiceDTO.getId())) { + controllerServiceDTO.setId(UUID.randomUUID().toString()); + } + + // create the controller service + final ControllerServiceNode controllerService = controllerServiceDAO.createControllerService(controllerServiceDTO); + + // save the update + if (properties.isClusterManager()) { + clusterManager.saveControllerServices(); + } else { + controllerFacade.save(); + } + + return dtoFactory.createControllerServiceDto(controllerService); + } + }); + } + + @Override + public ConfigurationSnapshot updateControllerService(final Revision revision, final ControllerServiceDTO controllerServiceDTO) { + // if controller service does not exist, then create new controller service + if (controllerServiceDAO.hasControllerService(controllerServiceDTO.getId()) == false) { + return createControllerService(revision, controllerServiceDTO); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ControllerServiceDTO execute() { + final ControllerServiceNode controllerService = controllerServiceDAO.updateControllerService(controllerServiceDTO); - // update the processor configuration - ProcessorNode processor = processorDAO.updateProcessor(groupId, processorDTO); + // save the update + if (properties.isClusterManager()) { + clusterManager.saveControllerServices(); + } else { + controllerFacade.save(); + } - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createProcessorDto(processor)); + return dtoFactory.createControllerServiceDto(controllerService); + } + }); + } - // save the flow - controllerFacade.save(); + @Override + public ConfigurationSnapshot> updateControllerServiceReferencingComponents(final Revision revision, final String controllerServiceId, final org.apache.nifi.controller.ScheduledState scheduledState, final org.apache.nifi.controller.service.ControllerServiceState controllerServiceState) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest>() { + @Override + public Set execute() { + final ControllerServiceReference reference = controllerServiceDAO.updateControllerServiceReferencingComponents(controllerServiceId, scheduledState, controllerServiceState); + return dtoFactory.createControllerServiceReferencingComponentsDto(reference); + } + }); + } - return response; + @Override + public ConfigurationSnapshot deleteControllerService(final Revision revision, final String controllerServiceId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + // delete the label + controllerServiceDAO.deleteControllerService(controllerServiceId); + + // save the update + if (properties.isClusterManager()) { + clusterManager.saveControllerServices(); + } else { + controllerFacade.save(); + } + + return null; + } + }); + } + + @Override + public ConfigurationSnapshot createReportingTask(final Revision revision, final ReportingTaskDTO reportingTaskDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ReportingTaskDTO execute() { + // ensure id is set + if (StringUtils.isBlank(reportingTaskDTO.getId())) { + reportingTaskDTO.setId(UUID.randomUUID().toString()); + } + + // create the reporting + final ReportingTaskNode reportingTask = reportingTaskDAO.createReportingTask(reportingTaskDTO); + + // save the update + if (properties.isClusterManager()) { + clusterManager.saveReportingTasks(); + } else { + controllerFacade.save(); + } + + return dtoFactory.createReportingTaskDto(reportingTask); + } + }); + } + + @Override + public ConfigurationSnapshot updateReportingTask(final Revision revision, final ReportingTaskDTO reportingTaskDTO) { + // if reporting task does not exist, then create new reporting task + if (reportingTaskDAO.hasReportingTask(reportingTaskDTO.getId()) == false) { + return createReportingTask(revision, reportingTaskDTO); + } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ReportingTaskDTO execute() { + final ReportingTaskNode reportingTask = reportingTaskDAO.updateReportingTask(reportingTaskDTO); + + // save the update + if (properties.isClusterManager()) { + clusterManager.saveReportingTasks(); + } else { + controllerFacade.save(); + } + + return dtoFactory.createReportingTaskDto(reportingTask); + } + }); + } + + @Override + public ConfigurationSnapshot deleteReportingTask(final Revision revision, final String reportingTaskId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + // delete the label + reportingTaskDAO.deleteReportingTask(reportingTaskId); + + // save the update + if (properties.isClusterManager()) { + clusterManager.saveReportingTasks(); + } else { + controllerFacade.save(); + } + + return null; + } + }); } @Override @@ -1408,9 +1488,10 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { // ----------------------------------------- // Read Operations // ----------------------------------------- + @Override public RevisionDTO getRevision() { - return dtoFactory.createRevisionDTO(optimisticLockingManager.getRevision()); + return dtoFactory.createRevisionDTO(optimisticLockingManager.getLastModification()); } @Override @@ -1636,6 +1717,16 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { return controllerFacade.getFlowFileProcessorTypes(); } + @Override + public Set getControllerServiceTypes(final String serviceType) { + return controllerFacade.getControllerServiceTypes(serviceType); + } + + @Override + public Set getReportingTaskTypes() { + return controllerFacade.getReportingTaskTypes(); + } + @Override public ProcessorDTO getProcessor(String groupId, String id) { final ProcessorNode processor = processorDAO.getProcessor(groupId, id); @@ -1643,6 +1734,19 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { return processorDto; } + @Override + public PropertyDescriptorDTO getProcessorPropertyDescriptor(String groupId, String id, String property) { + final ProcessorNode processor = processorDAO.getProcessor(groupId, id); + PropertyDescriptor descriptor = processor.getPropertyDescriptor(property); + + // return an invalid descriptor if the processor doesn't suppor this property + if (descriptor == null) { + descriptor = new PropertyDescriptor.Builder().name(property).addValidator(Validator.INVALID).dynamic(true).build(); + } + + return dtoFactory.createPropertyDescriptorDto(descriptor); + } + @Override public StatusHistoryDTO getProcessorStatusHistory(String groupId, String id) { return controllerFacade.getProcessorStatusHistory(groupId, id); @@ -1823,6 +1927,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { final Date now = new Date(); controllerConfig.setTimeOffset(TimeZone.getDefault().getOffset(now.getTime())); + controllerConfig.setCurrentTime(now); // determine the site to site configuration if (isClustered()) { @@ -1929,11 +2034,71 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { @Override public ConfigurationSnapshot getProcessGroup(String groupId, final boolean recurse) { ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId); - Long version = optimisticLockingManager.getRevision().getVersion(); - ConfigurationSnapshot response = new ConfigurationSnapshot<>(version, dtoFactory.createProcessGroupDto(processGroup, recurse)); + Revision revision = optimisticLockingManager.getLastModification().getRevision(); + ConfigurationSnapshot response = new ConfigurationSnapshot<>(revision.getVersion(), dtoFactory.createProcessGroupDto(processGroup, recurse)); return response; } + @Override + public Set getControllerServices() { + final Set controllerServiceDtos = new LinkedHashSet<>(); + for (ControllerServiceNode controllerService : controllerServiceDAO.getControllerServices()) { + controllerServiceDtos.add(dtoFactory.createControllerServiceDto(controllerService)); + } + return controllerServiceDtos; + } + + @Override + public ControllerServiceDTO getControllerService(String controllerServiceId) { + return dtoFactory.createControllerServiceDto(controllerServiceDAO.getControllerService(controllerServiceId)); + } + + @Override + public PropertyDescriptorDTO getControllerServicePropertyDescriptor(String id, String property) { + final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(id); + PropertyDescriptor descriptor = controllerService.getControllerServiceImplementation().getPropertyDescriptor(property); + + // return an invalid descriptor if the controller service doesn't support this property + if (descriptor == null) { + descriptor = new PropertyDescriptor.Builder().name(property).addValidator(Validator.INVALID).dynamic(true).build(); + } + + return dtoFactory.createPropertyDescriptorDto(descriptor); + } + + @Override + public Set getControllerServiceReferencingComponents(String controllerServiceId) { + final ControllerServiceNode service = controllerServiceDAO.getControllerService(controllerServiceId); + return dtoFactory.createControllerServiceReferencingComponentsDto(service.getReferences()); + } + + @Override + public Set getReportingTasks() { + final Set reportingTaskDtos = new LinkedHashSet<>(); + for (ReportingTaskNode reportingTask : reportingTaskDAO.getReportingTasks()) { + reportingTaskDtos.add(dtoFactory.createReportingTaskDto(reportingTask)); + } + return reportingTaskDtos; + } + + @Override + public ReportingTaskDTO getReportingTask(String reportingTaskId) { + return dtoFactory.createReportingTaskDto(reportingTaskDAO.getReportingTask(reportingTaskId)); + } + + @Override + public PropertyDescriptorDTO getReportingTaskPropertyDescriptor(String id, String property) { + final ReportingTaskNode reportingTask = reportingTaskDAO.getReportingTask(id); + PropertyDescriptor descriptor = reportingTask.getReportingTask().getPropertyDescriptor(property); + + // return an invalid descriptor if the reporting task doesn't support this property + if (descriptor == null) { + descriptor = new PropertyDescriptor.Builder().name(property).addValidator(Validator.INVALID).dynamic(true).build(); + } + + return dtoFactory.createPropertyDescriptorDto(descriptor); + } + @Override public StatusHistoryDTO getProcessGroupStatusHistory(String groupId) { return controllerFacade.getProcessGroupStatusHistory(groupId); @@ -1974,9 +2139,9 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { } @Override - public ProcessorHistoryDTO getProcessorHistory(String processorId) { + public ComponentHistoryDTO getComponentHistory(String componentId) { final Map propertyHistoryDtos = new LinkedHashMap<>(); - final Map> propertyHistory = auditService.getPreviousValues(processorId); + final Map> propertyHistory = auditService.getPreviousValues(componentId); for (final Map.Entry> entry : propertyHistory.entrySet()) { final List previousValueDtos = new ArrayList<>(); @@ -1996,8 +2161,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { } } - final ProcessorHistoryDTO history = new ProcessorHistoryDTO(); - history.setProcessorId(processorId); + final ComponentHistoryDTO history = new ComponentHistoryDTO(); + history.setComponentId(componentId); history.setPropertyHistory(propertyHistoryDtos); return history; @@ -2718,6 +2883,14 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { this.processGroupDAO = processGroupDAO; } + public void setControllerServiceDAO(ControllerServiceDAO controllerServiceDAO) { + this.controllerServiceDAO = controllerServiceDAO; + } + + public void setReportingTaskDAO(ReportingTaskDAO reportingTaskDAO) { + this.reportingTaskDAO = reportingTaskDAO; + } + public void setTemplateDAO(TemplateDAO templateDAO) { this.templateDAO = templateDAO; } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java new file mode 100644 index 0000000000..8d51a584f5 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java @@ -0,0 +1,736 @@ +/* + * 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; + +import java.io.Serializable; +import java.io.UnsupportedEncodingException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URLEncoder; +import java.util.Collection; +import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; + +import javax.ws.rs.HttpMethod; +import javax.ws.rs.core.MultivaluedMap; +import javax.ws.rs.core.Response; + +import org.apache.nifi.action.Action; +import org.apache.nifi.action.Component; +import org.apache.nifi.action.Operation; +import org.apache.nifi.action.component.details.ExtensionDetails; +import org.apache.nifi.action.details.ConfigureDetails; +import org.apache.nifi.admin.service.AuditService; +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.manager.impl.WebClusterManager; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.web.security.user.NiFiUserDetails; +import org.apache.nifi.web.security.user.NiFiUserUtils; +import org.apache.nifi.user.NiFiUser; +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.api.dto.ProcessorConfigDTO; +import org.apache.nifi.web.api.dto.ProcessorDTO; +import org.apache.nifi.web.api.dto.RevisionDTO; +import org.apache.nifi.web.api.entity.ProcessorEntity; +import org.apache.nifi.web.util.WebUtils; + +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.security.access.prepost.PreAuthorize; +import org.springframework.security.core.Authentication; +import org.springframework.security.core.context.SecurityContextHolder; + +import com.sun.jersey.core.util.MultivaluedMapImpl; +import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.controller.reporting.ReportingTaskProvider; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; +import org.apache.nifi.web.api.entity.ControllerServiceEntity; +import org.apache.nifi.web.api.entity.ReportingTaskEntity; +import org.apache.nifi.web.util.ClientResponseUtils; + +/** + * Implements the NiFiWebConfigurationContext interface to support a context in both + * standalone and clustered environments. + */ +public class StandardNiFiWebConfigurationContext implements NiFiWebConfigurationContext { + + private static final Logger logger = LoggerFactory.getLogger(StandardNiFiWebConfigurationContext.class); + public static final String CLIENT_ID_PARAM = "clientId"; + public static final String REVISION_PARAM = "revision"; + public static final String VERBOSE_PARAM = "verbose"; + + private NiFiProperties properties; + private NiFiServiceFacade serviceFacade; + private WebClusterManager clusterManager; + private ControllerServiceLookup controllerServiceLookup; + private ReportingTaskProvider reportingTaskProvider; + private AuditService auditService; + + @Override + public ControllerService getControllerService(String serviceIdentifier) { + return controllerServiceLookup.getControllerService(serviceIdentifier); + } + + @Override + @PreAuthorize("hasAnyRole('ROLE_DFM')") + public void saveActions(final NiFiWebRequestContext requestContext, final Collection configurationActions) { + Objects.requireNonNull(configurationActions, "Actions cannot be null."); + + // ensure the path could be + if (requestContext.getExtensionType() == null) { + throw new IllegalArgumentException("The UI extension type must be specified."); + } + + Component componentType = null; + switch (requestContext.getExtensionType()) { + case ProcessorConfiguration: + componentType = Component.Processor; + break; + case ControllerServiceConfiguration: + componentType = Component.ControllerService; + break; + case ReportingTaskConfiguration: + componentType = Component.ReportingTask; + break; + } + + if (componentType == null) { + throw new IllegalArgumentException("UI extension type must support Processor, ControllerService, or ReportingTask configuration."); + } + + // - when running standalone or cluster ncm - actions from custom UIs are stored locally + // - clustered nodes do not serve custom UIs directly to users so they should never be invoking this method + final Date now = new Date(); + final Collection actions = new HashSet<>(configurationActions.size()); + for (final ConfigurationAction configurationAction : configurationActions) { + final ExtensionDetails extensionDetails = new ExtensionDetails(); + extensionDetails.setType(configurationAction.getType()); + + final ConfigureDetails configureDetails = new ConfigureDetails(); + configureDetails.setName(configurationAction.getName()); + configureDetails.setPreviousValue(configurationAction.getPreviousValue()); + configureDetails.setValue(configurationAction.getValue()); + + final Action action = new Action(); + action.setTimestamp(now); + action.setSourceId(configurationAction.getId()); + action.setSourceName(configurationAction.getName()); + action.setSourceType(componentType); + action.setOperation(Operation.Configure); + action.setUserDn(getCurrentUserDn()); + action.setUserName(getCurrentUserName()); + action.setComponentDetails(extensionDetails); + action.setActionDetails(configureDetails); + actions.add(action); + } + + if (!actions.isEmpty()) { + try { + // record the operations + auditService.addActions(actions); + } catch (Throwable t) { + logger.warn("Unable to record actions: " + t.getMessage()); + if (logger.isDebugEnabled()) { + logger.warn(StringUtils.EMPTY, t); + } + } + } + } + + @Override + public String getCurrentUserDn() { + String userDn = NiFiUser.ANONYMOUS_USER_DN; + + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + if (user != null) { + userDn = user.getDn(); + } + + return userDn; + } + + @Override + public String getCurrentUserName() { + String userName = NiFiUser.ANONYMOUS_USER_DN; + + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + if (user != null) { + userName = user.getUserName(); + } + + return userName; + } + + @Override + public ComponentDetails getComponentDetails(final NiFiWebRequestContext requestContext) throws ResourceNotFoundException, ClusterRequestException { + final String id = requestContext.getId(); + + if (StringUtils.isBlank(id)) { + throw new ResourceNotFoundException(String.format("Configuration request context config did not have a component ID.")); + } + + // ensure the path could be + if (requestContext.getExtensionType() == null) { + throw new IllegalArgumentException("The UI extension type must be specified."); + } + + // get the component facade for interacting directly with that type of object + ComponentFacade componentFacade = null; + switch (requestContext.getExtensionType()) { + case ProcessorConfiguration: + componentFacade = new ProcessorFacade(); + break; + case ControllerServiceConfiguration: + componentFacade = new ControllerServiceFacade(); + break; + case ReportingTaskConfiguration: + componentFacade = new ReportingTaskFacade(); + break; + } + + if (componentFacade == null) { + throw new IllegalArgumentException("UI extension type must support Processor, ControllerService, or ReportingTask configuration."); + } + + return componentFacade.getComponentDetails(requestContext); + } + + @Override + @PreAuthorize("hasAnyRole('ROLE_DFM')") + public ComponentDetails setAnnotationData(final NiFiWebConfigurationRequestContext requestContext, final String annotationData) + throws ResourceNotFoundException, InvalidRevisionException, ClusterRequestException { + + final String id = requestContext.getId(); + + if (StringUtils.isBlank(id)) { + throw new ResourceNotFoundException(String.format("Configuration request context did not have a component ID.")); + } + + // ensure the path could be + if (requestContext.getExtensionType() == null) { + throw new IllegalArgumentException("The UI extension type must be specified."); + } + + // get the component facade for interacting directly with that type of object + ComponentFacade componentFacade = null; + switch (requestContext.getExtensionType()) { + case ProcessorConfiguration: + componentFacade = new ProcessorFacade(); + break; + case ControllerServiceConfiguration: + componentFacade = new ControllerServiceFacade(); + break; + case ReportingTaskConfiguration: + componentFacade = new ReportingTaskFacade(); + break; + } + + if (componentFacade == null) { + throw new IllegalArgumentException("UI extension type must support Processor, ControllerService, or ReportingTask configuration."); + } + + return componentFacade.setAnnotationData(requestContext, annotationData); + } + + /** + * Facade over accessing different types of NiFi components. + */ + private interface ComponentFacade { + /** + * Gets the component details using the specified request context. + * + * @param requestContext + * @return + */ + ComponentDetails getComponentDetails(NiFiWebRequestContext requestContext); + + /** + * Sets the annotation data using the specified request context. + * + * @param requestContext + * @param annotationData + * @return + */ + ComponentDetails setAnnotationData(NiFiWebConfigurationRequestContext requestContext, String annotationData); + } + + /** + * Interprets the request/response with the underlying Processor model. + */ + private class ProcessorFacade implements ComponentFacade { + @Override + public ComponentDetails getComponentDetails(final NiFiWebRequestContext requestContext) { + final String id = requestContext.getId(); + + final ProcessorDTO processor; + if (properties.isClusterManager()) { + // create the request URL + URI requestUrl; + try { + String path = "/nifi-api/cluster/processors/" + URLEncoder.encode(id, "UTF-8"); + requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null); + } catch (final URISyntaxException | UnsupportedEncodingException use) { + throw new ClusterRequestException(use); + } + + // set the request parameters + MultivaluedMap parameters = new MultivaluedMapImpl(); + parameters.add(VERBOSE_PARAM, "true"); + + // replicate request + NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.GET, requestUrl, parameters, getHeaders(requestContext)); + + // check for issues replicating request + checkResponse(nodeResponse, id); + + // return processor + ProcessorEntity entity = (ProcessorEntity) nodeResponse.getUpdatedEntity(); + if (entity == null) { + entity = nodeResponse.getClientResponse().getEntity(ProcessorEntity.class); + } + processor = entity.getProcessor(); + } else { + processor = serviceFacade.getProcessor(id); + } + + // return the processor info + return getComponentConfiguration(processor); + } + + @Override + public ComponentDetails setAnnotationData(final NiFiWebConfigurationRequestContext requestContext, final String annotationData) { + final Revision revision = requestContext.getRevision(); + final String id = requestContext.getId(); + + final ProcessorDTO processor; + if (properties.isClusterManager()) { + // create the request URL + URI requestUrl; + try { + String path = "/nifi-api/cluster/processors/" + URLEncoder.encode(id, "UTF-8"); + requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null); + } catch (final URISyntaxException | UnsupportedEncodingException use) { + throw new ClusterRequestException(use); + } + + // create the revision + RevisionDTO revisionDto = new RevisionDTO(); + revisionDto.setClientId(revision.getClientId()); + revisionDto.setVersion(revision.getVersion()); + + // create the processor entity + ProcessorEntity processorEntity = new ProcessorEntity(); + processorEntity.setRevision(revisionDto); + + // create the processor dto + ProcessorDTO processorDto = new ProcessorDTO(); + processorEntity.setProcessor(processorDto); + processorDto.setId(id); + + // create the processor configuration with the given annotation data + ProcessorConfigDTO configDto = new ProcessorConfigDTO(); + processorDto.setConfig(configDto); + configDto.setAnnotationData(annotationData); + + // set the content type to json + final Map headers = getHeaders(requestContext); + headers.put("Content-Type", "application/json"); + + // replicate request + NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.PUT, requestUrl, processorEntity, headers); + + // check for issues replicating request + checkResponse(nodeResponse, id); + + // return processor + ProcessorEntity entity = (ProcessorEntity) nodeResponse.getUpdatedEntity(); + if (entity == null) { + entity = nodeResponse.getClientResponse().getEntity(ProcessorEntity.class); + } + processor = entity.getProcessor(); + } else { + final ConfigurationSnapshot response = serviceFacade.setProcessorAnnotationData(revision, id, annotationData); + processor = response.getConfiguration(); + } + + // return the processor info + return getComponentConfiguration(processor); + } + + private ComponentDetails getComponentConfiguration(final ProcessorDTO processor) { + final ProcessorConfigDTO processorConfig = processor.getConfig(); + return new ComponentDetails.Builder() + .id(processor.getId()) + .name(processor.getName()) + .type(processor.getType()) + .state(processor.getState()) + .annotationData(processorConfig.getAnnotationData()) + .properties(processorConfig.getProperties()) + .validateErrors(processor.getValidationErrors()).build(); + } + } + + /** + * Interprets the request/response with the underlying ControllerService model. + */ + private class ControllerServiceFacade implements ComponentFacade { + @Override + public ComponentDetails getComponentDetails(final NiFiWebRequestContext requestContext) { + final String id = requestContext.getId(); + final ControllerServiceDTO controllerService; + + // if the lookup has the service that means we are either a node or + // the ncm and the service is available there only + if (controllerServiceLookup.getControllerService(id) != null) { + controllerService = serviceFacade.getControllerService(id); + } else { + // if this is a standalone instance the service should have been found above... there should + // no cluster to replicate the request to + if (!properties.isClusterManager()) { + throw new ResourceNotFoundException(String.format("Controller service[%s] could not be found on this NiFi.", id)); + } + + // create the request URL + URI requestUrl; + try { + String path = "/nifi-api/controller/controller-services/node/" + URLEncoder.encode(id, "UTF-8"); + requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null); + } catch (final URISyntaxException | UnsupportedEncodingException use) { + throw new ClusterRequestException(use); + } + + // set the request parameters + MultivaluedMap parameters = new MultivaluedMapImpl(); + + // replicate request + NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.GET, requestUrl, parameters, getHeaders(requestContext)); + + // check for issues replicating request + checkResponse(nodeResponse, id); + + // return controller service + ControllerServiceEntity entity = (ControllerServiceEntity) nodeResponse.getUpdatedEntity(); + if (entity == null) { + entity = nodeResponse.getClientResponse().getEntity(ControllerServiceEntity.class); + } + controllerService = entity.getControllerService(); + } + + // return the controller service info + return getComponentConfiguration(controllerService); + } + + @Override + public ComponentDetails setAnnotationData(final NiFiWebConfigurationRequestContext requestContext, final String annotationData) { + final Revision revision = requestContext.getRevision(); + final String id = requestContext.getId(); + + final ControllerServiceDTO controllerService; + if (controllerServiceLookup.getControllerService(id) != null) { + final ControllerServiceDTO controllerServiceDto = new ControllerServiceDTO(); + controllerServiceDto.setId(id); + controllerServiceDto.setAnnotationData(annotationData); + + final ConfigurationSnapshot response = serviceFacade.updateControllerService(revision, controllerServiceDto); + controllerService = response.getConfiguration(); + } else { + // if this is a standalone instance the service should have been found above... there should + // no cluster to replicate the request to + if (!properties.isClusterManager()) { + throw new ResourceNotFoundException(String.format("Controller service[%s] could not be found on this NiFi.", id)); + } + + // since this PUT request can be interpreted as a request to create a controller service + // we need to be sure that this service exists on the node before the request is replicated. + // this is done by attempting to get the details. if the service doesn't exist it will + // throw a ResourceNotFoundException + getComponentDetails(requestContext); + + // create the request URL + URI requestUrl; + try { + String path = "/nifi-api/controller/controller-services/node/" + URLEncoder.encode(id, "UTF-8"); + requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null); + } catch (final URISyntaxException | UnsupportedEncodingException use) { + throw new ClusterRequestException(use); + } + + // create the revision + RevisionDTO revisionDto = new RevisionDTO(); + revisionDto.setClientId(revision.getClientId()); + revisionDto.setVersion(revision.getVersion()); + + // create the controller service entity + ControllerServiceEntity controllerServiceEntity = new ControllerServiceEntity(); + controllerServiceEntity.setRevision(revisionDto); + + // create the controller service dto + ControllerServiceDTO controllerServiceDto = new ControllerServiceDTO(); + controllerServiceEntity.setControllerService(controllerServiceDto); + controllerServiceDto.setId(id); + controllerServiceDto.setAnnotationData(annotationData); + + // set the content type to json + final Map headers = getHeaders(requestContext); + headers.put("Content-Type", "application/json"); + + // replicate request + NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.PUT, requestUrl, controllerServiceEntity, headers); + + // check for issues replicating request + checkResponse(nodeResponse, id); + + // return controller service + ControllerServiceEntity entity = (ControllerServiceEntity) nodeResponse.getUpdatedEntity(); + if (entity == null) { + entity = nodeResponse.getClientResponse().getEntity(ControllerServiceEntity.class); + } + controllerService = entity.getControllerService(); + } + + // return the controller service info + return getComponentConfiguration(controllerService); + } + + private ComponentDetails getComponentConfiguration(final ControllerServiceDTO controllerService) { + return new ComponentDetails.Builder() + .id(controllerService.getId()) + .name(controllerService.getName()) + .type(controllerService.getType()) + .state(controllerService.getState()) + .annotationData(controllerService.getAnnotationData()) + .properties(controllerService.getProperties()) + .validateErrors(controllerService.getValidationErrors()).build(); + } + } + + /** + * Interprets the request/response with the underlying ControllerService model. + */ + private class ReportingTaskFacade implements ComponentFacade { + @Override + public ComponentDetails getComponentDetails(final NiFiWebRequestContext requestContext) { + final String id = requestContext.getId(); + final ReportingTaskDTO reportingTask; + + // if the provider has the service that means we are either a node or + // the ncm and the service is available there only + if (reportingTaskProvider.getReportingTaskNode(id) != null) { + reportingTask = serviceFacade.getReportingTask(id); + } else { + // if this is a standalone instance the task should have been found above... there should + // no cluster to replicate the request to + if (!properties.isClusterManager()) { + throw new ResourceNotFoundException(String.format("Reporting task[%s] could not be found on this NiFi.", id)); + } + + // create the request URL + URI requestUrl; + try { + String path = "/nifi-api/controller/reporting-tasks/node/" + URLEncoder.encode(id, "UTF-8"); + requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null); + } catch (final URISyntaxException | UnsupportedEncodingException use) { + throw new ClusterRequestException(use); + } + + // set the request parameters + MultivaluedMap parameters = new MultivaluedMapImpl(); + + // replicate request + NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.GET, requestUrl, parameters, getHeaders(requestContext)); + + // check for issues replicating request + checkResponse(nodeResponse, id); + + // return reporting task + ReportingTaskEntity entity = (ReportingTaskEntity) nodeResponse.getUpdatedEntity(); + if (entity == null) { + entity = nodeResponse.getClientResponse().getEntity(ReportingTaskEntity.class); + } + reportingTask = entity.getReportingTask(); + } + + // return the reporting task info + return getComponentConfiguration(reportingTask); + } + + @Override + public ComponentDetails setAnnotationData(final NiFiWebConfigurationRequestContext requestContext, final String annotationData) { + final Revision revision = requestContext.getRevision(); + final String id = requestContext.getId(); + + final ReportingTaskDTO reportingTask; + if (reportingTaskProvider.getReportingTaskNode(id) != null) { + final ReportingTaskDTO reportingTaskDto = new ReportingTaskDTO(); + reportingTaskDto.setId(id); + reportingTaskDto.setAnnotationData(annotationData); + + final ConfigurationSnapshot response = serviceFacade.updateReportingTask(revision, reportingTaskDto); + reportingTask = response.getConfiguration(); + } else { + // if this is a standalone instance the task should have been found above... there should + // no cluster to replicate the request to + if (!properties.isClusterManager()) { + throw new ResourceNotFoundException(String.format("Reporting task[%s] could not be found on this NiFi.", id)); + } + + // since this PUT request can be interpreted as a request to create a reporting task + // we need to be sure that this task exists on the node before the request is replicated. + // this is done by attempting to get the details. if the service doesn't exist it will + // throw a ResourceNotFoundException + getComponentDetails(requestContext); + + // create the request URL + URI requestUrl; + try { + String path = "/nifi-api/controller/reporting-tasks/node/" + URLEncoder.encode(id, "UTF-8"); + requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null); + } catch (final URISyntaxException | UnsupportedEncodingException use) { + throw new ClusterRequestException(use); + } + + // create the revision + RevisionDTO revisionDto = new RevisionDTO(); + revisionDto.setClientId(revision.getClientId()); + revisionDto.setVersion(revision.getVersion()); + + // create the reporting task entity + ReportingTaskEntity reportingTaskEntity = new ReportingTaskEntity(); + reportingTaskEntity.setRevision(revisionDto); + + // create the reporting task dto + ReportingTaskDTO reportingTaskDto = new ReportingTaskDTO(); + reportingTaskEntity.setReportingTask(reportingTaskDto); + reportingTaskDto.setId(id); + reportingTaskDto.setAnnotationData(annotationData); + + // set the content type to json + final Map headers = getHeaders(requestContext); + headers.put("Content-Type", "application/json"); + + // replicate request + NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.PUT, requestUrl, reportingTaskEntity, headers); + + // check for issues replicating request + checkResponse(nodeResponse, id); + + // return reporting task + ReportingTaskEntity entity = (ReportingTaskEntity) nodeResponse.getUpdatedEntity(); + if (entity == null) { + entity = nodeResponse.getClientResponse().getEntity(ReportingTaskEntity.class); + } + reportingTask = entity.getReportingTask(); + } + + // return the processor info + return getComponentConfiguration(reportingTask); + } + + private ComponentDetails getComponentConfiguration(final ReportingTaskDTO reportingTask) { + return new ComponentDetails.Builder() + .id(reportingTask.getId()) + .name(reportingTask.getName()) + .type(reportingTask.getType()) + .state(reportingTask.getState()) + .annotationData(reportingTask.getAnnotationData()) + .properties(reportingTask.getProperties()) + .validateErrors(reportingTask.getValidationErrors()).build(); + } + } + + /** + * Gets the headers for the request to replicate to each node while + * clustered. + * + * @param config + * @return + */ + private Map getHeaders(final NiFiWebRequestContext config) { + final Map headers = new HashMap<>(); + headers.put("Accept", "application/json,application/xml"); + if (StringUtils.isNotBlank(config.getProxiedEntitiesChain())) { + headers.put("X-ProxiedEntitiesChain", config.getProxiedEntitiesChain()); + } + + // add the user's authorities (if any) to the headers + final Authentication authentication = SecurityContextHolder.getContext().getAuthentication(); + if (authentication != null) { + final Object userDetailsObj = authentication.getPrincipal(); + if (userDetailsObj instanceof NiFiUserDetails) { + // serialize user details object + final String hexEncodedUserDetails = WebUtils.serializeObjectToHex((Serializable) userDetailsObj); + + // put serialized user details in header + headers.put("X-ProxiedEntityUserDetails", hexEncodedUserDetails); + } + } + return headers; + } + + /** + * Checks the specified response and drains the stream appropriately. + * + * @param nodeResponse + * @param revision + * @param id + */ + private void checkResponse(final NodeResponse nodeResponse, final String id) { + if (nodeResponse.hasThrowable()) { + ClientResponseUtils.drainClientResponse(nodeResponse.getClientResponse()); + throw new ClusterRequestException(nodeResponse.getThrowable()); + } else if (nodeResponse.getClientResponse().getStatus() == Response.Status.CONFLICT.getStatusCode()) { + ClientResponseUtils.drainClientResponse(nodeResponse.getClientResponse()); + throw new InvalidRevisionException(String.format("Conflict: the flow may have been updated by another user.")); + } else if (nodeResponse.getClientResponse().getStatus() == Response.Status.NOT_FOUND.getStatusCode()) { + ClientResponseUtils.drainClientResponse(nodeResponse.getClientResponse()); + throw new ResourceNotFoundException("Unable to find component with id: " + id); + } else if (nodeResponse.getClientResponse().getStatus() != Response.Status.OK.getStatusCode()) { + ClientResponseUtils.drainClientResponse(nodeResponse.getClientResponse()); + throw new ClusterRequestException("Method resulted in an unsuccessful HTTP response code: " + nodeResponse.getClientResponse().getStatus()); + } + } + + public void setClusterManager(WebClusterManager clusterManager) { + this.clusterManager = clusterManager; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } + + public void setServiceFacade(NiFiServiceFacade serviceFacade) { + this.serviceFacade = serviceFacade; + } + + public void setAuditService(AuditService auditService) { + this.auditService = auditService; + } + + public void setControllerServiceLookup(ControllerServiceLookup controllerServiceLookup) { + this.controllerServiceLookup = controllerServiceLookup; + } + + public void setReportingTaskProvider(ReportingTaskProvider reportingTaskProvider) { + this.reportingTaskProvider = reportingTaskProvider; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java index d9fa9e3016..eaf457ed45 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java @@ -35,7 +35,7 @@ import javax.ws.rs.core.Response; import org.apache.nifi.action.Action; import org.apache.nifi.action.Component; import org.apache.nifi.action.Operation; -import org.apache.nifi.action.component.details.ProcessorDetails; +import org.apache.nifi.action.component.details.ExtensionDetails; import org.apache.nifi.action.details.ConfigureDetails; import org.apache.nifi.admin.service.AuditService; import org.apache.nifi.cluster.manager.NodeResponse; @@ -49,7 +49,6 @@ import org.apache.nifi.web.api.dto.ProcessorConfigDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; import org.apache.nifi.web.api.dto.RevisionDTO; import org.apache.nifi.web.api.entity.ProcessorEntity; -import org.apache.nifi.web.controller.ControllerFacade; import org.apache.nifi.web.util.WebUtils; import org.apache.commons.lang3.StringUtils; @@ -60,12 +59,14 @@ import org.springframework.security.core.Authentication; import org.springframework.security.core.context.SecurityContextHolder; import com.sun.jersey.core.util.MultivaluedMapImpl; +import org.apache.nifi.controller.ControllerServiceLookup; import org.apache.nifi.web.util.ClientResponseUtils; /** * Implements the NiFiWebContext interface to support a context in both * standalone and clustered environments. */ +@Deprecated public class StandardNiFiWebContext implements NiFiWebContext { private static final Logger logger = LoggerFactory.getLogger(StandardNiFiWebContext.class); @@ -76,16 +77,12 @@ public class StandardNiFiWebContext implements NiFiWebContext { private NiFiProperties properties; private NiFiServiceFacade serviceFacade; private WebClusterManager clusterManager; - private ControllerFacade controllerFacade; + private ControllerServiceLookup controllerServiceLookup; private AuditService auditService; @Override public ControllerService getControllerService(String serviceIdentifier) { - if (properties.isClusterManager()) { - return clusterManager.getControllerService(serviceIdentifier); - } else { - return controllerFacade.getControllerService(serviceIdentifier); - } + return controllerServiceLookup.getControllerService(serviceIdentifier); } @Override @@ -98,7 +95,7 @@ public class StandardNiFiWebContext implements NiFiWebContext { final Date now = new Date(); final Collection actions = new HashSet<>(processorActions.size()); for (final ProcessorConfigurationAction processorAction : processorActions) { - final ProcessorDetails processorDetails = new ProcessorDetails(); + final ExtensionDetails processorDetails = new ExtensionDetails(); processorDetails.setType(processorAction.getProcessorType()); final ConfigureDetails configureDetails = new ConfigureDetails(); @@ -199,7 +196,10 @@ public class StandardNiFiWebContext implements NiFiWebContext { } // return processor - final ProcessorEntity entity = nodeResponse.getClientResponse().getEntity(ProcessorEntity.class); + ProcessorEntity entity = (ProcessorEntity) nodeResponse.getUpdatedEntity(); + if (entity == null) { + entity = nodeResponse.getClientResponse().getEntity(ProcessorEntity.class); + } processor = entity.getProcessor(); } else { processor = serviceFacade.getProcessor(processorId); @@ -325,12 +325,12 @@ public class StandardNiFiWebContext implements NiFiWebContext { this.serviceFacade = serviceFacade; } - public void setControllerFacade(ControllerFacade controllerFacade) { - this.controllerFacade = controllerFacade; - } - public void setAuditService(AuditService auditService) { this.auditService = auditService; } + public void setControllerServiceLookup(ControllerServiceLookup controllerServiceLookup) { + this.controllerServiceLookup = controllerServiceLookup; + } + } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java index 1b9ae7def2..787fffaf45 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java @@ -178,53 +178,55 @@ public abstract class ApplicationResource { // get cluster context from threadlocal ClusterContext clusterCtx = ClusterContextThreadLocal.getContext(); + if (clusterCtx != null) { + + // serialize cluster context + String serializedClusterContext = WebUtils.serializeObjectToHex(clusterCtx); + if (serializedClusterContext.length() > CLUSTER_CONTEXT_HEADER_VALUE_MAX_BYTES) { + /* + * Actions is the only field that can vary in size. If we have no + * actions and we exceeded the header size, then basic assumptions + * about the cluster context have been violated. + */ + if (clusterCtx.getActions().isEmpty()) { + throw new IllegalStateException( + String.format("Serialized Cluster context size '%d' is too big for response header", serializedClusterContext.length())); + } - // serialize cluster context - String serializedClusterContext = WebUtils.serializeObjectToHex(clusterCtx); - if (serializedClusterContext.length() > CLUSTER_CONTEXT_HEADER_VALUE_MAX_BYTES) { - /* - * Actions is the only field that can vary in size. If we have no - * actions and we exceeded the header size, then basic assumptions - * about the cluster context have been violated. - */ - if (clusterCtx.getActions().isEmpty()) { - throw new IllegalStateException( - String.format("Serialized Cluster context size '%d' is too big for response header", serializedClusterContext.length())); + // use the first action as the prototype for creating the "batch" action + Action prototypeAction = clusterCtx.getActions().get(0); + + // log the batched actions + StringBuilder loggedActions = new StringBuilder(); + createBatchedActionLogStatement(loggedActions, clusterCtx.getActions()); + logger.info(loggedActions.toString()); + + // remove current actions and replace with batch action + clusterCtx.getActions().clear(); + + // create the batch action + Action batchAction = new Action(); + batchAction.setOperation(Operation.Batch); + + // copy values from prototype action + batchAction.setTimestamp(prototypeAction.getTimestamp()); + batchAction.setUserDn(prototypeAction.getUserDn()); + batchAction.setUserName(prototypeAction.getUserName()); + batchAction.setSourceId(prototypeAction.getSourceId()); + batchAction.setSourceName(prototypeAction.getSourceName()); + batchAction.setSourceType(prototypeAction.getSourceType()); + + // add batch action + clusterCtx.getActions().add(batchAction); + + // create the final serialized copy of the cluster context + serializedClusterContext = WebUtils.serializeObjectToHex(clusterCtx); } - // use the first action as the prototype for creating the "batch" action - Action prototypeAction = clusterCtx.getActions().get(0); - - // log the batched actions - StringBuilder loggedActions = new StringBuilder(); - createBatchedActionLogStatement(loggedActions, clusterCtx.getActions()); - logger.info(loggedActions.toString()); - - // remove current actions and replace with batch action - clusterCtx.getActions().clear(); - - // create the batch action - Action batchAction = new Action(); - batchAction.setOperation(Operation.Batch); - - // copy values from prototype action - batchAction.setTimestamp(prototypeAction.getTimestamp()); - batchAction.setUserDn(prototypeAction.getUserDn()); - batchAction.setUserName(prototypeAction.getUserName()); - batchAction.setSourceId(prototypeAction.getSourceId()); - batchAction.setSourceName(prototypeAction.getSourceName()); - batchAction.setSourceType(prototypeAction.getSourceType()); - - // add batch action - clusterCtx.getActions().add(batchAction); - - // create the final serialized copy of the cluster context - serializedClusterContext = WebUtils.serializeObjectToHex(clusterCtx); + // put serialized cluster context in response header + response.header(WebClusterManager.CLUSTER_CONTEXT_HTTP_HEADER, serializedClusterContext); } - // put serialized cluster context in response header - response.header(WebClusterManager.CLUSTER_CONTEXT_HTTP_HEADER, serializedClusterContext); - return response; } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ClusterResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ClusterResource.java index e87f388d7d..3a747820ec 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ClusterResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ClusterResource.java @@ -379,7 +379,7 @@ public class ClusterResource extends ApplicationResource { // update the revision RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // generate the response entity final ProcessorEntity entity = new ProcessorEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java index a941444769..5d233f78c7 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java @@ -450,7 +450,7 @@ public class ConnectionResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // create the response entity ConnectionEntity entity = new ConnectionEntity(); @@ -684,7 +684,7 @@ public class ConnectionResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // create the response entity ConnectionEntity entity = new ConnectionEntity(); @@ -742,7 +742,7 @@ public class ConnectionResource extends ApplicationResource { // create the revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(clientId.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // create the response entity final ConnectionEntity entity = new ConnectionEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java index 3afe0e12cc..c0b4cd7519 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java @@ -72,6 +72,8 @@ import org.apache.nifi.web.api.request.ClientIdParameter; import org.apache.nifi.web.api.request.IntegerParameter; import org.apache.nifi.web.api.request.LongParameter; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.web.api.entity.ControllerServiceTypesEntity; +import org.apache.nifi.web.api.entity.ReportingTaskTypesEntity; import org.codehaus.enunciate.jaxrs.TypeHint; import org.springframework.security.access.prepost.PreAuthorize; @@ -149,7 +151,7 @@ public class ControllerResource extends ApplicationResource { } /** - * Locates the Template sub-resource. + * Locates the Snippets sub-resource. * * @return */ @@ -157,6 +159,26 @@ public class ControllerResource extends ApplicationResource { public SnippetResource getSnippetResource() { return resourceContext.getResource(SnippetResource.class); } + + /** + * Locates the Controller Services sub-resource. + * + * @return + */ + @Path("/controller-services") + public ControllerServiceResource getControllerServiceResource() { + return resourceContext.getResource(ControllerServiceResource.class); + } + + /** + * Locates the Reporting Tasks sub-resource. + * + * @return + */ + @Path("/reporting-tasks") + public ReportingTaskResource getReportingTaskResource() { + return resourceContext.getResource(ReportingTaskResource.class); + } /** * Locates the Group sub-resource. @@ -303,7 +325,7 @@ public class ControllerResource extends ApplicationResource { // create the revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(clientId.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // create the response entity final ProcessGroupEntity controllerEntity = new ProcessGroupEntity(); @@ -325,11 +347,6 @@ public class ControllerResource extends ApplicationResource { @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") @TypeHint(Entity.class) public Response getRevision() { - // replicate if cluster manager - if (properties.isClusterManager()) { - return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); - } - // create the current revision final RevisionDTO revision = serviceFacade.getRevision(); @@ -595,7 +612,7 @@ public class ControllerResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // create the response entity final ControllerConfigurationEntity entity = new ControllerConfigurationEntity(); @@ -713,6 +730,75 @@ public class ControllerResource extends ApplicationResource { // generate the response return clusterContext(generateOkResponse(entity)).build(); } + + /** + * Retrieves the types of controller services that this NiFi supports. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param serviceType Returns only services that implement this type + * @return A controllerServicesTypesEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/controller-service-types") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(ControllerServiceTypesEntity.class) + public Response getControllerServiceTypes( + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @QueryParam("serviceType") String serviceType) { + + // replicate if cluster manager + if (properties.isClusterManager()) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create response entity + final ControllerServiceTypesEntity entity = new ControllerServiceTypesEntity(); + entity.setRevision(revision); + entity.setControllerServiceTypes(serviceFacade.getControllerServiceTypes(serviceType)); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Retrieves the types of reporting tasks that this NiFi supports. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @return A controllerServicesTypesEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/reporting-task-types") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(ReportingTaskTypesEntity.class) + public Response getReportingTaskTypes(@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId) { + + // replicate if cluster manager + if (properties.isClusterManager()) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create response entity + final ReportingTaskTypesEntity entity = new ReportingTaskTypesEntity(); + entity.setRevision(revision); + entity.setReportingTaskTypes(serviceFacade.getReportingTaskTypes()); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } /** * Retrieves the types of prioritizers that this NiFi supports. diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java new file mode 100644 index 0000000000..1711f3cd67 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java @@ -0,0 +1,803 @@ +/* + * 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; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import javax.servlet.ServletContext; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.DELETE; +import javax.ws.rs.DefaultValue; +import javax.ws.rs.FormParam; +import javax.ws.rs.GET; +import javax.ws.rs.HttpMethod; +import javax.ws.rs.POST; +import javax.ws.rs.PUT; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.MultivaluedMap; +import javax.ws.rs.core.Response; +import org.apache.nifi.cluster.manager.impl.WebClusterManager; +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.ConfigurationSnapshot; +import org.apache.nifi.web.NiFiServiceFacade; +import org.apache.nifi.web.Revision; +import org.apache.nifi.web.api.dto.RevisionDTO; +import org.apache.nifi.web.api.request.ClientIdParameter; +import org.apache.nifi.web.api.request.LongParameter; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.service.ControllerServiceState; +import org.apache.nifi.ui.extension.UiExtension; +import org.apache.nifi.ui.extension.UiExtensionMapping; +import org.apache.nifi.web.UiExtensionType; +import static org.apache.nifi.web.api.ApplicationResource.CLIENT_ID; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; +import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; +import org.apache.nifi.web.api.entity.ControllerServiceEntity; +import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity; +import org.apache.nifi.web.api.entity.ControllerServicesEntity; +import org.apache.nifi.web.api.entity.PropertyDescriptorEntity; +import org.apache.nifi.web.util.Availability; +import org.codehaus.enunciate.jaxrs.TypeHint; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.security.access.prepost.PreAuthorize; + +/** + * RESTful endpoint for managing a Controller Service. + */ +public class ControllerServiceResource extends ApplicationResource { + + private static final Logger logger = LoggerFactory.getLogger(ControllerServiceResource.class); + + private NiFiServiceFacade serviceFacade; + private WebClusterManager clusterManager; + private NiFiProperties properties; + + @Context + private ServletContext servletContext; + + /** + * Populates the uri for the specified controller service. + * + * @param controllerServices + * @return + */ + private Set populateRemainingControllerServicesContent(final String availability, final Set controllerServices) { + for (ControllerServiceDTO controllerService : controllerServices) { + populateRemainingControllerServiceContent(availability, controllerService); + } + return controllerServices; + } + + /** + * Populates the uri for the specified controller service. + */ + private ControllerServiceDTO populateRemainingControllerServiceContent(final String availability, final ControllerServiceDTO controllerService) { + // populate the controller service href + controllerService.setUri(generateResourceUri("controller", "controller-services", availability, controllerService.getId())); + controllerService.setAvailability(availability); + + // see if this processor has any ui extensions + final UiExtensionMapping uiExtensionMapping = (UiExtensionMapping) servletContext.getAttribute("nifi-ui-extensions"); + if (uiExtensionMapping.hasUiExtension(controllerService.getType())) { + final List uiExtensions = uiExtensionMapping.getUiExtension(controllerService.getType()); + for (final UiExtension uiExtension : uiExtensions) { + if (UiExtensionType.ControllerServiceConfiguration.equals(uiExtension.getExtensionType())) { + controllerService.setCustomUiUrl(uiExtension.getContextPath() + "/configure"); + } + } + } + + return controllerService; + } + + /** + * Parses the availability and ensure that the specified availability makes sense for the + * given NiFi instance. + * + * @param availability + * @return + */ + private Availability parseAvailability(final String availability) { + final Availability avail; + try { + avail = Availability.valueOf(availability.toUpperCase()); + } catch (IllegalArgumentException iae) { + throw new IllegalArgumentException(String.format("Availability: Value must be one of [%s]", StringUtils.join(Availability.values(), ", "))); + } + + // ensure this nifi is an NCM is specifying NCM availability + if (!properties.isClusterManager() && Availability.NCM.equals(avail)) { + throw new IllegalArgumentException("Availability of NCM is only applicable when the NiFi instance is the cluster manager."); + } + + return avail; + } + + /** + * Retrieves all the of controller services in this NiFi. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @return A controllerServicesEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(ControllerServicesEntity.class) + public Response getControllerServices(@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, @PathParam("availability") String availability) { + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get all the controller services + final Set controllerServices = populateRemainingControllerServicesContent(availability, serviceFacade.getControllerServices()); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create the response entity + final ControllerServicesEntity entity = new ControllerServicesEntity(); + entity.setRevision(revision); + entity.setControllerServices(controllerServices); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Creates a new controller service. + * + * @param httpServletRequest + * @param version The revision is used to verify the client is working with + * the latest version of the flow. + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @param type The type of controller service to create. + * @return A controllerServiceEntity. + */ + @POST + @Consumes(MediaType.APPLICATION_FORM_URLENCODED) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ControllerServiceEntity.class) + public Response createControllerService( + @Context HttpServletRequest httpServletRequest, + @FormParam(VERSION) LongParameter version, + @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, + @FormParam("type") String type) { + + // create the controller service DTO + final ControllerServiceDTO controllerServiceDTO = new ControllerServiceDTO(); + controllerServiceDTO.setType(type); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + if (version != null) { + revision.setVersion(version.getLong()); + } + + // create the controller service entity + final ControllerServiceEntity controllerServiceEntity = new ControllerServiceEntity(); + controllerServiceEntity.setRevision(revision); + controllerServiceEntity.setControllerService(controllerServiceDTO); + + return createControllerService(httpServletRequest, availability, controllerServiceEntity); + } + + /** + * Creates a new Controller Service. + * + * @param httpServletRequest + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @param controllerServiceEntity A controllerServiceEntity. + * @return A controllerServiceEntity. + */ + @POST + @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ControllerServiceEntity.class) + public Response createControllerService( + @Context HttpServletRequest httpServletRequest, + @PathParam("availability") String availability, + ControllerServiceEntity controllerServiceEntity) { + + final Availability avail = parseAvailability(availability); + + if (controllerServiceEntity == null || controllerServiceEntity.getControllerService()== null) { + throw new IllegalArgumentException("Controller service details must be specified."); + } + + if (controllerServiceEntity.getRevision() == null) { + throw new IllegalArgumentException("Revision must be specified."); + } + + if (controllerServiceEntity.getControllerService().getId() != null) { + throw new IllegalArgumentException("Controller service ID cannot be specified."); + } + + if (StringUtils.isBlank(controllerServiceEntity.getControllerService().getType())) { + throw new IllegalArgumentException("The type of controller service to create must be specified."); + } + + // get the revision + final RevisionDTO revision = controllerServiceEntity.getRevision(); + + // if cluster manager, convert POST to PUT (to maintain same ID across nodes) and replicate + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + // create ID for resource + final String id = UUID.randomUUID().toString(); + + // set ID for resource + controllerServiceEntity.getControllerService().setId(id); + + // convert POST request to PUT request to force entity ID to be the same across nodes + URI putUri = null; + try { + putUri = new URI(getAbsolutePath().toString() + "/" + id); + } catch (final URISyntaxException e) { + throw new WebApplicationException(e); + } + + // change content type to JSON for serializing entity + final Map headersToOverride = new HashMap<>(); + headersToOverride.put("content-type", MediaType.APPLICATION_JSON); + + // replicate put request + return (Response) clusterManager.applyRequest(HttpMethod.PUT, putUri, updateClientId(controllerServiceEntity), getHeaders(headersToOverride)).getResponse(); + } + + // handle expects request (usually from the cluster manager) + final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER); + if (expects != null) { + return generateContinueResponse().build(); + } + + // create the controller service and generate the json + final ConfigurationSnapshot controllerResponse = serviceFacade.createControllerService( + new Revision(revision.getVersion(), revision.getClientId()), controllerServiceEntity.getControllerService()); + final ControllerServiceDTO controllerService = controllerResponse.getConfiguration(); + + // get the updated revision + final RevisionDTO updatedRevision = new RevisionDTO(); + updatedRevision.setClientId(revision.getClientId()); + updatedRevision.setVersion(controllerResponse.getVersion()); + + // build the response entity + final ControllerServiceEntity entity = new ControllerServiceEntity(); + entity.setRevision(updatedRevision); + entity.setControllerService(populateRemainingControllerServiceContent(availability, controllerService)); + + // build the response + return clusterContext(generateCreatedResponse(URI.create(controllerService.getUri()), entity)).build(); + } + + /** + * Retrieves the specified controller service. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @param id The id of the controller service to retrieve + * @return A controllerServiceEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(ControllerServiceEntity.class) + public Response getControllerService(@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id) { + + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get the controller service + final ControllerServiceDTO controllerService = serviceFacade.getControllerService(id); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create the response entity + final ControllerServiceEntity entity = new ControllerServiceEntity(); + entity.setRevision(revision); + entity.setControllerService(populateRemainingControllerServiceContent(availability, controllerService)); + + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Returns the descriptor for the specified property. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability + * @param id The id of the controller service. + * @param propertyName The property + * @return a propertyDescriptorEntity + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}/descriptors") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(PropertyDescriptorEntity.class) + public Response getPropertyDescriptor( + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id, + @QueryParam("propertyName") String propertyName) { + + final Availability avail = parseAvailability(availability); + + // ensure the property name is specified + if (propertyName == null) { + throw new IllegalArgumentException("The property name must be specified."); + } + + // replicate if cluster manager and service is on node + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get the property descriptor + final PropertyDescriptorDTO descriptor = serviceFacade.getControllerServicePropertyDescriptor(id, propertyName); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // generate the response entity + final PropertyDescriptorEntity entity = new PropertyDescriptorEntity(); + entity.setRevision(revision); + entity.setPropertyDescriptor(descriptor); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Retrieves the references of the specified controller service. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @param id The id of the controller service to retrieve + * @return A controllerServiceEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}/references") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(ControllerServiceEntity.class) + public Response getControllerServiceReferences( + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id) { + + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get the controller service + final Set controllerServiceReferences = serviceFacade.getControllerServiceReferencingComponents(id); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create the response entity + final ControllerServiceReferencingComponentsEntity entity = new ControllerServiceReferencingComponentsEntity(); + entity.setRevision(revision); + entity.setControllerServiceReferencingComponents(controllerServiceReferences); + + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Updates the references of the specified controller service. + * + * @param httpServletRequest + * @param version The revision is used to verify the client is working with + * the latest version of the flow. + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @param id The id of the controller service to retrieve + * @param state Sets the state of referencing components. A value of RUNNING or STOPPED will update + * referencing schedulable components (Processors and Reporting Tasks). A value of ENABLED or + * DISABLED will update referencing controller services. + * @return A controllerServiceEntity. + */ + @PUT + @Consumes(MediaType.APPLICATION_FORM_URLENCODED) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}/references") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(ControllerServiceEntity.class) + public Response updateControllerServiceReferences( + @Context HttpServletRequest httpServletRequest, + @FormParam(VERSION) LongParameter version, + @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id, + @FormParam("state") @DefaultValue(StringUtils.EMPTY) String state) { + + // parse the state to determine the desired action + + // need to consider controller service state first as it shares a state with + // scheduled state (disabled) which is applicable for referencing services + // but not referencing schedulable components + ControllerServiceState controllerServiceState = null; + try { + controllerServiceState = ControllerServiceState.valueOf(state); + } catch (final IllegalArgumentException iae) { + // ignore + } + + ScheduledState scheduledState = null; + try { + scheduledState = ScheduledState.valueOf(state); + } catch (final IllegalArgumentException iae) { + // ignore + } + + // ensure an action has been specified + if (scheduledState == null && controllerServiceState == null) { + throw new IllegalArgumentException("Must specify the updated state. To update referencing Processors " + + "and Reporting Tasks the state should be RUNNING or STOPPED. To update the referencing Controller Services the " + + "state should be ENABLED or DISABLED."); + } + + // ensure the controller service state is not ENABLING or DISABLING + if (controllerServiceState != null && (ControllerServiceState.ENABLING.equals(controllerServiceState) || ControllerServiceState.DISABLING.equals(controllerServiceState))) { + throw new IllegalArgumentException("Cannot set the referencing services to ENABLING or DISABLING"); + } + + // determine the availability + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // handle expects request (usually from the cluster manager) + final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER); + if (expects != null) { + serviceFacade.verifyUpdateControllerServiceReferencingComponents(id, scheduledState, controllerServiceState); + return generateContinueResponse().build(); + } + + // determine the specified version + Long clientVersion = null; + if (version != null) { + clientVersion = version.getLong(); + } + + // get the controller service + final ConfigurationSnapshot> response = + serviceFacade.updateControllerServiceReferencingComponents(new Revision(clientVersion, clientId.getClientId()), id, scheduledState, controllerServiceState); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + revision.setVersion(response.getVersion()); + + // create the response entity + final ControllerServiceReferencingComponentsEntity entity = new ControllerServiceReferencingComponentsEntity(); + entity.setRevision(revision); + entity.setControllerServiceReferencingComponents(response.getConfiguration()); + + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Updates the specified controller service. + * + * @param httpServletRequest + * @param version The revision is used to verify the client is working with + * the latest version of the flow. + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @param id The id of the controller service to update. + * @param name The name of the controller service + * @param annotationData The annotation data for the controller service + * @param comments The comments for the controller service + * @param state The state of this controller service. Should be ENABLED or DISABLED. + * @param markedForDeletion Array of property names whose value should be removed. + * @param formParams Additionally, the processor properties and styles are + * specified in the form parameters. Because the property names and styles + * differ from processor to processor they are specified in a map-like + * fashion: + *
+ *
    + *
  • properties[required.file.path]=/path/to/file
  • + *
  • properties[required.hostname]=localhost
  • + *
  • properties[required.port]=80
  • + *
  • properties[optional.file.path]=/path/to/file
  • + *
  • properties[optional.hostname]=localhost
  • + *
  • properties[optional.port]=80
  • + *
  • properties[user.defined.pattern]=^.*?s.*$
  • + *
+ * @return A controllerServiceEntity. + */ + @PUT + @Consumes(MediaType.APPLICATION_FORM_URLENCODED) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ControllerServiceEntity.class) + public Response updateControllerService( + @Context HttpServletRequest httpServletRequest, + @FormParam(VERSION) LongParameter version, + @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id, @FormParam("name") String name, + @FormParam("annotationData") String annotationData, @FormParam("comments") String comments, + @FormParam("state") String state, @FormParam("markedForDeletion[]") List markedForDeletion, + MultivaluedMap formParams) { + + // create collections for holding the controller service properties + final Map updatedProperties = new LinkedHashMap<>(); + + // go through each parameter and look for processor properties + for (String parameterName : formParams.keySet()) { + if (StringUtils.isNotBlank(parameterName)) { + // see if the parameter name starts with an expected parameter type... + // if so, store the parameter name and value in the corresponding collection + if (parameterName.startsWith("properties")) { + final int startIndex = StringUtils.indexOf(parameterName, "["); + final int endIndex = StringUtils.lastIndexOf(parameterName, "]"); + if (startIndex != -1 && endIndex != -1) { + final String propertyName = StringUtils.substring(parameterName, startIndex + 1, endIndex); + updatedProperties.put(propertyName, formParams.getFirst(parameterName)); + } + } + } + } + + // set the properties to remove + for (String propertyToDelete : markedForDeletion) { + updatedProperties.put(propertyToDelete, null); + } + + // create the controller service DTO + final ControllerServiceDTO controllerServiceDTO = new ControllerServiceDTO(); + controllerServiceDTO.setId(id); + controllerServiceDTO.setName(name); + controllerServiceDTO.setAnnotationData(annotationData); + controllerServiceDTO.setComments(comments); + controllerServiceDTO.setState(state); + + // only set the properties when appropriate + if (!updatedProperties.isEmpty()) { + controllerServiceDTO.setProperties(updatedProperties); + } + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + if (version != null) { + revision.setVersion(version.getLong()); + } + + // create the controller service entity + final ControllerServiceEntity controllerServiceEntity = new ControllerServiceEntity(); + controllerServiceEntity.setRevision(revision); + controllerServiceEntity.setControllerService(controllerServiceDTO); + + // update the controller service + return updateControllerService(httpServletRequest, availability, id, controllerServiceEntity); + } + + /** + * Updates the specified a new Controller Service. + * + * @param httpServletRequest + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @param id The id of the controller service to update. + * @param controllerServiceEntity A controllerServiceEntity. + * @return A controllerServiceEntity. + */ + @PUT + @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ControllerServiceEntity.class) + public Response updateControllerService( + @Context HttpServletRequest httpServletRequest, + @PathParam("availability") String availability, + @PathParam("id") String id, + ControllerServiceEntity controllerServiceEntity) { + + final Availability avail = parseAvailability(availability); + + if (controllerServiceEntity == null || controllerServiceEntity.getControllerService()== null) { + throw new IllegalArgumentException("Controller service details must be specified."); + } + + if (controllerServiceEntity.getRevision() == null) { + throw new IllegalArgumentException("Revision must be specified."); + } + + // ensure the ids are the same + final ControllerServiceDTO requestControllerServiceDTO = controllerServiceEntity.getControllerService(); + if (!id.equals(requestControllerServiceDTO.getId())) { + throw new IllegalArgumentException(String.format("The controller service id (%s) in the request body does not equal the " + + "controller service id of the requested resource (%s).", requestControllerServiceDTO.getId(), id)); + } + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + // change content type to JSON for serializing entity + final Map headersToOverride = new HashMap<>(); + headersToOverride.put("content-type", MediaType.APPLICATION_JSON); + + // replicate the request + return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), updateClientId(controllerServiceEntity), getHeaders(headersToOverride)).getResponse(); + } + + // handle expects request (usually from the cluster manager) + final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER); + if (expects != null) { + serviceFacade.verifyUpdateControllerService(requestControllerServiceDTO); + return generateContinueResponse().build(); + } + + // update the controller service + final RevisionDTO revision = controllerServiceEntity.getRevision(); + final ConfigurationSnapshot controllerResponse = serviceFacade.updateControllerService( + new Revision(revision.getVersion(), revision.getClientId()), requestControllerServiceDTO); + + // get the results + final ControllerServiceDTO responseControllerServiceDTO = controllerResponse.getConfiguration(); + + // get the updated revision + final RevisionDTO updatedRevision = new RevisionDTO(); + updatedRevision.setClientId(revision.getClientId()); + updatedRevision.setVersion(controllerResponse.getVersion()); + + // build the response entity + final ControllerServiceEntity entity = new ControllerServiceEntity(); + entity.setRevision(updatedRevision); + entity.setControllerService(populateRemainingControllerServiceContent(availability, responseControllerServiceDTO)); + + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Removes the specified controller service. + * + * @param httpServletRequest + * @param version The revision is used to verify the client is working with + * the latest version of the flow. + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @param id The id of the controller service to remove. + * @return A entity containing the client id and an updated revision. + */ + @DELETE + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ControllerServiceEntity.class) + public Response removeControllerService( + @Context HttpServletRequest httpServletRequest, + @QueryParam(VERSION) LongParameter version, + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id) { + + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.DELETE, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // handle expects request (usually from the cluster manager) + final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER); + if (expects != null) { + serviceFacade.verifyDeleteControllerService(id); + return generateContinueResponse().build(); + } + + // determine the specified version + Long clientVersion = null; + if (version != null) { + clientVersion = version.getLong(); + } + + // delete the specified controller service + final ConfigurationSnapshot controllerResponse = serviceFacade.deleteControllerService(new Revision(clientVersion, clientId.getClientId()), id); + + // get the updated revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + revision.setVersion(controllerResponse.getVersion()); + + // build the response entity + final ControllerServiceEntity entity = new ControllerServiceEntity(); + entity.setRevision(revision); + + return clusterContext(generateOkResponse(entity)).build(); + } + + // setters + + public void setServiceFacade(NiFiServiceFacade serviceFacade) { + this.serviceFacade = serviceFacade; + } + + public void setClusterManager(WebClusterManager clusterManager) { + this.clusterManager = clusterManager; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FunnelResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FunnelResource.java index 4406c2efba..3492de2093 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FunnelResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FunnelResource.java @@ -244,7 +244,7 @@ public class FunnelResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final FunnelEntity entity = new FunnelEntity(); @@ -408,7 +408,7 @@ public class FunnelResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final FunnelEntity entity = new FunnelEntity(); @@ -465,7 +465,7 @@ public class FunnelResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // build the response entity final FunnelEntity entity = new FunnelEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/HistoryResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/HistoryResource.java index 38806eb01c..0f60f52fad 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/HistoryResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/HistoryResource.java @@ -37,7 +37,7 @@ import org.apache.nifi.web.api.dto.RevisionDTO; import org.apache.nifi.web.api.dto.action.ActionDTO; import org.apache.nifi.web.api.dto.action.HistoryDTO; import org.apache.nifi.web.api.dto.action.HistoryQueryDTO; -import org.apache.nifi.web.api.entity.ProcessorHistoryEntity; +import org.apache.nifi.web.api.entity.ComponentHistoryEntity; import org.codehaus.enunciate.jaxrs.TypeHint; import org.springframework.security.access.prepost.PreAuthorize; @@ -245,7 +245,7 @@ public class HistoryResource extends ApplicationResource { @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") @Path("/processors/{processorId}") - @TypeHint(ProcessorHistoryEntity.class) + @TypeHint(ComponentHistoryEntity.class) public Response getProcessorHistory( @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, @PathParam("processorId") final String processorId) { @@ -255,14 +255,76 @@ public class HistoryResource extends ApplicationResource { revision.setClientId(clientId.getClientId()); // create the response entity - final ProcessorHistoryEntity entity = new ProcessorHistoryEntity(); + final ComponentHistoryEntity entity = new ComponentHistoryEntity(); entity.setRevision(revision); - entity.setProcessorHistory(serviceFacade.getProcessorHistory(processorId)); + entity.setComponentHistory(serviceFacade.getComponentHistory(processorId)); // generate the response return generateOkResponse(entity).build(); } + + /** + * Gets the actions for the specified controller service. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param controllerServiceId The id of the controller service. + * @return An componentHistoryEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @Path("/controller-services/{controllerServiceId}") + @TypeHint(ComponentHistoryEntity.class) + public Response getControllerServiceHistory( + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("controllerServiceId") final String controllerServiceId) { + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create the response entity + final ComponentHistoryEntity entity = new ComponentHistoryEntity(); + entity.setRevision(revision); + entity.setComponentHistory(serviceFacade.getComponentHistory(controllerServiceId)); + + // generate the response + return generateOkResponse(entity).build(); + } + + /** + * Gets the actions for the specified reporting task. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param reportingTaskId The id of the reporting task. + * @return An componentHistoryEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @Path("/reporting-tasks/{reportingTaskId}") + @TypeHint(ComponentHistoryEntity.class) + public Response getReportingTaskHistory( + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("reportingTaskId") final String reportingTaskId) { + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create the response entity + final ComponentHistoryEntity entity = new ComponentHistoryEntity(); + entity.setRevision(revision); + entity.setComponentHistory(serviceFacade.getComponentHistory(reportingTaskId)); + + // generate the response + return generateOkResponse(entity).build(); + } + /* setters */ public void setServiceFacade(NiFiServiceFacade serviceFacade) { this.serviceFacade = serviceFacade; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/InputPortResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/InputPortResource.java index 58c3c9e89a..f3a6326604 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/InputPortResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/InputPortResource.java @@ -251,7 +251,7 @@ public class InputPortResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final InputPortEntity entity = new InputPortEntity(); @@ -446,7 +446,7 @@ public class InputPortResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final InputPortEntity entity = new InputPortEntity(); @@ -503,7 +503,7 @@ public class InputPortResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // build the response entity final InputPortEntity entity = new InputPortEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/LabelResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/LabelResource.java index 9a61cfcc6e..6435671d49 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/LabelResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/LabelResource.java @@ -260,7 +260,7 @@ public class LabelResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final LabelEntity entity = new LabelEntity(); @@ -463,7 +463,7 @@ public class LabelResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final LabelEntity entity = new LabelEntity(); @@ -519,7 +519,7 @@ public class LabelResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // build the response entity final LabelEntity entity = new LabelEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/OutputPortResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/OutputPortResource.java index 224ab18588..a9dce5fdfa 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/OutputPortResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/OutputPortResource.java @@ -251,7 +251,7 @@ public class OutputPortResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final OutputPortEntity entity = new OutputPortEntity(); @@ -447,7 +447,7 @@ public class OutputPortResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final OutputPortEntity entity = new OutputPortEntity(); @@ -504,7 +504,7 @@ public class OutputPortResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // build the response entity final OutputPortEntity entity = new OutputPortEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java index 6439bda1a6..1bf3f7779a 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java @@ -287,7 +287,7 @@ public class ProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // create the response entity final ProcessGroupEntity processGroupEntity = new ProcessGroupEntity(); @@ -365,7 +365,7 @@ public class ProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // create the response entity final FlowSnippetEntity entity = new FlowSnippetEntity(); @@ -441,7 +441,7 @@ public class ProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(response.getRevision()); + revision.setVersion(response.getVersion()); // create the response entity final FlowSnippetEntity entity = new FlowSnippetEntity(); @@ -559,7 +559,7 @@ public class ProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(response.getRevision()); + updatedRevision.setVersion(response.getVersion()); // create the response entity final ProcessGroupEntity entity = new ProcessGroupEntity(); @@ -616,7 +616,7 @@ public class ProcessGroupResource extends ApplicationResource { // create the revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // create the response entity final ProcessGroupEntity processGroupEntity = new ProcessGroupEntity(); @@ -795,7 +795,7 @@ public class ProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // create the response entity final ProcessGroupEntity entity = new ProcessGroupEntity(); @@ -931,7 +931,7 @@ public class ProcessGroupResource extends ApplicationResource { // create the revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(response.getRevision()); + updatedRevision.setVersion(response.getVersion()); // create the response entity final ProcessGroupEntity entity = new ProcessGroupEntity(); @@ -989,7 +989,7 @@ public class ProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // create the response entity final ProcessGroupEntity entity = new ProcessGroupEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java index b11c40a0e9..31ab10b6d9 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java @@ -70,6 +70,11 @@ import org.apache.nifi.web.api.request.IntegerParameter; import org.apache.nifi.web.api.request.LongParameter; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.ui.extension.UiExtension; +import org.apache.nifi.ui.extension.UiExtensionMapping; +import org.apache.nifi.web.UiExtensionType; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; +import org.apache.nifi.web.api.entity.PropertyDescriptorEntity; import org.codehaus.enunciate.jaxrs.TypeHint; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -118,9 +123,21 @@ public class ProcessorResource extends ApplicationResource { // get the config details and see if there is a custom ui for this processor type ProcessorConfigDTO config = processor.getConfig(); if (config != null) { + // consider legacy custom ui fist String customUiUrl = servletContext.getInitParameter(processor.getType()); if (StringUtils.isNotBlank(customUiUrl)) { config.setCustomUiUrl(customUiUrl); + } else { + // see if this processor has any ui extensions + final UiExtensionMapping uiExtensionMapping = (UiExtensionMapping) servletContext.getAttribute("nifi-ui-extensions"); + if (uiExtensionMapping.hasUiExtension(processor.getType())) { + final List uiExtensions = uiExtensionMapping.getUiExtension(processor.getType()); + for (final UiExtension uiExtension : uiExtensions) { + if (UiExtensionType.ProcessorConfiguration.equals(uiExtension.getExtensionType())) { + config.setCustomUiUrl(uiExtension.getContextPath() + "/configure"); + } + } + } } } @@ -245,6 +262,10 @@ public class ProcessorResource extends ApplicationResource { if (processorEntity.getProcessor().getId() != null) { throw new IllegalArgumentException("Processor ID cannot be specified."); } + + if (StringUtils.isBlank(processorEntity.getProcessor().getType())) { + throw new IllegalArgumentException("The type of processor to create must be specified."); + } // if cluster manager, convert POST to PUT (to maintain same ID across nodes) and replicate if (properties.isClusterManager()) { @@ -288,7 +309,7 @@ public class ProcessorResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // generate the response entity final ProcessorEntity entity = new ProcessorEntity(); @@ -373,6 +394,51 @@ public class ProcessorResource extends ApplicationResource { // generate the response return clusterContext(generateOkResponse(entity)).build(); } + + /** + * Returns the descriptor for the specified property. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param id The id of the processor + * @param propertyName The property + * @return a propertyDescriptorEntity + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{id}/descriptors") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(PropertyDescriptorEntity.class) + public Response getPropertyDescriptor( + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("id") String id, @QueryParam("propertyName") String propertyName) { + + // ensure the property name is specified + if (propertyName == null) { + throw new IllegalArgumentException("The property name must be specified."); + } + + // replicate if cluster manager + if (properties.isClusterManager()) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get the property descriptor + final PropertyDescriptorDTO descriptor = serviceFacade.getProcessorPropertyDescriptor(groupId, id, propertyName); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // generate the response entity + final PropertyDescriptorEntity entity = new PropertyDescriptorEntity(); + entity.setRevision(revision); + entity.setPropertyDescriptor(descriptor); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } /** * Updates the specified processor with the specified values. @@ -607,7 +673,7 @@ public class ProcessorResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // generate the response entity final ProcessorEntity entity = new ProcessorEntity(); @@ -664,7 +730,7 @@ public class ProcessorResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(clientId.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // generate the response entity final ProcessorEntity entity = new ProcessorEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java index 27fa29236f..4e15c36743 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java @@ -396,7 +396,7 @@ public class RemoteProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final RemoteProcessGroupEntity entity = new RemoteProcessGroupEntity(); @@ -452,7 +452,7 @@ public class RemoteProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // create the response entity final RemoteProcessGroupEntity entity = new RemoteProcessGroupEntity(); @@ -586,7 +586,7 @@ public class RemoteProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final RemoteProcessGroupPortEntity entity = new RemoteProcessGroupPortEntity(); @@ -720,7 +720,7 @@ public class RemoteProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity RemoteProcessGroupPortEntity entity = new RemoteProcessGroupPortEntity(); @@ -890,7 +890,7 @@ public class RemoteProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final RemoteProcessGroupEntity entity = new RemoteProcessGroupEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java new file mode 100644 index 0000000000..38ddc36628 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java @@ -0,0 +1,663 @@ +/* + * 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; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import javax.servlet.ServletContext; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.DELETE; +import javax.ws.rs.DefaultValue; +import javax.ws.rs.FormParam; +import javax.ws.rs.GET; +import javax.ws.rs.HttpMethod; +import javax.ws.rs.POST; +import javax.ws.rs.PUT; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.MultivaluedMap; +import javax.ws.rs.core.Response; +import org.apache.nifi.cluster.manager.impl.WebClusterManager; +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.ConfigurationSnapshot; +import org.apache.nifi.web.NiFiServiceFacade; +import org.apache.nifi.web.Revision; +import org.apache.nifi.web.api.dto.RevisionDTO; +import org.apache.nifi.web.api.request.ClientIdParameter; +import org.apache.nifi.web.api.request.LongParameter; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.ui.extension.UiExtension; +import org.apache.nifi.ui.extension.UiExtensionMapping; +import org.apache.nifi.web.UiExtensionType; +import static org.apache.nifi.web.api.ApplicationResource.CLIENT_ID; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; +import org.apache.nifi.web.api.entity.PropertyDescriptorEntity; +import org.apache.nifi.web.api.entity.ReportingTaskEntity; +import org.apache.nifi.web.api.entity.ReportingTasksEntity; +import org.apache.nifi.web.util.Availability; +import org.codehaus.enunciate.jaxrs.TypeHint; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.security.access.prepost.PreAuthorize; + +/** + * RESTful endpoint for managing a Reporting Task. + */ +public class ReportingTaskResource extends ApplicationResource { + + private static final Logger logger = LoggerFactory.getLogger(ReportingTaskResource.class); + + private NiFiServiceFacade serviceFacade; + private WebClusterManager clusterManager; + private NiFiProperties properties; + + @Context + private ServletContext servletContext; + + /** + * Populates the uri for the specified reporting task. + * + * @param reportingTasks + * @return + */ + private Set populateRemainingReportingTasksContent(final String availability, final Set reportingTasks) { + for (ReportingTaskDTO reportingTask : reportingTasks) { + populateRemainingReportingTaskContent(availability, reportingTask); + } + return reportingTasks; + } + + /** + * Populates the uri for the specified reporting task. + */ + private ReportingTaskDTO populateRemainingReportingTaskContent(final String availability, final ReportingTaskDTO reportingTask) { + // populate the reporting task href + reportingTask.setUri(generateResourceUri("controller", "reporting-tasks", availability, reportingTask.getId())); + reportingTask.setAvailability(availability); + + // see if this processor has any ui extensions + final UiExtensionMapping uiExtensionMapping = (UiExtensionMapping) servletContext.getAttribute("nifi-ui-extensions"); + if (uiExtensionMapping.hasUiExtension(reportingTask.getType())) { + final List uiExtensions = uiExtensionMapping.getUiExtension(reportingTask.getType()); + for (final UiExtension uiExtension : uiExtensions) { + if (UiExtensionType.ReportingTaskConfiguration.equals(uiExtension.getExtensionType())) { + reportingTask.setCustomUiUrl(uiExtension.getContextPath() + "/configure"); + } + } + } + + return reportingTask; + } + + /** + * Parses the availability and ensure that the specified availability makes sense for the + * given NiFi instance. + * + * @param availability + * @return + */ + private Availability parseAvailability(final String availability) { + final Availability avail; + try { + avail = Availability.valueOf(availability.toUpperCase()); + } catch (IllegalArgumentException iae) { + throw new IllegalArgumentException(String.format("Availability: Value must be one of [%s]", StringUtils.join(Availability.values(), ", "))); + } + + // ensure this nifi is an NCM is specifying NCM availability + if (!properties.isClusterManager() && Availability.NCM.equals(avail)) { + throw new IllegalArgumentException("Availability of NCM is only applicable when the NiFi instance is the cluster manager."); + } + + return avail; + } + + /** + * Retrieves all the of reporting tasks in this NiFi. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the reporting task is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all tasks should use the node availability. + * @return A reportingTasksEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(ReportingTasksEntity.class) + public Response getReportingTasks(@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, @PathParam("availability") String availability) { + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get all the reporting tasks + final Set reportingTasks = populateRemainingReportingTasksContent(availability, serviceFacade.getReportingTasks()); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create the response entity + final ReportingTasksEntity entity = new ReportingTasksEntity(); + entity.setRevision(revision); + entity.setReportingTasks(reportingTasks); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Creates a new reporting task. + * + * @param httpServletRequest + * @param version The revision is used to verify the client is working with + * the latest version of the flow. + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the reporting task is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all tasks should use the node availability. + * @param type The type of reporting task to create. + * @return A reportingTaskEntity. + */ + @POST + @Consumes(MediaType.APPLICATION_FORM_URLENCODED) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ReportingTaskEntity.class) + public Response createReportingTask( + @Context HttpServletRequest httpServletRequest, + @FormParam(VERSION) LongParameter version, + @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, + @FormParam("type") String type) { + + // create the reporting task DTO + final ReportingTaskDTO reportingTaskDTO = new ReportingTaskDTO(); + reportingTaskDTO.setType(type); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + if (version != null) { + revision.setVersion(version.getLong()); + } + + // create the reporting task entity + final ReportingTaskEntity reportingTaskEntity = new ReportingTaskEntity(); + reportingTaskEntity.setRevision(revision); + reportingTaskEntity.setReportingTask(reportingTaskDTO); + + return createReportingTask(httpServletRequest, availability, reportingTaskEntity); + } + + /** + * Creates a new Reporting Task. + * + * @param httpServletRequest + * @param availability Whether the reporting task is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all tasks should use the node availability. + * @param reportingTaskEntity A reportingTaskEntity. + * @return A reportingTaskEntity. + */ + @POST + @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ReportingTaskEntity.class) + public Response createReportingTask( + @Context HttpServletRequest httpServletRequest, + @PathParam("availability") String availability, + ReportingTaskEntity reportingTaskEntity) { + + final Availability avail = parseAvailability(availability); + + if (reportingTaskEntity == null || reportingTaskEntity.getReportingTask()== null) { + throw new IllegalArgumentException("Reporting task details must be specified."); + } + + if (reportingTaskEntity.getRevision() == null) { + throw new IllegalArgumentException("Revision must be specified."); + } + + if (reportingTaskEntity.getReportingTask().getId() != null) { + throw new IllegalArgumentException("Reporting task ID cannot be specified."); + } + + if (StringUtils.isBlank(reportingTaskEntity.getReportingTask().getType())) { + throw new IllegalArgumentException("The type of reporting task to create must be specified."); + } + + // get the revision + final RevisionDTO revision = reportingTaskEntity.getRevision(); + + // if cluster manager, convert POST to PUT (to maintain same ID across nodes) and replicate + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + // create ID for resource + final String id = UUID.randomUUID().toString(); + + // set ID for resource + reportingTaskEntity.getReportingTask().setId(id); + + // convert POST request to PUT request to force entity ID to be the same across nodes + URI putUri = null; + try { + putUri = new URI(getAbsolutePath().toString() + "/" + id); + } catch (final URISyntaxException e) { + throw new WebApplicationException(e); + } + + // change content type to JSON for serializing entity + final Map headersToOverride = new HashMap<>(); + headersToOverride.put("content-type", MediaType.APPLICATION_JSON); + + // replicate put request + return (Response) clusterManager.applyRequest(HttpMethod.PUT, putUri, updateClientId(reportingTaskEntity), getHeaders(headersToOverride)).getResponse(); + } + + // handle expects request (usually from the cluster manager) + final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER); + if (expects != null) { + return generateContinueResponse().build(); + } + + // create the reporting task and generate the json + final ConfigurationSnapshot controllerResponse = serviceFacade.createReportingTask( + new Revision(revision.getVersion(), revision.getClientId()), reportingTaskEntity.getReportingTask()); + final ReportingTaskDTO reportingTask = controllerResponse.getConfiguration(); + + // get the updated revision + final RevisionDTO updatedRevision = new RevisionDTO(); + updatedRevision.setClientId(revision.getClientId()); + updatedRevision.setVersion(controllerResponse.getVersion()); + + // build the response entity + final ReportingTaskEntity entity = new ReportingTaskEntity(); + entity.setRevision(updatedRevision); + entity.setReportingTask(populateRemainingReportingTaskContent(availability, reportingTask)); + + // build the response + return clusterContext(generateCreatedResponse(URI.create(reportingTask.getUri()), entity)).build(); + } + + /** + * Retrieves the specified reporting task. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the reporting task is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all tasks should use the node availability. + * @param id The id of the reporting task to retrieve + * @return A reportingTaskEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(ReportingTaskEntity.class) + public Response getReportingTask(@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id) { + + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get the reporting task + final ReportingTaskDTO reportingTask = serviceFacade.getReportingTask(id); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create the response entity + final ReportingTaskEntity entity = new ReportingTaskEntity(); + entity.setRevision(revision); + entity.setReportingTask(populateRemainingReportingTaskContent(availability, reportingTask)); + + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Returns the descriptor for the specified property. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability + * @param id The id of the reporting task. + * @param propertyName The property + * @return a propertyDescriptorEntity + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}/descriptors") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(PropertyDescriptorEntity.class) + public Response getPropertyDescriptor( + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id, + @QueryParam("propertyName") String propertyName) { + + final Availability avail = parseAvailability(availability); + + // ensure the property name is specified + if (propertyName == null) { + throw new IllegalArgumentException("The property name must be specified."); + } + + // replicate if cluster manager and task is on node + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get the property descriptor + final PropertyDescriptorDTO descriptor = serviceFacade.getReportingTaskPropertyDescriptor(id, propertyName); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // generate the response entity + final PropertyDescriptorEntity entity = new PropertyDescriptorEntity(); + entity.setRevision(revision); + entity.setPropertyDescriptor(descriptor); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Updates the specified reporting task. + * + * @param httpServletRequest + * @param version The revision is used to verify the client is working with + * the latest version of the flow. + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the reporting task is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all tasks should use the node availability. + * @param id The id of the reporting task to update. + * @param name The name of the reporting task + * @param annotationData The annotation data for the reporting task + * @param markedForDeletion Array of property names whose value should be removed. + * @param state The updated scheduled state + * @param schedulingStrategy The scheduling strategy for this reporting task + * @param schedulingPeriod The scheduling period for this reporting task + * @param comments The comments for this reporting task + * @param formParams Additionally, the processor properties and styles are + * specified in the form parameters. Because the property names and styles + * differ from processor to processor they are specified in a map-like + * fashion: + *
+ *
    + *
  • properties[required.file.path]=/path/to/file
  • + *
  • properties[required.hostname]=localhost
  • + *
  • properties[required.port]=80
  • + *
  • properties[optional.file.path]=/path/to/file
  • + *
  • properties[optional.hostname]=localhost
  • + *
  • properties[optional.port]=80
  • + *
  • properties[user.defined.pattern]=^.*?s.*$
  • + *
+ * @return A reportingTaskEntity. + */ + @PUT + @Consumes(MediaType.APPLICATION_FORM_URLENCODED) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ReportingTaskEntity.class) + public Response updateReportingTask( + @Context HttpServletRequest httpServletRequest, + @FormParam(VERSION) LongParameter version, + @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id, @FormParam("name") String name, + @FormParam("annotationData") String annotationData, @FormParam("markedForDeletion[]") List markedForDeletion, + @FormParam("state") String state, @FormParam("schedulingStrategy") String schedulingStrategy, + @FormParam("schedulingPeriod") String schedulingPeriod, @FormParam("comments") String comments, + MultivaluedMap formParams) { + + // create collections for holding the reporting task properties + final Map updatedProperties = new LinkedHashMap<>(); + + // go through each parameter and look for processor properties + for (String parameterName : formParams.keySet()) { + if (StringUtils.isNotBlank(parameterName)) { + // see if the parameter name starts with an expected parameter type... + // if so, store the parameter name and value in the corresponding collection + if (parameterName.startsWith("properties")) { + final int startIndex = StringUtils.indexOf(parameterName, "["); + final int endIndex = StringUtils.lastIndexOf(parameterName, "]"); + if (startIndex != -1 && endIndex != -1) { + final String propertyName = StringUtils.substring(parameterName, startIndex + 1, endIndex); + updatedProperties.put(propertyName, formParams.getFirst(parameterName)); + } + } + } + } + + // set the properties to remove + for (String propertyToDelete : markedForDeletion) { + updatedProperties.put(propertyToDelete, null); + } + + // create the reporting task DTO + final ReportingTaskDTO reportingTaskDTO = new ReportingTaskDTO(); + reportingTaskDTO.setId(id); + reportingTaskDTO.setName(name); + reportingTaskDTO.setState(state); + reportingTaskDTO.setSchedulingStrategy(schedulingStrategy); + reportingTaskDTO.setSchedulingPeriod(schedulingPeriod); + reportingTaskDTO.setAnnotationData(annotationData); + reportingTaskDTO.setComments(comments); + + // only set the properties when appropriate + if (!updatedProperties.isEmpty()) { + reportingTaskDTO.setProperties(updatedProperties); + } + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + if (version != null) { + revision.setVersion(version.getLong()); + } + + // create the reporting task entity + final ReportingTaskEntity reportingTaskEntity = new ReportingTaskEntity(); + reportingTaskEntity.setRevision(revision); + reportingTaskEntity.setReportingTask(reportingTaskDTO); + + // update the reporting task + return updateReportingTask(httpServletRequest, availability, id, reportingTaskEntity); + } + + /** + * Updates the specified a Reporting Task. + * + * @param httpServletRequest + * @param availability Whether the reporting task is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all tasks should use the node availability. + * @param id The id of the reporting task to update. + * @param reportingTaskEntity A reportingTaskEntity. + * @return A reportingTaskEntity. + */ + @PUT + @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ReportingTaskEntity.class) + public Response updateReportingTask( + @Context HttpServletRequest httpServletRequest, + @PathParam("availability") String availability, + @PathParam("id") String id, + ReportingTaskEntity reportingTaskEntity) { + + final Availability avail = parseAvailability(availability); + + if (reportingTaskEntity == null || reportingTaskEntity.getReportingTask() == null) { + throw new IllegalArgumentException("Reporting task details must be specified."); + } + + if (reportingTaskEntity.getRevision() == null) { + throw new IllegalArgumentException("Revision must be specified."); + } + + // ensure the ids are the same + final ReportingTaskDTO requestReportingTaskDTO = reportingTaskEntity.getReportingTask(); + if (!id.equals(requestReportingTaskDTO.getId())) { + throw new IllegalArgumentException(String.format("The reporting task id (%s) in the request body does not equal the " + + "reporting task id of the requested resource (%s).", requestReportingTaskDTO.getId(), id)); + } + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + // change content type to JSON for serializing entity + final Map headersToOverride = new HashMap<>(); + headersToOverride.put("content-type", MediaType.APPLICATION_JSON); + + // replicate the request + return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), updateClientId(reportingTaskEntity), getHeaders(headersToOverride)).getResponse(); + } + + // handle expects request (usually from the cluster manager) + final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER); + if (expects != null) { + serviceFacade.verifyUpdateReportingTask(requestReportingTaskDTO); + return generateContinueResponse().build(); + } + + // update the reporting task + final RevisionDTO revision = reportingTaskEntity.getRevision(); + final ConfigurationSnapshot controllerResponse = serviceFacade.updateReportingTask( + new Revision(revision.getVersion(), revision.getClientId()), requestReportingTaskDTO); + + // get the results + final ReportingTaskDTO responseReportingTaskDTO = controllerResponse.getConfiguration(); + + // get the updated revision + final RevisionDTO updatedRevision = new RevisionDTO(); + updatedRevision.setClientId(revision.getClientId()); + updatedRevision.setVersion(controllerResponse.getVersion()); + + // build the response entity + final ReportingTaskEntity entity = new ReportingTaskEntity(); + entity.setRevision(updatedRevision); + entity.setReportingTask(populateRemainingReportingTaskContent(availability, responseReportingTaskDTO)); + + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Removes the specified reporting task. + * + * @param httpServletRequest + * @param version The revision is used to verify the client is working with + * the latest version of the flow. + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the reporting task is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all tasks should use the node availability. + * @param id The id of the reporting task to remove. + * @return A entity containing the client id and an updated revision. + */ + @DELETE + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ReportingTaskEntity.class) + public Response removeReportingTask( + @Context HttpServletRequest httpServletRequest, + @QueryParam(VERSION) LongParameter version, + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id) { + + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.DELETE, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // handle expects request (usually from the cluster manager) + final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER); + if (expects != null) { + serviceFacade.verifyDeleteReportingTask(id); + return generateContinueResponse().build(); + } + + // determine the specified version + Long clientVersion = null; + if (version != null) { + clientVersion = version.getLong(); + } + + // delete the specified reporting task + final ConfigurationSnapshot controllerResponse = serviceFacade.deleteReportingTask(new Revision(clientVersion, clientId.getClientId()), id); + + // get the updated revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + revision.setVersion(controllerResponse.getVersion()); + + // build the response entity + final ReportingTaskEntity entity = new ReportingTaskEntity(); + entity.setRevision(revision); + + return clusterContext(generateOkResponse(entity)).build(); + } + + // setters + + public void setServiceFacade(NiFiServiceFacade serviceFacade) { + this.serviceFacade = serviceFacade; + } + + public void setClusterManager(WebClusterManager clusterManager) { + this.clusterManager = clusterManager; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java index e0b7788221..275b133a86 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java @@ -340,7 +340,7 @@ public class SnippetResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(response.getRevision()); + updatedRevision.setVersion(response.getVersion()); // build the response entity SnippetEntity entity = new SnippetEntity(); @@ -520,7 +520,7 @@ public class SnippetResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity SnippetEntity entity = new SnippetEntity(); @@ -577,7 +577,7 @@ public class SnippetResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // build the response entity SnippetEntity entity = new SnippetEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/AdministrationExceptionMapper.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/AdministrationExceptionMapper.java index ef912545e0..d631d8214c 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/AdministrationExceptionMapper.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/AdministrationExceptionMapper.java @@ -35,11 +35,7 @@ public class AdministrationExceptionMapper implements ExceptionMapper @Override public Response toResponse(ClusterException exception) { // log the error - logger.info(String.format("%s. Returning %s response.", exception, Response.Status.INTERNAL_SERVER_ERROR)); - - if (logger.isDebugEnabled()) { - logger.debug(StringUtils.EMPTY, exception); - } + logger.error(String.format("%s. Returning %s response.", exception, Response.Status.INTERNAL_SERVER_ERROR), exception); return Response.status(Response.Status.INTERNAL_SERVER_ERROR).entity(exception.getMessage()).type("text/plain").build(); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/NotFoundExceptionMapper.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/NotFoundExceptionMapper.java new file mode 100644 index 0000000000..512391f6ff --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/NotFoundExceptionMapper.java @@ -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.config; + +import com.sun.jersey.api.NotFoundException; +import com.sun.jersey.api.Responses; +import javax.ws.rs.core.Response; +import javax.ws.rs.ext.ExceptionMapper; +import javax.ws.rs.ext.Provider; +import org.apache.nifi.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Maps not found exceptions into client responses. + */ +@Provider +public class NotFoundExceptionMapper implements ExceptionMapper { + + private static final Logger logger = LoggerFactory.getLogger(NotFoundExceptionMapper.class); + + @Override + public Response toResponse(NotFoundException exception) { + // log the error + logger.info(String.format("%s. Returning %s response.", exception, Response.Status.NOT_FOUND)); + + if (logger.isDebugEnabled()) { + logger.debug(StringUtils.EMPTY, exception); + } + + return Responses.notFound().entity("The specified resource could not be found.").type("text/plain").build(); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/ThrowableMapper.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/ThrowableMapper.java index 091653a4cb..72772508d9 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/ThrowableMapper.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/ThrowableMapper.java @@ -34,11 +34,7 @@ public class ThrowableMapper implements ExceptionMapper { @Override public Response toResponse(Throwable exception) { // log the error - logger.info(String.format("An unexpected error has occurred: %s. Returning %s response.", exception, Response.Status.INTERNAL_SERVER_ERROR)); - - if (logger.isDebugEnabled()) { - logger.debug(StringUtils.EMPTY, exception); - } + logger.error(String.format("An unexpected error has occurred: %s. Returning %s response.", exception, Response.Status.INTERNAL_SERVER_ERROR), exception); return Response.status(Response.Status.INTERNAL_SERVER_ERROR).entity("An unexpected error has occurred. Please check the logs for additional details.").type("text/plain").build(); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java index 92a54499d1..4e83ac5df0 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java @@ -39,7 +39,7 @@ import javax.ws.rs.WebApplicationException; import org.apache.nifi.action.Action; import org.apache.nifi.action.component.details.ComponentDetails; -import org.apache.nifi.action.component.details.ProcessorDetails; +import org.apache.nifi.action.component.details.ExtensionDetails; import org.apache.nifi.action.component.details.RemoteProcessGroupDetails; import org.apache.nifi.action.details.ActionDetails; import org.apache.nifi.action.details.ConfigureDetails; @@ -97,14 +97,12 @@ import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.user.NiFiUser; import org.apache.nifi.user.NiFiUserGroup; import org.apache.nifi.util.FormatUtils; -import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.web.Revision; -import org.apache.nifi.web.api.dto.ProcessorConfigDTO.AllowableValueDTO; -import org.apache.nifi.web.api.dto.ProcessorConfigDTO.PropertyDescriptorDTO; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO.AllowableValueDTO; import org.apache.nifi.web.api.dto.action.ActionDTO; import org.apache.nifi.web.api.dto.action.HistoryDTO; import org.apache.nifi.web.api.dto.action.component.details.ComponentDetailsDTO; -import org.apache.nifi.web.api.dto.action.component.details.ProcessorDetailsDTO; +import org.apache.nifi.web.api.dto.action.component.details.ExtensionDetailsDTO; import org.apache.nifi.web.api.dto.action.component.details.RemoteProcessGroupDetailsDTO; import org.apache.nifi.web.api.dto.action.details.ActionDetailsDTO; import org.apache.nifi.web.api.dto.action.details.ConfigureDetailsDTO; @@ -124,6 +122,12 @@ import org.apache.nifi.web.api.dto.status.ProcessorStatusDTO; import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusDTO; import org.apache.nifi.web.api.dto.status.StatusDTO; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.controller.ConfiguredComponent; +import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceReference; +import org.apache.nifi.reporting.ReportingTask; +import org.apache.nifi.web.FlowModification; /** * @@ -218,9 +222,9 @@ public final class DtoFactory { return null; } - if (componentDetails instanceof ProcessorDetails) { - final ProcessorDetailsDTO processorDetails = new ProcessorDetailsDTO(); - processorDetails.setType(((ProcessorDetails) componentDetails).getType()); + if (componentDetails instanceof ExtensionDetails) { + final ExtensionDetailsDTO processorDetails = new ExtensionDetailsDTO(); + processorDetails.setType(((ExtensionDetails) componentDetails).getType()); return processorDetails; } else if (componentDetails instanceof RemoteProcessGroupDetails) { final RemoteProcessGroupDetailsDTO remoteProcessGroupDetails = new RemoteProcessGroupDetailsDTO(); @@ -834,6 +838,241 @@ public final class DtoFactory { return dto; } + public ReportingTaskDTO createReportingTaskDto(final ReportingTaskNode reportingTaskNode) { + final ReportingTaskDTO dto = new ReportingTaskDTO(); + dto.setId(reportingTaskNode.getIdentifier()); + dto.setName(reportingTaskNode.getName()); + dto.setType(reportingTaskNode.getReportingTask().getClass().getName()); + dto.setSchedulingStrategy(reportingTaskNode.getSchedulingStrategy().name()); + dto.setSchedulingPeriod(reportingTaskNode.getSchedulingPeriod()); + dto.setState(reportingTaskNode.getScheduledState().name()); + dto.setActiveThreadCount(reportingTaskNode.getActiveThreadCount()); + dto.setAnnotationData(reportingTaskNode.getAnnotationData()); + dto.setComments(reportingTaskNode.getComments()); + + final Map defaultSchedulingPeriod = new HashMap<>(); + defaultSchedulingPeriod.put(SchedulingStrategy.TIMER_DRIVEN.name(), SchedulingStrategy.TIMER_DRIVEN.getDefaultSchedulingPeriod()); + defaultSchedulingPeriod.put(SchedulingStrategy.CRON_DRIVEN.name(), SchedulingStrategy.CRON_DRIVEN.getDefaultSchedulingPeriod()); + dto.setDefaultSchedulingPeriod(defaultSchedulingPeriod); + + // sort a copy of the properties + final Map sortedProperties = new TreeMap<>(new Comparator() { + @Override + public int compare(PropertyDescriptor o1, PropertyDescriptor o2) { + return Collator.getInstance(Locale.US).compare(o1.getName(), o2.getName()); + } + }); + sortedProperties.putAll(reportingTaskNode.getProperties()); + + // get the property order from the reporting task + final ReportingTask reportingTask = reportingTaskNode.getReportingTask(); + final Map orderedProperties = new LinkedHashMap<>(); + final List descriptors = reportingTask.getPropertyDescriptors(); + if (descriptors != null && !descriptors.isEmpty()) { + for (PropertyDescriptor descriptor : descriptors) { + orderedProperties.put(descriptor, null); + } + } + orderedProperties.putAll(sortedProperties); + + // build the descriptor and property dtos + dto.setDescriptors(new LinkedHashMap()); + dto.setProperties(new LinkedHashMap()); + for (final Map.Entry entry : orderedProperties.entrySet()) { + final PropertyDescriptor descriptor = entry.getKey(); + + // store the property descriptor + dto.getDescriptors().put(descriptor.getName(), createPropertyDescriptorDto(descriptor)); + + // determine the property value - don't include sensitive properties + String propertyValue = entry.getValue(); + if (propertyValue != null && descriptor.isSensitive()) { + propertyValue = "********"; + } + + // set the property value + dto.getProperties().put(descriptor.getName(), propertyValue); + } + + // add the validation errors + final Collection validationErrors = reportingTaskNode.getValidationErrors(); + if (validationErrors != null && !validationErrors.isEmpty()) { + final List errors = new ArrayList<>(); + for (final ValidationResult validationResult : validationErrors) { + errors.add(validationResult.toString()); + } + + dto.setValidationErrors(errors); + } + + return dto; + } + + public ControllerServiceDTO createControllerServiceDto(final ControllerServiceNode controllerServiceNode) { + final ControllerServiceDTO dto = new ControllerServiceDTO(); + dto.setId(controllerServiceNode.getIdentifier()); + dto.setName(controllerServiceNode.getName()); + dto.setType(controllerServiceNode.getControllerServiceImplementation().getClass().getName()); + dto.setState(controllerServiceNode.getState().name()); + dto.setAnnotationData(controllerServiceNode.getAnnotationData()); + dto.setComments(controllerServiceNode.getComments()); + + // sort a copy of the properties + final Map sortedProperties = new TreeMap<>(new Comparator() { + @Override + public int compare(PropertyDescriptor o1, PropertyDescriptor o2) { + return Collator.getInstance(Locale.US).compare(o1.getName(), o2.getName()); + } + }); + sortedProperties.putAll(controllerServiceNode.getProperties()); + + // get the property order from the controller service + final ControllerService controllerService = controllerServiceNode.getControllerServiceImplementation(); + final Map orderedProperties = new LinkedHashMap<>(); + final List descriptors = controllerService.getPropertyDescriptors(); + if (descriptors != null && !descriptors.isEmpty()) { + for (PropertyDescriptor descriptor : descriptors) { + orderedProperties.put(descriptor, null); + } + } + orderedProperties.putAll(sortedProperties); + + // build the descriptor and property dtos + dto.setDescriptors(new LinkedHashMap()); + dto.setProperties(new LinkedHashMap()); + for (final Map.Entry entry : orderedProperties.entrySet()) { + final PropertyDescriptor descriptor = entry.getKey(); + + // store the property descriptor + dto.getDescriptors().put(descriptor.getName(), createPropertyDescriptorDto(descriptor)); + + // determine the property value - don't include sensitive properties + String propertyValue = entry.getValue(); + if (propertyValue != null && descriptor.isSensitive()) { + propertyValue = "********"; + } + + // set the property value + dto.getProperties().put(descriptor.getName(), propertyValue); + } + + // create the reference dto's + dto.setReferencingComponents(createControllerServiceReferencingComponentsDto(controllerServiceNode.getReferences())); + + // add the validation errors + final Collection validationErrors = controllerServiceNode.getValidationErrors(); + if (validationErrors != null && !validationErrors.isEmpty()) { + final List errors = new ArrayList<>(); + for (final ValidationResult validationResult : validationErrors) { + errors.add(validationResult.toString()); + } + + dto.setValidationErrors(errors); + } + + return dto; + } + + public Set createControllerServiceReferencingComponentsDto(final ControllerServiceReference reference) { + return createControllerServiceReferencingComponentsDto(reference, new HashSet()); + } + + private Set createControllerServiceReferencingComponentsDto(final ControllerServiceReference reference, final Set visited) { + final Set referencingComponents = new LinkedHashSet<>(); + + // get all references + for (final ConfiguredComponent component : reference.getReferencingComponents()) { + final ControllerServiceReferencingComponentDTO dto = new ControllerServiceReferencingComponentDTO(); + dto.setId(component.getIdentifier()); + dto.setName(component.getName()); + + List propertyDescriptors = null; + Collection validationErrors = null; + if (component instanceof ProcessorNode) { + final ProcessorNode node = ((ProcessorNode) component); + dto.setGroupId(node.getProcessGroup().getIdentifier()); + dto.setState(node.getScheduledState().name()); + dto.setActiveThreadCount(node.getActiveThreadCount()); + dto.setType(node.getProcessor().getClass().getName()); + dto.setReferenceType(Processor.class.getSimpleName()); + + propertyDescriptors = node.getProcessor().getPropertyDescriptors(); + validationErrors = node.getValidationErrors(); + } else if (component instanceof ControllerServiceNode) { + final ControllerServiceNode node = ((ControllerServiceNode) component); + dto.setState(node.getState().name()); + dto.setType(node.getControllerServiceImplementation().getClass().getName()); + dto.setReferenceType(ControllerService.class.getSimpleName()); + dto.setReferenceCycle(visited.contains(node)); + + // if we haven't encountered this service before include it's referencing components + if (!dto.getReferenceCycle()) { + dto.setReferencingComponents(createControllerServiceReferencingComponentsDto(node.getReferences(), visited)); + } + + propertyDescriptors = node.getControllerServiceImplementation().getPropertyDescriptors(); + validationErrors = node.getValidationErrors(); + } else if (component instanceof ReportingTaskNode) { + final ReportingTaskNode node = ((ReportingTaskNode) component); + dto.setState(node.getScheduledState().name()); + dto.setActiveThreadCount(node.getActiveThreadCount()); + dto.setType(node.getReportingTask().getClass().getName()); + dto.setReferenceType(ReportingTask.class.getSimpleName()); + + propertyDescriptors = node.getReportingTask().getPropertyDescriptors(); + validationErrors = node.getValidationErrors(); + } + + if (propertyDescriptors != null && !propertyDescriptors.isEmpty()) { + final Map sortedProperties = new TreeMap<>(new Comparator() { + @Override + public int compare(PropertyDescriptor o1, PropertyDescriptor o2) { + return Collator.getInstance(Locale.US).compare(o1.getName(), o2.getName()); + } + }); + sortedProperties.putAll(component.getProperties()); + + final Map orderedProperties = new LinkedHashMap<>(); + for (PropertyDescriptor descriptor : propertyDescriptors) { + orderedProperties.put(descriptor, null); + } + orderedProperties.putAll(sortedProperties); + + // build the descriptor and property dtos + dto.setDescriptors(new LinkedHashMap()); + dto.setProperties(new LinkedHashMap()); + for (final Map.Entry entry : orderedProperties.entrySet()) { + final PropertyDescriptor descriptor = entry.getKey(); + + // store the property descriptor + dto.getDescriptors().put(descriptor.getName(), createPropertyDescriptorDto(descriptor)); + + // determine the property value - don't include sensitive properties + String propertyValue = entry.getValue(); + if (propertyValue != null && descriptor.isSensitive()) { + propertyValue = "********"; + } + + // set the property value + dto.getProperties().put(descriptor.getName(), propertyValue); + } + } + + if (validationErrors != null && !validationErrors.isEmpty()) { + final List errors = new ArrayList<>(); + for (final ValidationResult validationResult : validationErrors) { + errors.add(validationResult.toString()); + } + + dto.setValidationErrors(errors); + } + + referencingComponents.add(dto); + } + + return referencingComponents; + } + public RemoteProcessGroupPortDTO createRemoteProcessGroupPortDto(final RemoteGroupPort port) { if (port == null) { return null; @@ -1143,7 +1382,7 @@ public final class DtoFactory { return types; } - + /** * Creates a ProcessorDTO from the specified ProcessorNode. * @@ -1575,12 +1814,12 @@ public final class DtoFactory { * @param propertyDescriptor * @return */ - private ProcessorConfigDTO.PropertyDescriptorDTO createPropertyDescriptorDto(final PropertyDescriptor propertyDescriptor) { + public PropertyDescriptorDTO createPropertyDescriptorDto(final PropertyDescriptor propertyDescriptor) { if (propertyDescriptor == null) { return null; } - final ProcessorConfigDTO.PropertyDescriptorDTO dto = new ProcessorConfigDTO.PropertyDescriptorDTO(); + final PropertyDescriptorDTO dto = new PropertyDescriptorDTO(); dto.setName(propertyDescriptor.getName()); dto.setDisplayName(propertyDescriptor.getDisplayName()); @@ -1590,18 +1829,20 @@ public final class DtoFactory { dto.setDescription(propertyDescriptor.getDescription()); dto.setDefaultValue(propertyDescriptor.getDefaultValue()); dto.setSupportsEl(propertyDescriptor.isExpressionLanguageSupported()); + + // set the identifies controller service is applicable + if (propertyDescriptor.getControllerServiceDefinition() != null) { + dto.setIdentifiesControllerService(propertyDescriptor.getControllerServiceDefinition().getName()); + } final Class serviceDefinition = propertyDescriptor.getControllerServiceDefinition(); if (propertyDescriptor.getAllowableValues() == null) { if (serviceDefinition == null) { dto.setAllowableValues(null); } else { - final Set allowableValues = new LinkedHashSet<>(); + final List allowableValues = new ArrayList<>(); for (final String serviceIdentifier : controllerServiceLookup.getControllerServiceIdentifiers(serviceDefinition)) { - String displayName = serviceIdentifier; - - // TODO: attempt to get the controller service name - final ControllerService controllerService = controllerServiceLookup.getControllerService(serviceIdentifier); + final String displayName = controllerServiceLookup.getControllerServiceName(serviceIdentifier); final AllowableValueDTO allowableValue = new AllowableValueDTO(); allowableValue.setDisplayName(displayName); @@ -1611,7 +1852,7 @@ public final class DtoFactory { dto.setAllowableValues(allowableValues); } } else { - final Set allowableValues = new LinkedHashSet<>(); + final List allowableValues = new ArrayList<>(); for (final AllowableValue allowableValue : propertyDescriptor.getAllowableValues()) { final AllowableValueDTO allowableValueDto = new AllowableValueDTO(); allowableValueDto.setDisplayName(allowableValue.getDisplayName()); @@ -1642,6 +1883,25 @@ public final class DtoFactory { return copy; } + + public ControllerServiceDTO copy(final ControllerServiceDTO original) { + final ControllerServiceDTO copy = new ControllerServiceDTO(); + copy.setAnnotationData(original.getAnnotationData()); + copy.setAvailability(original.getAvailability()); + copy.setComments(original.getComments()); + copy.setCustomUiUrl(original.getCustomUiUrl()); + copy.setDescriptors(copy(original.getDescriptors())); + copy.setId(original.getId()); + copy.setName(original.getName()); + copy.setProperties(copy(original.getProperties())); + copy.setReferencingComponents(copy(original.getReferencingComponents())); + copy.setState(original.getState()); + copy.setType(original.getType()); + copy.setUri(original.getUri()); + copy.setValidationErrors(copy(original.getValidationErrors())); + return copy; + } + public FunnelDTO copy(final FunnelDTO original) { final FunnelDTO copy = new FunnelDTO(); copy.setId(original.getId()); @@ -2029,14 +2289,17 @@ public final class DtoFactory { /** * Factory method for creating a new RevisionDTO based on this controller. * - * @param revision + * @param lastMod * @return */ - public RevisionDTO createRevisionDTO(Revision revision) { + public RevisionDTO createRevisionDTO(FlowModification lastMod) { + final Revision revision = lastMod.getRevision(); + // create the dto final RevisionDTO revisionDTO = new RevisionDTO(); revisionDTO.setVersion(revision.getVersion()); revisionDTO.setClientId(revision.getClientId()); + revisionDTO.setLastModifier(lastMod.getLastModifier()); return revisionDTO; } @@ -2146,8 +2409,6 @@ public final class DtoFactory { } /* setters */ - public void setProperties(NiFiProperties properties) { - } public void setControllerServiceLookup(ControllerServiceLookup lookup) { this.controllerServiceLookup = lookup; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java index 117555a262..b5e6f7e2f7 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java @@ -52,8 +52,6 @@ import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.repository.ContentNotFoundException; import org.apache.nifi.controller.repository.claim.ContentDirection; -import org.apache.nifi.controller.service.ControllerServiceNode; -import org.apache.nifi.controller.service.ControllerServiceProvider; import org.apache.nifi.controller.status.ProcessGroupStatus; import org.apache.nifi.diagnostics.SystemDiagnostics; import org.apache.nifi.flowfile.FlowFilePrioritizer; @@ -79,11 +77,11 @@ import org.apache.nifi.provenance.search.SearchableField; import org.apache.nifi.remote.RootGroupPort; import org.apache.nifi.reporting.Bulletin; import org.apache.nifi.reporting.BulletinRepository; +import org.apache.nifi.reporting.ReportingTask; import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.search.SearchContext; import org.apache.nifi.search.SearchResult; import org.apache.nifi.search.Searchable; -import org.apache.nifi.web.security.user.NiFiUserUtils; import org.apache.nifi.services.FlowService; import org.apache.nifi.user.NiFiUser; import org.apache.nifi.util.FormatUtils; @@ -110,10 +108,12 @@ import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO; import org.apache.nifi.web.api.dto.status.StatusHistoryDTO; import org.apache.nifi.web.DownloadableContent; import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.ClassUtils; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.admin.service.UserService; import org.apache.nifi.authorization.DownloadAuthorization; import org.apache.nifi.processor.DataUnit; +import org.apache.nifi.web.security.user.NiFiUserUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.security.access.AccessDeniedException; @@ -121,7 +121,7 @@ import org.springframework.security.access.AccessDeniedException; /** * */ -public class ControllerFacade implements ControllerServiceProvider { +public class ControllerFacade { private static final Logger logger = LoggerFactory.getLogger(ControllerFacade.class); @@ -347,6 +347,60 @@ public class ControllerFacade implements ControllerServiceProvider { public Set getFlowFileComparatorTypes() { return dtoFactory.fromDocumentedTypes(ExtensionManager.getExtensions(FlowFilePrioritizer.class)); } + + /** + * Returns whether the specified type implements the specified serviceType. + * + * @param baseType + * @param type + * @return + */ + private boolean implementsServiceType(final String serviceType, final Class type) { + final List> interfaces = ClassUtils.getAllInterfaces(type); + for (final Class i : interfaces) { + if (ControllerService.class.isAssignableFrom(i) && i.getName().equals(serviceType)) { + return true; + } + } + + return false; + } + + /** + * Gets the ControllerService types that this controller supports. + * + * @param serviceType + * @return + */ + public Set getControllerServiceTypes(final String serviceType) { + final Set serviceImplementations = ExtensionManager.getExtensions(ControllerService.class); + + // identify the controller services that implement the specified serviceType if applicable + final Set matchingServiceImplementions; + if (serviceType != null) { + matchingServiceImplementions = new HashSet<>(); + + // check each type and remove those that aren't in the specified ancestry + for (final Class type : serviceImplementations) { + if (implementsServiceType(serviceType, type)) { + matchingServiceImplementions.add(type); + } + } + } else { + matchingServiceImplementions = serviceImplementations; + } + + return dtoFactory.fromDocumentedTypes(matchingServiceImplementions); + } + + /** + * Gets the ReportingTask types that this controller supports. + * + * @return + */ + public Set getReportingTaskTypes() { + return dtoFactory.fromDocumentedTypes(ExtensionManager.getExtensions(ReportingTask.class)); + } /** * Gets the counters for this controller. @@ -371,56 +425,6 @@ public class ControllerFacade implements ControllerServiceProvider { return counter; } - - /** - * Return the controller service for the specified identifier. - * - * @param serviceIdentifier - * @return - */ - @Override - public ControllerService getControllerService(String serviceIdentifier) { - return flowController.getControllerService(serviceIdentifier); - } - - @Override - public ControllerServiceNode createControllerService(String type, String id, boolean firstTimeAdded) { - return flowController.createControllerService(type, id, firstTimeAdded); - } - - public void removeControllerService(ControllerServiceNode serviceNode) { - flowController.removeControllerService(serviceNode); - } - - @Override - public Set getControllerServiceIdentifiers(Class serviceType) { - return flowController.getControllerServiceIdentifiers(serviceType); - } - - @Override - public ControllerServiceNode getControllerServiceNode(final String id) { - return flowController.getControllerServiceNode(id); - } - - @Override - public boolean isControllerServiceEnabled(final ControllerService service) { - return flowController.isControllerServiceEnabled(service); - } - - @Override - public boolean isControllerServiceEnabled(final String serviceIdentifier) { - return flowController.isControllerServiceEnabled(serviceIdentifier); - } - - @Override - public void enableControllerService(final ControllerServiceNode serviceNode) { - flowController.enableControllerService(serviceNode); - } - - @Override - public void disableControllerService(ControllerServiceNode serviceNode) { - flowController.disableControllerService(serviceNode); - } /** * Gets the status of this controller. diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java new file mode 100644 index 0000000000..c1fba0c6b5 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java @@ -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.web.dao; + +import java.util.Set; +import org.apache.nifi.controller.ScheduledState; + +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceReference; +import org.apache.nifi.controller.service.ControllerServiceState; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; + +/** + * + */ +public interface ControllerServiceDAO { + + /** + * Determines if the specified controller service exists. + * + * @param controllerServiceId + * @return + */ + boolean hasControllerService(String controllerServiceId); + + /** + * Creates a controller service. + * + * @param controllerServiceDTO The controller service DTO + * @return The controller service + */ + ControllerServiceNode createControllerService(ControllerServiceDTO controllerServiceDTO); + + /** + * Gets the specified controller service. + * + * @param controllerServiceId The controller service id + * @return The controller service + */ + ControllerServiceNode getControllerService(String controllerServiceId); + + /** + * Gets all of the controller services. + * + * @return The controller services + */ + Set getControllerServices(); + + /** + * Updates the specified controller service. + * + * @param controllerServiceDTO The controller service DTO + * @return The controller service + */ + ControllerServiceNode updateControllerService(ControllerServiceDTO controllerServiceDTO); + + /** + * Updates the referencing components for the specified controller service. + * + * @param controllerServiceId + * @param scheduledState + * @param controllerServiceState the value of state + * @return the org.apache.nifi.controller.service.ControllerServiceReference + */ + ControllerServiceReference updateControllerServiceReferencingComponents(String controllerServiceId, ScheduledState scheduledState, ControllerServiceState controllerServiceState); + + /** + * Determines whether this controller service can be updated. + * + * @param controllerServiceDTO + */ + void verifyUpdate(ControllerServiceDTO controllerServiceDTO); + + /** + * Determines whether the referencing component of the specified controller service can be updated. + * + * @param controllerServiceId + * @param scheduledState + * @param controllerServiceState + */ + void verifyUpdateReferencingComponents(String controllerServiceId, ScheduledState scheduledState, ControllerServiceState controllerServiceState); + + /** + * Determines whether this controller service can be removed. + * + * @param controllerServiceId + */ + void verifyDelete(String controllerServiceId); + + /** + * Deletes the specified controller service. + * + * @param controllerServiceId The controller service id + */ + void deleteControllerService(String controllerServiceId); +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ReportingTaskDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ReportingTaskDAO.java new file mode 100644 index 0000000000..49446d396e --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ReportingTaskDAO.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.dao; + +import java.util.Set; +import org.apache.nifi.controller.ReportingTaskNode; + +import org.apache.nifi.web.api.dto.ReportingTaskDTO; + +/** + * + */ +public interface ReportingTaskDAO { + + /** + * Determines if the specified reporting task exists. + * + * @param reportingTaskId + * @return + */ + boolean hasReportingTask(String reportingTaskId); + + /** + * Creates a reporting task. + * + * @param reportingTaskDTO The reporting task DTO + * @return The reporting task + */ + ReportingTaskNode createReportingTask(ReportingTaskDTO reportingTaskDTO); + + /** + * Gets the specified reporting task. + * + * @param reportingTaskId The reporting task id + * @return The reporting task + */ + ReportingTaskNode getReportingTask(String reportingTaskId); + + /** + * Gets all of the reporting tasks. + * + * @return The reporting tasks + */ + Set getReportingTasks(); + + /** + * Updates the specified reporting task. + * + * @param reportingTaskDTO The reporting task DTO + * @return The reporting task + */ + ReportingTaskNode updateReportingTask(ReportingTaskDTO reportingTaskDTO); + + /** + * Determines whether this reporting task can be updated. + * + * @param reportingTaskDTO + */ + void verifyUpdate(ReportingTaskDTO reportingTaskDTO); + + /** + * Determines whether this reporting task can be removed. + * + * @param reportingTaskId + */ + void verifyDelete(String reportingTaskId); + + /** + * Deletes the specified reporting task. + * + * @param reportingTaskId The reporting task id + */ + void deleteReportingTask(String reportingTaskId); +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java new file mode 100644 index 0000000000..14217c583f --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java @@ -0,0 +1,320 @@ +/* + * 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.dao.impl; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.exception.ControllerServiceInstantiationException; + +import org.apache.nifi.controller.exception.ValidationException; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.controller.service.ControllerServiceReference; +import org.apache.nifi.controller.service.ControllerServiceState; +import org.apache.nifi.web.NiFiCoreException; +import org.apache.nifi.web.ResourceNotFoundException; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; +import org.apache.nifi.web.dao.ControllerServiceDAO; + +public class StandardControllerServiceDAO extends ComponentDAO implements ControllerServiceDAO { + + private ControllerServiceProvider serviceProvider; + + /** + * Locates the specified controller service. + * + * @param controllerServiceId + * @return + */ + private ControllerServiceNode locateControllerService(final String controllerServiceId) { + // get the controller service + final ControllerServiceNode controllerService = serviceProvider.getControllerServiceNode(controllerServiceId); + + // ensure the controller service exists + if (controllerService == null) { + throw new ResourceNotFoundException(String.format("Unable to locate controller service with id '%s'.", controllerServiceId)); + } + + return controllerService; + } + + /** + * Creates a controller service. + * + * @param controllerServiceDTO The controller service DTO + * @return The controller service + */ + @Override + public ControllerServiceNode createControllerService(final ControllerServiceDTO controllerServiceDTO) { + // ensure the type is specified + if (controllerServiceDTO.getType() == null) { + throw new IllegalArgumentException("The controller service type must be specified."); + } + + try { + // create the controller service + final ControllerServiceNode controllerService = serviceProvider.createControllerService(controllerServiceDTO.getType(), controllerServiceDTO.getId(), true); + + // ensure we can perform the update + verifyUpdate(controllerService, controllerServiceDTO); + + // perform the update + configureControllerService(controllerService, controllerServiceDTO); + + return controllerService; + } catch (final ControllerServiceInstantiationException csie) { + throw new NiFiCoreException(csie.getMessage(), csie); + } + } + + /** + * Gets the specified controller service. + * + * @param controllerServiceId The controller service id + * @return The controller service + */ + @Override + public ControllerServiceNode getControllerService(final String controllerServiceId) { + return locateControllerService(controllerServiceId); + } + + /** + * Determines if the specified controller service exists. + * + * @param controllerServiceId + * @return + */ + @Override + public boolean hasControllerService(final String controllerServiceId) { + return serviceProvider.getControllerServiceNode(controllerServiceId) != null; + } + + /** + * Gets all of the controller services. + * + * @return The controller services + */ + @Override + public Set getControllerServices() { + return serviceProvider.getAllControllerServices(); + } + + /** + * Updates the specified controller service. + * + * @param controllerServiceDTO The controller service DTO + * @return The controller service + */ + @Override + public ControllerServiceNode updateControllerService(final ControllerServiceDTO controllerServiceDTO) { + // get the controller service + final ControllerServiceNode controllerService = locateControllerService(controllerServiceDTO.getId()); + + // ensure we can perform the update + verifyUpdate(controllerService, controllerServiceDTO); + + // perform the update + configureControllerService(controllerService, controllerServiceDTO); + + // enable or disable as appropriate + if (isNotNull(controllerServiceDTO.getState())) { + final ControllerServiceState purposedControllerServiceState = ControllerServiceState.valueOf(controllerServiceDTO.getState()); + + // only attempt an action if it is changing + if (!purposedControllerServiceState.equals(controllerService.getState())) { + if (ControllerServiceState.ENABLED.equals(purposedControllerServiceState)) { + serviceProvider.enableControllerService(controllerService); + } else if (ControllerServiceState.DISABLED.equals(purposedControllerServiceState)) { + serviceProvider.disableControllerService(controllerService); + } + } + } + + return controllerService; + } + + @Override + public ControllerServiceReference updateControllerServiceReferencingComponents(final String controllerServiceId, final ScheduledState scheduledState, final ControllerServiceState controllerServiceState) { + // get the controller service + final ControllerServiceNode controllerService = locateControllerService(controllerServiceId); + + // this request is either acting upon referncing services or schedulable components + if (controllerServiceState != null) { + if (ControllerServiceState.ENABLED.equals(controllerServiceState)) { + serviceProvider.enableReferencingServices(controllerService); + } else { + serviceProvider.disableReferencingServices(controllerService); + } + } else if (scheduledState != null) { + if (ScheduledState.RUNNING.equals(scheduledState)) { + serviceProvider.scheduleReferencingComponents(controllerService); + } else { + serviceProvider.unscheduleReferencingComponents(controllerService); + } + } + + return controllerService.getReferences(); + } + + /** + * Validates the specified configuration for the specified controller service. + * + * @param controllerService + * @param controllerServiceDTO + * @return + */ + private List validateProposedConfiguration(final ControllerServiceNode controllerService, final ControllerServiceDTO controllerServiceDTO) { + final List validationErrors = new ArrayList<>(); + return validationErrors; + } + + @Override + public void verifyDelete(final String controllerServiceId) { + final ControllerServiceNode controllerService = locateControllerService(controllerServiceId); + controllerService.verifyCanDelete(); + } + + @Override + public void verifyUpdate(final ControllerServiceDTO controllerServiceDTO) { + final ControllerServiceNode controllerService = locateControllerService(controllerServiceDTO.getId()); + verifyUpdate(controllerService, controllerServiceDTO); + } + + @Override + public void verifyUpdateReferencingComponents(String controllerServiceId, ScheduledState scheduledState, ControllerServiceState controllerServiceState) { + final ControllerServiceNode controllerService = locateControllerService(controllerServiceId); + + if (controllerServiceState != null) { + if (ControllerServiceState.ENABLED.equals(controllerServiceState)) { + serviceProvider.verifyCanEnableReferencingServices(controllerService); + } else { + serviceProvider.verifyCanDisableReferencingServices(controllerService); + } + } else if (scheduledState != null) { + if (ScheduledState.RUNNING.equals(scheduledState)) { + serviceProvider.verifyCanScheduleReferencingComponents(controllerService); + } else { + serviceProvider.verifyCanStopReferencingComponents(controllerService); + } + } + } + + /** + * Verifies the controller service can be updated. + * + * @param controllerService + * @param controllerServiceDTO + */ + private void verifyUpdate(final ControllerServiceNode controllerService, final ControllerServiceDTO controllerServiceDTO) { + // validate the new controller service state if appropriate + if (isNotNull(controllerServiceDTO.getState())) { + try { + // attempt to parse the service state + final ControllerServiceState purposedControllerServiceState = ControllerServiceState.valueOf(controllerServiceDTO.getState()); + + // ensure the state is valid + if (ControllerServiceState.ENABLING.equals(purposedControllerServiceState) || ControllerServiceState.DISABLING.equals(purposedControllerServiceState)) { + throw new IllegalArgumentException(); + } + + // only attempt an action if it is changing + if (!purposedControllerServiceState.equals(controllerService.getState())) { + if (ControllerServiceState.ENABLED.equals(purposedControllerServiceState)) { + controllerService.verifyCanEnable(); + } else if (ControllerServiceState.DISABLED.equals(purposedControllerServiceState)) { + controllerService.verifyCanDisable(); + } + } + } catch (IllegalArgumentException iae) { + throw new IllegalArgumentException("Controller Service state: Value must be one of [ENABLED, DISABLED]"); + } + } + + boolean modificationRequest = false; + if (isAnyNotNull(controllerServiceDTO.getName(), + controllerServiceDTO.getAnnotationData(), + controllerServiceDTO.getComments(), + controllerServiceDTO.getProperties())) { + modificationRequest = true; + + // validate the request + final List requestValidation = validateProposedConfiguration(controllerService, controllerServiceDTO); + + // ensure there was no validation errors + if (!requestValidation.isEmpty()) { + throw new ValidationException(requestValidation); + } + } + + if (modificationRequest) { + controllerService.verifyCanUpdate(); + } + } + + /** + * Configures the specified controller service. + * + * @param controllerService + * @param controllerServiceDTO + */ + private void configureControllerService(final ControllerServiceNode controllerService, final ControllerServiceDTO controllerServiceDTO) { + final String name = controllerServiceDTO.getName(); + final String annotationData = controllerServiceDTO.getAnnotationData(); + final String comments = controllerServiceDTO.getComments(); + final Map properties = controllerServiceDTO.getProperties(); + + if (isNotNull(name)) { + controllerService.setName(name); + } + if (isNotNull(annotationData)) { + controllerService.setAnnotationData(annotationData); + } + if (isNotNull(comments)) { + controllerService.setComments(comments); + } + if (isNotNull(properties)) { + for (final Map.Entry entry : properties.entrySet()) { + final String propName = entry.getKey(); + final String propVal = entry.getValue(); + if (isNotNull(propName) && propVal == null) { + controllerService.removeProperty(propName); + } else if (isNotNull(propName)) { + controllerService.setProperty(propName, propVal); + } + } + } + } + + /** + * Deletes the specified controller service. + * + * @param controllerServiceId The controller service id + */ + @Override + public void deleteControllerService(String controllerServiceId) { + final ControllerServiceNode controllerService = locateControllerService(controllerServiceId); + serviceProvider.removeControllerService(controllerService); + } + + /* setters */ + public void setServiceProvider(ControllerServiceProvider serviceProvider) { + this.serviceProvider = serviceProvider; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java index 633f8e20ed..b291b4f63b 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java @@ -32,7 +32,7 @@ import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.exception.ProcessorInstantiationException; -import org.apache.nifi.controller.exception.ProcessorLifeCycleException; +import org.apache.nifi.controller.exception.ComponentLifeCycleException; import org.apache.nifi.controller.exception.ValidationException; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.logging.LogLevel; @@ -99,6 +99,11 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO { if (processorDTO.getParentGroupId() != null && !flowController.areGroupsSame(groupId, processorDTO.getParentGroupId())) { throw new IllegalArgumentException("Cannot specify a different Parent Group ID than the Group to which the Processor is being added."); } + + // ensure the type is specified + if (processorDTO.getType() == null) { + throw new IllegalArgumentException("The processor type must be specified."); + } // get the group to add the processor to ProcessGroup group = locateProcessGroup(flowController, groupId); @@ -119,7 +124,7 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO { return processor; } catch (ProcessorInstantiationException pse) { throw new NiFiCoreException(String.format("Unable to create processor of type %s due to: %s", processorDTO.getType(), pse.getMessage()), pse); - } catch (IllegalStateException | ProcessorLifeCycleException ise) { + } catch (IllegalStateException | ComponentLifeCycleException ise) { throw new NiFiCoreException(ise.getMessage(), ise); } } @@ -455,7 +460,7 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO { parentGroup.disableProcessor(processor); break; } - } catch (IllegalStateException | ProcessorLifeCycleException ise) { + } catch (IllegalStateException | ComponentLifeCycleException ise) { throw new NiFiCoreException(ise.getMessage(), ise); } catch (RejectedExecutionException ree) { throw new NiFiCoreException("Unable to schedule all tasks for the specified processor.", ree); @@ -491,7 +496,7 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO { try { // attempt remove the processor group.removeProcessor(processor); - } catch (ProcessorLifeCycleException plce) { + } catch (ComponentLifeCycleException plce) { throw new NiFiCoreException(plce.getMessage(), plce); } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java new file mode 100644 index 0000000000..46b7070879 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java @@ -0,0 +1,365 @@ +/* + * 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.dao.impl; + +import java.text.ParseException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.RejectedExecutionException; +import java.util.regex.Matcher; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.exception.ComponentLifeCycleException; + +import org.apache.nifi.controller.exception.ValidationException; +import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException; +import org.apache.nifi.controller.reporting.ReportingTaskProvider; +import org.apache.nifi.scheduling.SchedulingStrategy; +import org.apache.nifi.util.FormatUtils; +import org.apache.nifi.web.NiFiCoreException; +import org.apache.nifi.web.ResourceNotFoundException; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; +import org.apache.nifi.web.dao.ReportingTaskDAO; +import org.quartz.CronExpression; + +public class StandardReportingTaskDAO extends ComponentDAO implements ReportingTaskDAO { + + private ReportingTaskProvider reportingTaskProvider; + + /** + * Locates the specified reporting task. + * + * @param reportingTaskId + * @return + */ + private ReportingTaskNode locateReportingTask(final String reportingTaskId) { + // get the reporting task + final ReportingTaskNode reportingTask = reportingTaskProvider.getReportingTaskNode(reportingTaskId); + + // ensure the reporting task exists + if (reportingTask == null) { + throw new ResourceNotFoundException(String.format("Unable to locate reporting task with id '%s'.", reportingTaskId)); + } + + return reportingTask; + } + + /** + * Creates a reporting task. + * + * @param reportingTaskDTO The reporting task DTO + * @return The reporting task + */ + @Override + public ReportingTaskNode createReportingTask(final ReportingTaskDTO reportingTaskDTO) { + // ensure the type is specified + if (reportingTaskDTO.getType() == null) { + throw new IllegalArgumentException("The reporting task type must be specified."); + } + + try { + // create the reporting task + final ReportingTaskNode reportingTask = reportingTaskProvider.createReportingTask(reportingTaskDTO.getType(), reportingTaskDTO.getId(), true); + + // ensure we can perform the update + verifyUpdate(reportingTask, reportingTaskDTO); + + // perform the update + configureReportingTask(reportingTask, reportingTaskDTO); + + return reportingTask; + } catch (ReportingTaskInstantiationException rtie) { + throw new NiFiCoreException(rtie.getMessage(), rtie); + } + } + + /** + * Gets the specified reporting task. + * + * @param reportingTaskId The reporting task id + * @return The reporting task + */ + @Override + public ReportingTaskNode getReportingTask(final String reportingTaskId) { + return locateReportingTask(reportingTaskId); + } + + /** + * Determines if the specified reporting task exists. + * + * @param reportingTaskId + * @return + */ + @Override + public boolean hasReportingTask(final String reportingTaskId) { + return reportingTaskProvider.getReportingTaskNode(reportingTaskId) != null; + } + + /** + * Gets all of the reporting tasks. + * + * @return The reporting tasks + */ + @Override + public Set getReportingTasks() { + return reportingTaskProvider.getAllReportingTasks(); + } + + /** + * Updates the specified reporting task. + * + * @param reportingTaskDTO The reporting task DTO + * @return The reporting task + */ + @Override + public ReportingTaskNode updateReportingTask(final ReportingTaskDTO reportingTaskDTO) { + // get the reporting task + final ReportingTaskNode reportingTask = locateReportingTask(reportingTaskDTO.getId()); + + // ensure we can perform the update + verifyUpdate(reportingTask, reportingTaskDTO); + + // perform the update + configureReportingTask(reportingTask, reportingTaskDTO); + + // configure scheduled state + // see if an update is necessary + if (isNotNull(reportingTaskDTO.getState())) { + final ScheduledState purposedScheduledState = ScheduledState.valueOf(reportingTaskDTO.getState()); + + // only attempt an action if it is changing + if (!purposedScheduledState.equals(reportingTask.getScheduledState())) { + try { + // perform the appropriate action + switch (purposedScheduledState) { + case RUNNING: + reportingTaskProvider.startReportingTask(reportingTask); + break; + case STOPPED: + switch (reportingTask.getScheduledState()) { + case RUNNING: + reportingTaskProvider.stopReportingTask(reportingTask); + break; + case DISABLED: + reportingTaskProvider.enableReportingTask(reportingTask); + break; + } + break; + case DISABLED: + reportingTaskProvider.disableReportingTask(reportingTask); + break; + } + } catch (IllegalStateException | ComponentLifeCycleException ise) { + throw new NiFiCoreException(ise.getMessage(), ise); + } catch (RejectedExecutionException ree) { + throw new NiFiCoreException("Unable to schedule all tasks for the specified reporting task.", ree); + } catch (NullPointerException npe) { + throw new NiFiCoreException("Unable to update reporting task run state.", npe); + } catch (Exception e) { + throw new NiFiCoreException("Unable to update reporting task run state: " + e, e); + } + } + } + + return reportingTask; + } + + /** + * Validates the specified configuration for the specified reporting task. + * + * @param reportingTask + * @param reportingTaskDTO + * @return + */ + private List validateProposedConfiguration(final ReportingTaskNode reportingTask, final ReportingTaskDTO reportingTaskDTO) { + final List validationErrors = new ArrayList<>(); + + // get the current scheduling strategy + SchedulingStrategy schedulingStrategy = reportingTask.getSchedulingStrategy(); + + // validate the new scheduling strategy if appropriate + if (isNotNull(reportingTaskDTO.getSchedulingStrategy())) { + try { + // this will be the new scheduling strategy so use it + schedulingStrategy = SchedulingStrategy.valueOf(reportingTaskDTO.getSchedulingStrategy()); + } catch (IllegalArgumentException iae) { + validationErrors.add(String.format("Scheduling strategy: Value must be one of [%s]", StringUtils.join(SchedulingStrategy.values(), ", "))); + } + } + + // validate the scheduling period based on the scheduling strategy + if (isNotNull(reportingTaskDTO.getSchedulingPeriod())) { + switch (schedulingStrategy) { + case TIMER_DRIVEN: + final Matcher schedulingMatcher = FormatUtils.TIME_DURATION_PATTERN.matcher(reportingTaskDTO.getSchedulingPeriod()); + if (!schedulingMatcher.matches()) { + validationErrors.add("Scheduling period is not a valid time duration (ie 30 sec, 5 min)"); + } + break; + case CRON_DRIVEN: + try { + new CronExpression(reportingTaskDTO.getSchedulingPeriod()); + } catch (final ParseException pe) { + throw new IllegalArgumentException(String.format("Scheduling Period '%s' is not a valid cron expression: %s", reportingTaskDTO.getSchedulingPeriod(), pe.getMessage())); + } catch (final Exception e) { + throw new IllegalArgumentException("Scheduling Period is not a valid cron expression: " + reportingTaskDTO.getSchedulingPeriod()); + } + break; + } + } + + return validationErrors; + } + + @Override + public void verifyDelete(final String reportingTaskId) { + final ReportingTaskNode reportingTask = locateReportingTask(reportingTaskId); + reportingTask.verifyCanDelete(); + } + + @Override + public void verifyUpdate(final ReportingTaskDTO reportingTaskDTO) { + final ReportingTaskNode reportingTask = locateReportingTask(reportingTaskDTO.getId()); + verifyUpdate(reportingTask, reportingTaskDTO); + } + + /** + * Verifies the reporting task can be updated. + * + * @param reportingTask + * @param reportingTaskDTO + */ + private void verifyUpdate(final ReportingTaskNode reportingTask, final ReportingTaskDTO reportingTaskDTO) { + // ensure the state, if specified, is valid + if (isNotNull(reportingTaskDTO.getState())) { + try { + final ScheduledState purposedScheduledState = ScheduledState.valueOf(reportingTaskDTO.getState()); + + // only attempt an action if it is changing + if (!purposedScheduledState.equals(reportingTask.getScheduledState())) { + // perform the appropriate action + switch (purposedScheduledState) { + case RUNNING: + reportingTask.verifyCanStart(); + break; + case STOPPED: + switch (reportingTask.getScheduledState()) { + case RUNNING: + reportingTask.verifyCanStop(); + break; + case DISABLED: + reportingTask.verifyCanEnable(); + break; + } + break; + case DISABLED: + reportingTask.verifyCanDisable(); + break; + } + } + } catch (IllegalArgumentException iae) { + throw new IllegalArgumentException(String.format( + "The specified reporting task state (%s) is not valid. Valid options are 'RUNNING', 'STOPPED', and 'DISABLED'.", + reportingTaskDTO.getState())); + } + } + + boolean modificationRequest = false; + if (isAnyNotNull(reportingTaskDTO.getName(), + reportingTaskDTO.getSchedulingStrategy(), + reportingTaskDTO.getSchedulingPeriod(), + reportingTaskDTO.getAnnotationData(), + reportingTaskDTO.getProperties())) { + modificationRequest = true; + + // validate the request + final List requestValidation = validateProposedConfiguration(reportingTask, reportingTaskDTO); + + // ensure there was no validation errors + if (!requestValidation.isEmpty()) { + throw new ValidationException(requestValidation); + } + } + + if (modificationRequest) { + reportingTask.verifyCanUpdate(); + } + } + + /** + * Configures the specified reporting task. + * + * @param reportingTask + * @param reportingTaskDTO + */ + private void configureReportingTask(final ReportingTaskNode reportingTask, final ReportingTaskDTO reportingTaskDTO) { + final String name = reportingTaskDTO.getName(); + final String schedulingStrategy = reportingTaskDTO.getSchedulingStrategy(); + final String schedulingPeriod = reportingTaskDTO.getSchedulingPeriod(); + final String annotationData = reportingTaskDTO.getAnnotationData(); + final String comments = reportingTaskDTO.getComments(); + final Map properties = reportingTaskDTO.getProperties(); + + // ensure scheduling strategy is set first + if (isNotNull(schedulingStrategy)) { + reportingTask.setSchedulingStrategy(SchedulingStrategy.valueOf(schedulingStrategy)); + } + + if (isNotNull(name)) { + reportingTask.setName(name); + } + if (isNotNull(schedulingPeriod)) { + reportingTask.setScheduldingPeriod(schedulingPeriod); + } + if (isNotNull(annotationData)) { + reportingTask.setAnnotationData(annotationData); + } + if (isNotNull(comments)) { + reportingTask.setComments(comments); + } + if (isNotNull(properties)) { + for (final Map.Entry entry : properties.entrySet()) { + final String propName = entry.getKey(); + final String propVal = entry.getValue(); + if (isNotNull(propName) && propVal == null) { + reportingTask.removeProperty(propName); + } else if (isNotNull(propName)) { + reportingTask.setProperty(propName, propVal); + } + } + } + } + + /** + * Deletes the specified reporting task. + * + * @param reportingTaskId The reporting task id + */ + @Override + public void deleteReportingTask(String reportingTaskId) { + final ReportingTaskNode reportingTask = locateReportingTask(reportingTaskId); + reportingTaskProvider.removeReportingTask(reportingTask); + } + + /* setters */ + public void setReportingTaskProvider(ReportingTaskProvider reportingTaskProvider) { + this.reportingTaskProvider = reportingTaskProvider; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardSnippetDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardSnippetDAO.java index 92e3a8d90a..6447464a84 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardSnippetDAO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardSnippetDAO.java @@ -26,9 +26,11 @@ import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.Snippet; import org.apache.nifi.controller.StandardSnippet; import org.apache.nifi.controller.exception.ProcessorInstantiationException; +import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.web.NiFiCoreException; import org.apache.nifi.web.ResourceNotFoundException; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.FlowSnippetDTO; import org.apache.nifi.web.api.dto.ProcessGroupDTO; import org.apache.nifi.web.api.dto.ProcessorConfigDTO; @@ -36,7 +38,6 @@ import org.apache.nifi.web.api.dto.ProcessorDTO; import org.apache.nifi.web.api.dto.SnippetDTO; import org.apache.nifi.web.dao.SnippetDAO; import org.apache.nifi.web.util.SnippetUtils; - import org.apache.commons.lang3.StringUtils; /** @@ -285,9 +286,13 @@ public class StandardSnippetDAO implements SnippetDAO { if (snippet != null) { // go through each processor if specified if (snippet.getProcessors() != null) { - lookupSensitiveProperties(snippet.getProcessors()); + lookupSensitiveProcessorProperties(snippet.getProcessors()); } + if ( snippet.getControllerServices() != null ) { + lookupSensitiveControllerServiceProperties(snippet.getControllerServices()); + } + // go through each process group if specified if (snippet.getProcessGroups() != null) { for (final ProcessGroupDTO group : snippet.getProcessGroups()) { @@ -303,7 +308,7 @@ public class StandardSnippetDAO implements SnippetDAO { * * @param snippet */ - private void lookupSensitiveProperties(final Set processors) { + private void lookupSensitiveProcessorProperties(final Set processors) { final ProcessGroup rootGroup = flowController.getGroup(flowController.getRootGroupId()); // go through each processor @@ -331,6 +336,31 @@ public class StandardSnippetDAO implements SnippetDAO { } } } + + private void lookupSensitiveControllerServiceProperties(final Set controllerServices) { + // go through each service + for (final ControllerServiceDTO serviceDTO : controllerServices) { + + // ensure that some property configuration have been specified + final Map serviceProperties = serviceDTO.getProperties(); + if (serviceProperties != null) { + // find the corresponding controller service + final ControllerServiceNode serviceNode = flowController.getControllerServiceNode(serviceDTO.getId()); + if (serviceNode == null) { + throw new IllegalArgumentException(String.format("Unable to create snippet because Controller Service '%s' could not be found", serviceDTO.getId())); + } + + // look for sensitive properties get the actual value + for (Entry entry : serviceNode.getProperties().entrySet()) { + final PropertyDescriptor descriptor = entry.getKey(); + + if (descriptor.isSensitive()) { + serviceProperties.put(descriptor.getName(), entry.getValue()); + } + } + } + } + } /* setters */ public void setFlowController(FlowController flowController) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/ControllerServiceProviderFactoryBean.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/ControllerServiceProviderFactoryBean.java new file mode 100644 index 0000000000..5c10de6af5 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/ControllerServiceProviderFactoryBean.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.spring; + +import org.apache.nifi.cluster.manager.impl.WebClusterManager; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.util.NiFiProperties; +import org.springframework.beans.BeansException; +import org.springframework.beans.factory.FactoryBean; +import org.springframework.context.ApplicationContext; +import org.springframework.context.ApplicationContextAware; + +/** + * + */ +public class ControllerServiceProviderFactoryBean implements FactoryBean, ApplicationContextAware { + + private ApplicationContext context; + private ControllerServiceProvider controllerServiceProvider; + private NiFiProperties properties; + + @Override + public Object getObject() throws Exception { + if (controllerServiceProvider == null) { + if (properties.isClusterManager()) { + controllerServiceProvider = context.getBean("clusterManager", WebClusterManager.class); + } else { + controllerServiceProvider = context.getBean("flowController", FlowController.class); + } + } + + return controllerServiceProvider; + } + + @Override + public Class getObjectType() { + return ControllerServiceProvider.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } + + @Override + public void setApplicationContext(ApplicationContext context) throws BeansException { + this.context = context; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/OptimisticLockingManagerFactoryBean.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/OptimisticLockingManagerFactoryBean.java new file mode 100644 index 0000000000..84367932b5 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/OptimisticLockingManagerFactoryBean.java @@ -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.web.spring; + +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.OptimisticLockingManager; +import org.apache.nifi.web.StandardOptimisticLockingManager; +import org.springframework.beans.BeansException; +import org.springframework.beans.factory.FactoryBean; +import org.springframework.context.ApplicationContext; +import org.springframework.context.ApplicationContextAware; + +/** + * + */ +public class OptimisticLockingManagerFactoryBean implements FactoryBean, ApplicationContextAware { + + private ApplicationContext context; + private OptimisticLockingManager optimisticLockingManager; + private NiFiProperties properties; + + @Override + public Object getObject() throws Exception { + if (optimisticLockingManager == null) { + if (properties.isClusterManager()) { + optimisticLockingManager = context.getBean("clusterManagerOptimisticLockingManager", OptimisticLockingManager.class); + } else { + optimisticLockingManager = new StandardOptimisticLockingManager(); + } + } + + return optimisticLockingManager; + } + + @Override + public Class getObjectType() { + return OptimisticLockingManager.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } + + @Override + public void setApplicationContext(ApplicationContext context) throws BeansException { + this.context = context; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/ReportingTaskProviderFactoryBean.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/ReportingTaskProviderFactoryBean.java new file mode 100644 index 0000000000..d344fa6a06 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/ReportingTaskProviderFactoryBean.java @@ -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.web.spring; + +import org.apache.nifi.cluster.manager.impl.WebClusterManager; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.reporting.ReportingTaskProvider; +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.dao.ControllerServiceDAO; +import org.springframework.beans.BeansException; +import org.springframework.beans.factory.FactoryBean; +import org.springframework.context.ApplicationContext; +import org.springframework.context.ApplicationContextAware; + +/** + * + */ +public class ReportingTaskProviderFactoryBean implements FactoryBean, ApplicationContextAware { + + private ApplicationContext context; + private ReportingTaskProvider reportingTaskProvider; + private NiFiProperties properties; + + @Override + public Object getObject() throws Exception { + if (reportingTaskProvider == null) { + if (properties.isClusterManager()) { + reportingTaskProvider = context.getBean("clusterManager", WebClusterManager.class); + } else { + reportingTaskProvider = context.getBean("flowController", FlowController.class); + } + } + + return reportingTaskProvider; + } + + @Override + public Class getObjectType() { + return ReportingTaskProvider.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } + + @Override + public void setApplicationContext(ApplicationContext context) throws BeansException { + this.context = context; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/Availability.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/Availability.java new file mode 100644 index 0000000000..29ba4f8e21 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/Availability.java @@ -0,0 +1,34 @@ +/* + * 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.util; + +/** + * Where a given controller service or reporting task should run. + */ +public enum Availability { + + /** + * Service or reporting task will run only on the NiFi Cluster Manager (NCM) + */ + NCM, + + /** + * Service or reporting task will run only on NiFi Nodes (or standalone + * instance, if not clustered) + */ + NODE; +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java index 86530947cc..40e5730527 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java @@ -18,6 +18,7 @@ package org.apache.nifi.web.util; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashSet; @@ -28,6 +29,7 @@ import java.util.UUID; import org.apache.nifi.cluster.context.ClusterContext; import org.apache.nifi.cluster.context.ClusterContextThreadLocal; +import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; import org.apache.nifi.connectable.Funnel; @@ -37,17 +39,22 @@ import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.Snippet; import org.apache.nifi.controller.label.Label; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceState; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.groups.RemoteProcessGroup; import org.apache.nifi.web.api.dto.ConnectableDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.DtoFactory; import org.apache.nifi.web.api.dto.FlowSnippetDTO; import org.apache.nifi.web.api.dto.FunnelDTO; import org.apache.nifi.web.api.dto.LabelDTO; import org.apache.nifi.web.api.dto.PortDTO; import org.apache.nifi.web.api.dto.ProcessGroupDTO; +import org.apache.nifi.web.api.dto.ProcessorConfigDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupContentsDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO; @@ -181,11 +188,100 @@ public final class SnippetUtils { snippetDto.setRemoteProcessGroups(remoteProcessGroups); } + addControllerServicesToSnippet(snippetDto); + return snippetDto; } + + private void addControllerServicesToSnippet(final FlowSnippetDTO snippetDto) { + for ( final ProcessorDTO processorDto : snippetDto.getProcessors() ) { + addControllerServicesToSnippet(snippetDto, processorDto); + } + + for ( final ProcessGroupDTO processGroupDto : snippetDto.getProcessGroups() ) { + final FlowSnippetDTO childGroupDto = processGroupDto.getContents(); + addControllerServicesToSnippet(childGroupDto); + } + } + + private void addControllerServicesToSnippet(final FlowSnippetDTO snippet, final ProcessorDTO processorDto) { + final ProcessorConfigDTO configDto = processorDto.getConfig(); + if ( configDto == null ) { + return; + } + + final Map descriptors = configDto.getDescriptors(); + final Map properties = configDto.getProperties(); + + if ( properties != null && descriptors != null ) { + for ( final Map.Entry entry : properties.entrySet() ) { + final String propName = entry.getKey(); + final String propValue = entry.getValue(); + if ( propValue == null ) { + continue; + } + + final PropertyDescriptorDTO propertyDescriptorDto = descriptors.get(propName); + if ( propertyDescriptorDto != null && propertyDescriptorDto.getIdentifiesControllerService() != null ) { + final ControllerServiceNode serviceNode = flowController.getControllerServiceNode(propValue); + if ( serviceNode != null ) { + addControllerServicesToSnippet(snippet, serviceNode); + } + } + } + } + } + + private void addControllerServicesToSnippet(final FlowSnippetDTO snippet, final ControllerServiceNode serviceNode) { + if ( isServicePresent(serviceNode.getIdentifier(), snippet.getControllerServices()) ) { + return; + } + + final ControllerServiceDTO serviceNodeDto = dtoFactory.createControllerServiceDto(serviceNode); + Set existingServiceDtos = snippet.getControllerServices(); + if ( existingServiceDtos == null ) { + existingServiceDtos = new HashSet<>(); + snippet.setControllerServices(existingServiceDtos); + } + existingServiceDtos.add(serviceNodeDto); + for ( final Map.Entry entry : serviceNode.getProperties().entrySet() ) { + final PropertyDescriptor descriptor = entry.getKey(); + final String propertyValue = entry.getValue(); + + if ( descriptor.getControllerServiceDefinition() != null ) { + final ControllerServiceNode referencedNode = flowController.getControllerServiceNode(propertyValue); + if ( referencedNode == null ) { + throw new IllegalStateException("Controller Service with ID " + propertyValue + " is referenced in template but cannot be found"); + } + + final String referencedNodeId = referencedNode.getIdentifier(); + + final boolean alreadyPresent = isServicePresent(referencedNodeId, snippet.getControllerServices()); + if ( !alreadyPresent ) { + addControllerServicesToSnippet(snippet, referencedNode); + } + } + } + } + + private boolean isServicePresent(final String serviceId, final Collection services) { + if ( services == null ) { + return false; + } + + for ( final ControllerServiceDTO existingService : services ) { + if ( serviceId.equals(existingService.getId()) ) { + return true; + } + } + + return false; + } + + public FlowSnippetDTO copy(final FlowSnippetDTO snippetContents, final ProcessGroup group) { - final FlowSnippetDTO snippetCopy = copyContentsForGroup(snippetContents, group.getIdentifier(), null); + final FlowSnippetDTO snippetCopy = copyContentsForGroup(snippetContents, group.getIdentifier(), null, null); resolveNameConflicts(snippetCopy, group); return snippetCopy; } @@ -240,9 +336,48 @@ public final class SnippetUtils { } } - private FlowSnippetDTO copyContentsForGroup(final FlowSnippetDTO snippetContents, final String groupId, final Map parentConnectableMap) { + private FlowSnippetDTO copyContentsForGroup(final FlowSnippetDTO snippetContents, final String groupId, final Map parentConnectableMap, Map serviceIdMap) { final FlowSnippetDTO snippetContentsCopy = new FlowSnippetDTO(); + // + // Copy the Controller Services + // + if ( serviceIdMap == null ) { + serviceIdMap = new HashMap<>(); + final Set services = new HashSet<>(); + if ( snippetContents.getControllerServices() != null ) { + for (final ControllerServiceDTO serviceDTO : snippetContents.getControllerServices() ) { + final ControllerServiceDTO service = dtoFactory.copy(serviceDTO); + service.setId(generateId(serviceDTO.getId())); + service.setState(ControllerServiceState.DISABLED.name()); + services.add(service); + + // Map old service ID to new service ID so that we can make sure that we reference the new ones. + serviceIdMap.put(serviceDTO.getId(), service.getId()); + } + } + + // if there is any controller service that maps to another controller service, update the id's + for ( final ControllerServiceDTO serviceDTO : services ) { + final Map properties = serviceDTO.getProperties(); + final Map descriptors = serviceDTO.getDescriptors(); + if ( properties != null && descriptors != null ) { + for ( final PropertyDescriptorDTO descriptor : descriptors.values() ) { + if ( descriptor.getIdentifiesControllerService() != null ) { + final String currentServiceId = properties.get(descriptor.getName()); + if ( currentServiceId == null ) { + continue; + } + + final String newServiceId = serviceIdMap.get(currentServiceId); + properties.put(descriptor.getName(), newServiceId); + } + } + } + } + snippetContentsCopy.setControllerServices(services); + } + // // Copy the labels // @@ -332,6 +467,9 @@ public final class SnippetUtils { } snippetContentsCopy.setProcessors(processors); + // if there is any controller service that maps to another controller service, update the id's + updateControllerServiceIdentifiers(snippetContentsCopy, serviceIdMap); + // // Copy ProcessGroups // @@ -344,7 +482,7 @@ public final class SnippetUtils { cp.setParentGroupId(groupId); // copy the contents of this group - we do not copy via the dto factory since we want to specify new ids - final FlowSnippetDTO contentsCopy = copyContentsForGroup(groupDTO.getContents(), cp.getId(), connectableMap); + final FlowSnippetDTO contentsCopy = copyContentsForGroup(groupDTO.getContents(), cp.getId(), connectableMap, serviceIdMap); cp.setContents(contentsCopy); groups.add(cp); } @@ -396,6 +534,43 @@ public final class SnippetUtils { return snippetContentsCopy; } + + + private void updateControllerServiceIdentifiers(final FlowSnippetDTO snippet, final Map serviceIdMap) { + final Set processors = snippet.getProcessors(); + if ( processors != null ) { + for ( final ProcessorDTO processor : processors ) { + updateControllerServiceIdentifiers(processor.getConfig(), serviceIdMap); + } + } + + for ( final ProcessGroupDTO processGroupDto : snippet.getProcessGroups() ) { + updateControllerServiceIdentifiers(processGroupDto.getContents(), serviceIdMap); + } + } + + private void updateControllerServiceIdentifiers(final ProcessorConfigDTO configDto, final Map serviceIdMap) { + if ( configDto == null ) { + return; + } + + final Map properties = configDto.getProperties(); + final Map descriptors = configDto.getDescriptors(); + if ( properties != null && descriptors != null ) { + for ( final PropertyDescriptorDTO descriptor : descriptors.values() ) { + if ( descriptor.getIdentifiesControllerService() != null ) { + final String currentServiceId = properties.get(descriptor.getName()); + if ( currentServiceId == null ) { + continue; + } + + final String newServiceId = serviceIdMap.get(currentServiceId); + properties.put(descriptor.getName(), newServiceId); + } + } + } + } + /** * Generates a new id for the current id that is specified. If no seed is diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml index a822442bca..e034baa0bf 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml @@ -24,11 +24,17 @@ http://www.springframework.org/schema/context http://www.springframework.org/schema/context/spring-context-3.1.xsd http://www.springframework.org/schema/aop http://www.springframework.org/schema/aop/spring-aop-3.1.xsd"> + + + + + + + + - - - - + + @@ -40,8 +46,7 @@ - - + @@ -75,6 +80,12 @@ + + + + + + @@ -101,22 +112,35 @@ + + - + + - + + + + + + + + + + + @@ -133,6 +157,16 @@ + + + + + + + + + + @@ -236,6 +270,7 @@ + @@ -303,6 +338,16 @@ + + + + + + + + + + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java index dd9bb73276..1f286096bb 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java @@ -18,12 +18,14 @@ package org.apache.nifi.integration.util; import com.sun.jersey.api.client.Client; import java.io.File; +import java.util.Collections; import javax.servlet.ServletContext; import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.web.util.WebUtils; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.framework.security.util.SslContextFactory; import org.apache.nifi.services.FlowService; +import org.apache.nifi.ui.extension.UiExtensionMapping; import org.eclipse.jetty.server.HttpConfiguration; import org.eclipse.jetty.server.HttpConnectionFactory; import org.eclipse.jetty.server.SecureRequestCustomizer; @@ -160,6 +162,9 @@ public class NiFiTestServer { */ public void startServer() throws Exception { jetty.start(); + + // ensure the ui extensions are set + webappContext.getServletContext().setAttribute("nifi-ui-extensions", new UiExtensionMapping(Collections.EMPTY_MAP)); } /** diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/java/org/apache/nifi/web/ContentViewerController.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/java/org/apache/nifi/web/ContentViewerController.java index d9b082d9ba..4d99a698a1 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/java/org/apache/nifi/web/ContentViewerController.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/java/org/apache/nifi/web/ContentViewerController.java @@ -68,10 +68,21 @@ public class ContentViewerController extends HttpServlet { final ServletContext servletContext = request.getServletContext(); final ContentAccess contentAccess = (ContentAccess) servletContext.getAttribute("nifi-content-access"); + final ContentRequestContext contentRequest = getContentRequest(request); + if (contentRequest.getDataUri() == null) { + request.setAttribute("title", "Error"); + request.setAttribute("messages", "The data reference must be specified."); + + // forward to the error page + final ServletContext viewerContext = servletContext.getContext("/nifi"); + viewerContext.getRequestDispatcher("/message").forward(request, response); + return; + } + // get the content final DownloadableContent downloadableContent; try { - downloadableContent = contentAccess.getContent(getContentRequest(request)); + downloadableContent = contentAccess.getContent(contentRequest); } catch (final ResourceNotFoundException rnfe) { request.setAttribute("title", "Error"); request.setAttribute("messages", "Unable to find the specified content"); @@ -138,9 +149,6 @@ public class ContentViewerController extends HttpServlet { final String mimeType = mediatype.toString(); // add attributes needed for the header - final StringBuffer requestUrl = request.getRequestURL(); - request.setAttribute("requestUrl", requestUrl.toString()); - request.setAttribute("dataRef", request.getParameter("ref")); request.setAttribute("filename", downloadableContent.getFilename()); request.setAttribute("contentType", mimeType); @@ -148,8 +156,6 @@ public class ContentViewerController extends HttpServlet { request.getRequestDispatcher("/WEB-INF/jsp/header.jsp").include(request, response); // remove the attributes needed for the header - request.removeAttribute("requestUrl"); - request.removeAttribute("dataRef"); request.removeAttribute("filename"); request.removeAttribute("contentType"); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/header.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/header.jsp index 82382f619d..b847dd5a65 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/header.jsp +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/header.jsp @@ -31,8 +31,8 @@ + + + +
View as
-
${filename}
-
${contentType}
+
<%= request.getAttribute("filename") == null ? "" : org.apache.nifi.util.EscapeUtils.escapeHtml(request.getAttribute("filename").toString()) %>
+
<%= request.getAttribute("contentType") == null ? "" : org.apache.nifi.util.EscapeUtils.escapeHtml(request.getAttribute("contentType").toString()) %>
\ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/hexview.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/hexview.jsp index c6e7f382ff..f4e9fd29de 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/hexview.jsp +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/hexview.jsp @@ -29,4 +29,4 @@
-
Showing up to 1.5kb
\ No newline at end of file +
Showing up to 1.5 KB
\ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/css/main.css b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/css/main.css index a66198de20..fe68b627f6 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/css/main.css +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/css/main.css @@ -87,7 +87,7 @@ padding: 2px; } -#trancation-message { +#truncation-message { position: absolute; left: 100px; bottom: 35px; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/js/hexview/hexview.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/js/hexview/hexview.js index 7c461d57f1..65fee95521 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/js/hexview/hexview.js +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/js/hexview/hexview.js @@ -148,7 +148,7 @@ $(document).ready(function () { $("table", div).addClass("hexviewerwindow_table"); $("table", div).append("").addClass("hexviewerwindow"); - $("table tr:last", div).append("" + (decimal_offset ? ("00000000"+offset).slice(-8) : "0x" + dec_to_hex8(offset)) + ""); + $("table tr:last", div).append("" + escapeHtml((decimal_offset ? ("00000000"+offset).slice(-8) : "0x" + dec_to_hex8(offset))) + ""); $("table tr td:last", div).addClass("hexviewerwindow_offset"); var runlen = 0; @@ -162,7 +162,7 @@ $(document).ready(function () { num += dec2_to_hex(line_data.charCodeAt(i+j)); } - $("table tr:last", div).append("" + (hide_0x ? "" : "0x") + num + ""); + $("table tr:last", div).append("" + escapeHtml((hide_0x ? "" : "0x") + num) + ""); apply_highlights(offset+i); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml index 5aa9a06480..2aadb03c4c 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml @@ -26,12 +26,18 @@ org.apache.commons commons-lang3 + provided org.apache.nifi nifi-nar-utils provided + + org.apache.nifi + nifi-utils + provided + javax.servlet.jsp javax.servlet.jsp-api diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/java/org/apache/nifi/web/docs/DocumentationController.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/java/org/apache/nifi/web/docs/DocumentationController.java index a5428cdba0..7d8ec6ca23 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/java/org/apache/nifi/web/docs/DocumentationController.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/java/org/apache/nifi/web/docs/DocumentationController.java @@ -39,6 +39,9 @@ import org.apache.commons.lang3.StringUtils; @WebServlet(name = "DocumenationController", urlPatterns = {"/*"}) public class DocumentationController extends HttpServlet { + private static final int GENERAL_LINK_COUNT = 4; + private static final int DEVELOPER_LINK_COUNT = 2; + // context for accessing the extension mapping private ServletContext servletContext; @@ -82,7 +85,7 @@ public class DocumentationController extends HttpServlet { request.setAttribute("processors", processors); request.setAttribute("controllerServices", controllerServices); request.setAttribute("reportingTasks", reportingTasks); - request.setAttribute("totalComponents", processors.size() + controllerServices.size() + reportingTasks.size()); + request.setAttribute("totalComponents", GENERAL_LINK_COUNT + processors.size() + controllerServices.size() + reportingTasks.size() + DEVELOPER_LINK_COUNT); // forward appropriately request.getRequestDispatcher("/WEB-INF/jsp/documentation.jsp").forward(request, response); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/WEB-INF/jsp/documentation.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/WEB-INF/jsp/documentation.jsp index d2397de73d..9690def89d 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/WEB-INF/jsp/documentation.jsp +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/WEB-INF/jsp/documentation.jsp @@ -30,7 +30,7 @@ - +
NiFi Documentation
@@ -55,11 +55,11 @@