mirror of https://github.com/apache/nifi.git
Merge remote-tracking branch 'upstream/develop' into nifi-solr-bundle
This commit is contained in:
commit
bd6159e97f
|
@ -24,15 +24,24 @@ import java.lang.annotation.RetentionPolicy;
|
|||
import java.lang.annotation.Target;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Marker annotation a {@link org.apache.nifi.processor.Processor Processor},
|
||||
* {@link org.apache.nifi.controller.ControllerService ControllerService}, or
|
||||
* {@link org.apache.nifi.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.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* Methods with this annotation are called without any arguments, as all settings
|
||||
* and properties can be assumed to be the defaults.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* If any method annotated with this annotation throws a Throwable, the component
|
||||
* will not be added to the flow.
|
||||
* </p>
|
||||
*
|
||||
* @author none
|
||||
*/
|
||||
|
|
|
@ -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.
|
||||
*
|
||||
* <p>
|
||||
* 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.
|
||||
* Marker annotation a {@link org.apache.nifi.controller.ControllerService ControllerService}
|
||||
* can use to indicate a method should be called whenever the service is disabled.
|
||||
*</p>
|
||||
*
|
||||
* <p>
|
||||
* 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.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* 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.
|
||||
* </p>
|
||||
*
|
||||
* @author none
|
||||
*/
|
||||
@Documented
|
||||
@Target({ElementType.METHOD})
|
||||
|
|
|
@ -25,35 +25,35 @@ import java.lang.annotation.Target;
|
|||
|
||||
/**
|
||||
* <p>
|
||||
* Marker annotation a {@link org.apache.nifi.processor.Processor Processor},
|
||||
* {@link org.apache.nifi.controller.ControllerService ControllerService} or
|
||||
* {@link org.apache.nifi.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.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* Methods using this annotation must take either 0 arguments or a single argument.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* 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}.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* 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}.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* 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.
|
||||
* </p>
|
||||
*
|
||||
* @author none
|
||||
* <p>
|
||||
* 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.
|
||||
* </p>
|
||||
*
|
||||
*
|
||||
*/
|
||||
@Documented
|
||||
@Target({ElementType.METHOD})
|
||||
|
|
|
@ -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;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* 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,6 +36,14 @@ 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.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* Methods with this annotation are permitted to take no arguments or to take a single
|
||||
* argument. If using a single argument, that argument must be of type {@link ConfigurationContext}
|
||||
* if the component is a ReportingTask or a ControllerService. If the component is a Processor,
|
||||
* then the argument must be of type {@link ProcessContext}.
|
||||
* </p>
|
||||
*
|
||||
* @author none
|
||||
*/
|
||||
|
|
|
@ -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;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* 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,6 +35,13 @@ 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.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* 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.
|
||||
* </p>
|
||||
*
|
||||
* @author none
|
||||
*/
|
||||
|
|
|
@ -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;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Marker annotation a {@link org.apache.nifi.processor.Processor Processor} or
|
||||
|
@ -48,6 +51,12 @@ import java.lang.annotation.Target;
|
|||
* <code>onTrigger</code> method), see the {@link OnUnscheduled} annotation.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* 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.
|
||||
* </p>
|
||||
*
|
||||
* @author none
|
||||
*/
|
||||
@Documented
|
||||
|
|
|
@ -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}.
|
||||
* </p>
|
||||
*
|
||||
* @author none
|
||||
*/
|
||||
@Documented
|
||||
@Target({ElementType.METHOD})
|
||||
|
|
|
@ -142,10 +142,20 @@ public final class PropertyDescriptor implements Comparable<PropertyDescriptor>
|
|||
final Set<String> 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)
|
||||
.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")
|
||||
.build();
|
||||
|
|
|
@ -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 <code>false</code>
|
||||
* 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 <code>true</code> if the given value contains a NiFi Expression Language expression,
|
||||
* <code>false</code> if it does not
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -41,6 +41,18 @@ public interface ControllerServiceLookup {
|
|||
*/
|
||||
boolean isControllerServiceEnabled(String serviceIdentifier);
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> if the Controller Service with the given
|
||||
* identifier has been enabled but is still in the transitioning state,
|
||||
* otherwise returns <code>false</code>.
|
||||
* If the given identifier is not known by this ControllerServiceLookup,
|
||||
* returns <code>false</code>.
|
||||
*
|
||||
* @param serviceIdentifier
|
||||
* @return
|
||||
*/
|
||||
boolean isControllerServiceEnabling(String serviceIdentifier);
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> if the given Controller Service is enabled,
|
||||
* <code>false</code> otherwise. If the given Controller Service is not
|
||||
|
@ -63,4 +75,11 @@ public interface ControllerServiceLookup {
|
|||
*/
|
||||
Set<String> getControllerServiceIdentifiers(Class<? extends ControllerService> 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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* 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:
|
||||
* </p>
|
||||
*
|
||||
* <ul>
|
||||
* <li>
|
||||
* The <code>toString()</code> 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.
|
||||
* </li>
|
||||
* <li>
|
||||
* If the last value in an Object[] argument that is passed to the logger is a Throwable, then the logged message
|
||||
* will include a <code>toString()</code> 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.
|
||||
* </li>
|
||||
* <li>
|
||||
* 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.
|
||||
* </li>
|
||||
* </ul>
|
||||
*
|
||||
*/
|
||||
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);
|
||||
}
|
|
@ -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 {
|
||||
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
||||
|
|
|
@ -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<String, String> properties;
|
||||
private final Collection<String> 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<String, String> getProperties() {
|
||||
return properties;
|
||||
}
|
||||
|
||||
/**
|
||||
* Current validation errors for the component.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public Collection<String> 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<String, String> properties;
|
||||
private Collection<String> 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<String, String> properties) {
|
||||
this.properties = properties;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder validateErrors(final Collection<String> validationErrors) {
|
||||
this.validationErrors = validationErrors;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ComponentDetails build() {
|
||||
return new ComponentDetails(this);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<ConfigurationAction> 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;
|
||||
}
|
|
@ -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();
|
||||
|
||||
}
|
|
@ -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 {
|
||||
|
||||
/**
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.nifi.web;
|
|||
/**
|
||||
* Context configuration for methods invoked from the NiFiWebContext.
|
||||
*/
|
||||
@Deprecated
|
||||
public interface NiFiWebContextConfig {
|
||||
|
||||
/**
|
||||
|
|
|
@ -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:
|
||||
*
|
||||
* <code>
|
||||
* <CN=original-proxied-entity><CN=first-proxy><CN=second-proxy>...
|
||||
* </code>
|
||||
*
|
||||
* @return the proxied entities chain or null if no chain
|
||||
*/
|
||||
String getProxiedEntitiesChain();
|
||||
|
||||
}
|
|
@ -19,6 +19,7 @@ package org.apache.nifi.web;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
@Deprecated
|
||||
public class ProcessorConfigurationAction {
|
||||
|
||||
private final String processorId;
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.Map;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
@Deprecated
|
||||
public class ProcessorInfo {
|
||||
|
||||
private final String id;
|
||||
|
|
|
@ -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) {
|
||||
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -192,8 +192,6 @@
|
|||
|
||||
<nifi.flow.configuration.file>./conf/flow.xml.gz</nifi.flow.configuration.file>
|
||||
<nifi.flow.configuration.archive.dir>./conf/archive/</nifi.flow.configuration.archive.dir>
|
||||
<nifi.reporting.task.configuration.file>./conf/reporting-tasks.xml</nifi.reporting.task.configuration.file>
|
||||
<nifi.controller.service.configuration.file>./conf/controller-services.xml</nifi.controller.service.configuration.file>
|
||||
<nifi.authority.provider.configuration.file>./conf/authority-providers.xml</nifi.authority.provider.configuration.file>
|
||||
<nifi.templates.directory>./conf/templates</nifi.templates.directory>
|
||||
<nifi.database.directory>./database_repository</nifi.database.directory>
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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";
|
||||
|
|
|
@ -700,7 +700,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));
|
||||
|
|
|
@ -39,7 +39,7 @@ public class TestEndpointConnectionStatePool {
|
|||
collection.add(new NodeInformation("ShouldGetMedium", 5, 5555, true, 4096));
|
||||
|
||||
clusterNodeInfo.setNodeInformation(collection);
|
||||
final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
|
||||
final List<PeerStatus> 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<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND);
|
||||
final List<PeerStatus> destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.RECEIVE);
|
||||
for ( final PeerStatus peerStatus : destinations ) {
|
||||
System.out.println(peerStatus.getPeerDescription());
|
||||
}
|
||||
|
|
|
@ -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 <<system_properties>> 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 <<system_properties>> 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"):
|
||||
|
||||
----
|
||||
<logger name="org.apache.nifi.web.api.config" level="INFO"
|
||||
additivity="false">
|
||||
<appender-ref ref="USER_FILE"/>
|
||||
</logger>
|
||||
----
|
||||
|
||||
|
||||
|
||||
[[system_properties]]
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -34,8 +41,18 @@ public class MockControllerServiceInitializationContext extends MockControllerSe
|
|||
return identifier;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getControllerServiceName(final String serviceIdentifier) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerServiceLookup getControllerServiceLookup() {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ComponentLog getLogger() {
|
||||
return logger;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<String> getControllerServiceIdentifiers(final Class<? extends ControllerService> serviceType) {
|
||||
final Set<String> 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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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<PropertyDescriptor, String> 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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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<String, String> 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<PropertyDescriptor, String> resolvedProps = new HashMap<>();
|
||||
|
|
|
@ -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<String, List<PreviousValue>> getPreviousValues(String processorId);
|
||||
Map<String, List<PreviousValue>> getPreviousValues(String componentId);
|
||||
|
||||
/**
|
||||
* Finds the specified action.
|
||||
|
|
|
@ -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<String, String> columnMap;
|
||||
private final Connection connection;
|
||||
private final Map<String, String> 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<String, List<PreviousValue>> getPreviousValues(String processorId) {
|
||||
public Map<String, List<PreviousValue>> getPreviousValues(String componentId) {
|
||||
Map<String, List<PreviousValue>> 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<PreviousValue> getPreviousValuesForProperty(final String processorId, final String property) {
|
||||
private List<PreviousValue> getPreviousValuesForProperty(final String componentId, final String property) {
|
||||
List<PreviousValue> 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
|
||||
|
|
|
@ -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<String, List<PreviousValue>> getPreviousValues(String processorId);
|
||||
Map<String, List<PreviousValue>> getPreviousValues(String componentId);
|
||||
|
||||
/**
|
||||
* Get the actions within the given date range.
|
||||
|
|
|
@ -28,16 +28,16 @@ import org.apache.nifi.history.PreviousValue;
|
|||
*/
|
||||
public class GetPreviousValues implements AdministrationAction<Map<String, List<PreviousValue>>> {
|
||||
|
||||
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<String, List<PreviousValue>> execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) {
|
||||
ActionDAO actionDao = daoFactory.getActionDAO();
|
||||
return actionDao.getPreviousValues(processorId);
|
||||
return actionDao.getPreviousValues(componentId);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -81,7 +81,7 @@ public class StandardAuditService implements AuditService {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Map<String, List<PreviousValue>> getPreviousValues(String processorId) {
|
||||
public Map<String, List<PreviousValue>> getPreviousValues(String componentId) {
|
||||
Transaction transaction = null;
|
||||
Map<String, List<PreviousValue>> 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
|
||||
|
|
|
@ -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<String, PropertyHistoryDTO> 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
|
||||
*/
|
|
@ -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.
|
||||
*
|
||||
|
|
|
@ -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<String, String> properties;
|
||||
private Map<String, PropertyDescriptorDTO> descriptors;
|
||||
|
||||
private String customUiUrl;
|
||||
private String annotationData;
|
||||
|
||||
private Set<ControllerServiceReferencingComponentDTO> referencingComponents;
|
||||
|
||||
private Collection<String> 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<String, String> getProperties() {
|
||||
return properties;
|
||||
}
|
||||
|
||||
public void setProperties(Map<String, String> properties) {
|
||||
this.properties = properties;
|
||||
}
|
||||
|
||||
/**
|
||||
* The descriptors for the controller service properties.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public Map<String, PropertyDescriptorDTO> getDescriptors() {
|
||||
return descriptors;
|
||||
}
|
||||
|
||||
public void setDescriptors(Map<String, PropertyDescriptorDTO> 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<ControllerServiceReferencingComponentDTO> getReferencingComponents() {
|
||||
return referencingComponents;
|
||||
}
|
||||
|
||||
public void setReferencingComponents(Set<ControllerServiceReferencingComponentDTO> 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<String> getValidationErrors() {
|
||||
return validationErrors;
|
||||
}
|
||||
|
||||
public void setValidationErrors(Collection<String> validationErrors) {
|
||||
this.validationErrors = validationErrors;
|
||||
}
|
||||
}
|
|
@ -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<String, String> properties;
|
||||
private Map<String, PropertyDescriptorDTO> descriptors;
|
||||
|
||||
private Collection<String> validationErrors;
|
||||
|
||||
private String referenceType;
|
||||
private Integer activeThreadCount;
|
||||
|
||||
private Boolean referenceCycle;
|
||||
private Set<ControllerServiceReferencingComponentDTO> 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<String, String> getProperties() {
|
||||
return properties;
|
||||
}
|
||||
|
||||
public void setProperties(Map<String, String> properties) {
|
||||
this.properties = properties;
|
||||
}
|
||||
|
||||
/**
|
||||
* The descriptors for the components properties.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public Map<String, PropertyDescriptorDTO> getDescriptors() {
|
||||
return descriptors;
|
||||
}
|
||||
|
||||
public void setDescriptors(Map<String, PropertyDescriptorDTO> descriptors) {
|
||||
this.descriptors = descriptors;
|
||||
}
|
||||
|
||||
/**
|
||||
* Any validation error associated with this component.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public Collection<String> getValidationErrors() {
|
||||
return validationErrors;
|
||||
}
|
||||
|
||||
public void setValidationErrors(Collection<String> 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<ControllerServiceReferencingComponentDTO> getReferencingComponents() {
|
||||
return referencingComponents;
|
||||
}
|
||||
|
||||
public void setReferencingComponents(Set<ControllerServiceReferencingComponentDTO> 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;
|
||||
}
|
||||
|
||||
}
|
|
@ -20,13 +20,13 @@ 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 {
|
||||
|
||||
private String type;
|
||||
private Set<DocumentedTypeDTO> childTypes;
|
||||
private String description;
|
||||
private Set<String> tags;
|
||||
|
||||
|
@ -57,7 +57,7 @@ public class DocumentedTypeDTO {
|
|||
}
|
||||
|
||||
/**
|
||||
* The tags associated with this type
|
||||
* The tags associated with this type.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
|
@ -68,4 +68,18 @@ public class DocumentedTypeDTO {
|
|||
public void setTags(final Set<String> tags) {
|
||||
this.tags = tags;
|
||||
}
|
||||
|
||||
/**
|
||||
* Child types for this type.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public Set<DocumentedTypeDTO> getChildTypes() {
|
||||
return childTypes;
|
||||
}
|
||||
|
||||
public void setChildTypes(Set<DocumentedTypeDTO> childTypes) {
|
||||
this.childTypes = childTypes;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -34,6 +34,7 @@ public class FlowSnippetDTO {
|
|||
private Set<ConnectionDTO> connections = new LinkedHashSet<>();
|
||||
private Set<LabelDTO> labels = new LinkedHashSet<>();
|
||||
private Set<FunnelDTO> funnels = new LinkedHashSet<>();
|
||||
private Set<ControllerServiceDTO> controllerServices = new LinkedHashSet<>();
|
||||
|
||||
/**
|
||||
* The connections in this flow snippet.
|
||||
|
@ -138,4 +139,16 @@ public class FlowSnippetDTO {
|
|||
public void setRemoteProcessGroups(Set<RemoteProcessGroupDTO> remoteProcessGroups) {
|
||||
this.remoteProcessGroups = remoteProcessGroups;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the Controller Services in this flow snippet
|
||||
* @return
|
||||
*/
|
||||
public Set<ControllerServiceDTO> getControllerServices() {
|
||||
return controllerServices;
|
||||
}
|
||||
|
||||
public void setControllerServices(Set<ControllerServiceDTO> controllerServices) {
|
||||
this.controllerServices = controllerServices;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
*/
|
||||
|
|
|
@ -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 <code>null</code>
|
||||
* 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<AllowableValueDTO> 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<AllowableValueDTO> getAllowableValues() {
|
||||
return allowableValues;
|
||||
}
|
||||
|
||||
public void setAllowableValues(Set<AllowableValueDTO> 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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,243 @@
|
|||
/*
|
||||
* 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<AllowableValueDTO> allowableValues;
|
||||
private boolean required;
|
||||
private boolean sensitive;
|
||||
private boolean dynamic;
|
||||
private boolean supportsEl;
|
||||
private boolean identifiesControllerService;
|
||||
|
||||
/**
|
||||
* The set of allowable values for this property. If empty then the
|
||||
* allowable values are not constrained.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public List<AllowableValueDTO> getAllowableValues() {
|
||||
return allowableValues;
|
||||
}
|
||||
|
||||
public void setAllowableValues(List<AllowableValueDTO> 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;
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether this descriptor represents a controller service.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public boolean isIdentifiesControllerService() {
|
||||
return identifiesControllerService;
|
||||
}
|
||||
|
||||
public void setIdentifiesControllerService(boolean 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 <code>null</code>
|
||||
* 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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<String, String> defaultSchedulingPeriod;
|
||||
|
||||
private Map<String, String> properties;
|
||||
private Map<String, PropertyDescriptorDTO> descriptors;
|
||||
|
||||
private String customUiUrl;
|
||||
private String annotationData;
|
||||
|
||||
private Collection<String> 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<String, String> getProperties() {
|
||||
return properties;
|
||||
}
|
||||
|
||||
public void setProperties(Map<String, String> properties) {
|
||||
this.properties = properties;
|
||||
}
|
||||
|
||||
/**
|
||||
* Map of property name to descriptor
|
||||
* @return
|
||||
*/
|
||||
public Map<String, PropertyDescriptorDTO> getDescriptors() {
|
||||
return descriptors;
|
||||
}
|
||||
|
||||
public void setDescriptors(Map<String, PropertyDescriptorDTO> 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<String> getValidationErrors() {
|
||||
return validationErrors;
|
||||
}
|
||||
|
||||
public void setValidationErrors(Collection<String> validationErrors) {
|
||||
this.validationErrors = validationErrors;
|
||||
}
|
||||
|
||||
/**
|
||||
* The default scheduling period for the different scheduling strategies.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public Map<String, String> getDefaultSchedulingPeriod() {
|
||||
return defaultSchedulingPeriod;
|
||||
}
|
||||
|
||||
public void setDefaultSchedulingPeriod(Map<String, String> 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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -24,7 +24,7 @@ import javax.xml.bind.annotation.XmlType;
|
|||
*/
|
||||
@XmlType(name = "componentDetails")
|
||||
@XmlSeeAlso({
|
||||
ProcessorDetailsDTO.class,
|
||||
ExtensionDetailsDTO.class,
|
||||
RemoteProcessGroupDetailsDTO.class
|
||||
})
|
||||
public class ComponentDetailsDTO {
|
||||
|
|
|
@ -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
|
||||
*/
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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<ControllerServiceReferencingComponentDTO> controllerServiceReferencingComponents;
|
||||
|
||||
/**
|
||||
* The list of controller service referencing components that are being serialized.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public Set<ControllerServiceReferencingComponentDTO> getControllerServiceReferencingComponents() {
|
||||
return controllerServiceReferencingComponents;
|
||||
}
|
||||
|
||||
public void setControllerServiceReferencingComponents(Set<ControllerServiceReferencingComponentDTO> controllerServiceReferencingComponents) {
|
||||
this.controllerServiceReferencingComponents = controllerServiceReferencingComponents;
|
||||
}
|
||||
|
||||
}
|
|
@ -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<DocumentedTypeDTO> controllerServiceTypes;
|
||||
|
||||
/**
|
||||
* The list of controller service types that are being serialized.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public Set<DocumentedTypeDTO> getControllerServiceTypes() {
|
||||
return controllerServiceTypes;
|
||||
}
|
||||
|
||||
public void setControllerServiceTypes(Set<DocumentedTypeDTO> controllerServiceTypes) {
|
||||
this.controllerServiceTypes = controllerServiceTypes;
|
||||
}
|
||||
|
||||
}
|
|
@ -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<ControllerServiceDTO> controllerServices;
|
||||
|
||||
/**
|
||||
* The list of controller services that are being serialized.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public Set<ControllerServiceDTO> getControllerServices() {
|
||||
return controllerServices;
|
||||
}
|
||||
|
||||
public void setControllerServices(Set<ControllerServiceDTO> controllerServices) {
|
||||
this.controllerServices = controllerServices;
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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<DocumentedTypeDTO> reportingTaskTypes;
|
||||
|
||||
/**
|
||||
* The list of reporting task types that are being serialized.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public Set<DocumentedTypeDTO> getReportingTaskTypes() {
|
||||
return reportingTaskTypes;
|
||||
}
|
||||
|
||||
public void setReportingTaskTypes(Set<DocumentedTypeDTO> reportingTaskTypes) {
|
||||
this.reportingTaskTypes = reportingTaskTypes;
|
||||
}
|
||||
|
||||
}
|
|
@ -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<ReportingTaskDTO> reportingTasks;
|
||||
|
||||
/**
|
||||
* The list of reporting tasks that are being serialized.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public Set<ReportingTaskDTO> getReportingTasks() {
|
||||
return reportingTasks;
|
||||
}
|
||||
|
||||
public void setReportingTasks(Set<ReportingTaskDTO> reportingTasks) {
|
||||
this.reportingTasks = reportingTasks;
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,10 +32,6 @@
|
|||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-properties</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-web-optimistic-locking</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-administration</artifactId>
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -45,6 +45,10 @@
|
|||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-client-dto</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-web-optimistic-locking</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-framework-core</artifactId>
|
||||
|
|
|
@ -27,12 +27,23 @@ 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() {
|
||||
|
|
|
@ -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.
|
||||
*
|
||||
|
|
|
@ -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,6 +475,8 @@ 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))) {
|
||||
|
@ -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))) {
|
||||
|
||||
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);
|
||||
|
|
|
@ -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,18 +153,75 @@ 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() {
|
||||
resourceLock.lock();
|
||||
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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;
|
||||
|
@ -51,6 +50,8 @@ public class WebClusterManagerFactoryBean implements FactoryBean, ApplicationCon
|
|||
|
||||
private StringEncryptor encryptor;
|
||||
|
||||
private OptimisticLockingManager optimisticLockingManager;
|
||||
|
||||
@Override
|
||||
public Object getObject() throws Exception {
|
||||
if (properties.isClusterManager() && properties.isNode()) {
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -91,10 +91,14 @@
|
|||
<property name="properties" ref="nifiProperties"/>
|
||||
</bean>
|
||||
|
||||
<!-- cluster manager optimistic locking manager -->
|
||||
<bean id="clusterManagerOptimisticLockingManager" class="org.apache.nifi.web.StandardOptimisticLockingManager"/>
|
||||
|
||||
<!-- cluster manager -->
|
||||
<bean id="clusterManager" class="org.apache.nifi.cluster.spring.WebClusterManagerFactoryBean">
|
||||
<property name="properties" ref="nifiProperties"/>
|
||||
<property name="encryptor" ref="stringEncryptor"/>
|
||||
<property name="optimisticLockingManager" ref="clusterManagerOptimisticLockingManager"/>
|
||||
</bean>
|
||||
|
||||
<!-- discoverable services -->
|
||||
|
|
|
@ -1 +1,2 @@
|
|||
/target
|
||||
/target/
|
||||
|
|
|
@ -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<ValidationResult> getValidationErrors() {
|
||||
return getValidationErrors(Collections.<String>emptySet());
|
||||
}
|
||||
|
||||
public Collection<ValidationResult> getValidationErrors(final Set<String> serviceIdentifiersNotToValidate) {
|
||||
final List<ValidationResult> results = new ArrayList<>();
|
||||
lock.lock();
|
||||
try {
|
||||
final ValidationContext validationContext = validationContextFactory.newValidationContext(getProperties(), getAnnotationData());
|
||||
final ValidationContext validationContext = validationContextFactory.newValidationContext(serviceIdentifiersNotToValidate, getProperties(), getAnnotationData());
|
||||
|
||||
final Collection<ValidationResult> validationResults;
|
||||
try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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<String, String> 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<ControllerServiceNode> ignoredReferences);
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
@ -54,6 +52,12 @@ public interface ReportingTaskNode extends ConfiguredComponent {
|
|||
|
||||
boolean isRunning();
|
||||
|
||||
/**
|
||||
* Returns the number of threads (concurrent tasks) currently being used by this ReportingTask
|
||||
* @return
|
||||
*/
|
||||
int getActiveThreadCount();
|
||||
|
||||
/**
|
||||
* Indicates the {@link ScheduledState} of this <code>ReportingTask</code>. A
|
||||
* value of stopped does NOT indicate that the <code>ReportingTask</code> has
|
||||
|
@ -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<ControllerServiceNode> ignoredReferences);
|
||||
|
||||
void verifyCanStart();
|
||||
void verifyCanStop();
|
||||
void verifyCanDisable();
|
||||
|
|
|
@ -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<PropertyDescriptor, String> properties, String annotationData);
|
||||
|
||||
ValidationContext newValidationContext(Set<String> serviceIdentifiersToNotValidate, Map<PropertyDescriptor, String> properties, String annotationData);
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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 <code>null</code> 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<ReportingTaskNode> 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);
|
||||
}
|
|
@ -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,13 +27,8 @@ 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();
|
||||
|
||||
|
@ -40,8 +36,33 @@ public interface ControllerServiceNode extends ConfiguredComponent {
|
|||
|
||||
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<ControllerServiceNode> 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<ControllerServiceNode> ignoredReferences);
|
||||
|
||||
void verifyCanDelete();
|
||||
void verifyCanUpdate();
|
||||
}
|
||||
|
|
|
@ -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 <code>firstTimeadded</code>
|
||||
* Creates a new Controller Service of the specified type and assigns it the given id. If <code>firstTimeadded</code>
|
||||
* 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<ControllerServiceNode> 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<ControllerServiceNode> 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);
|
||||
}
|
||||
|
|
|
@ -41,10 +41,11 @@ public interface ControllerServiceReference {
|
|||
Set<ConfiguredComponent> 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<ConfiguredComponent> getRunningReferences();
|
||||
Set<ConfiguredComponent> getActiveReferences();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1064,6 +1076,22 @@ 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<String, String> 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<String> processorClasses = new ArrayList<>();
|
||||
final Set<String> processorClasses = new HashSet<>();
|
||||
for (final Class<?> c : ExtensionManager.getExtensions(Processor.class)) {
|
||||
processorClasses.add(c.getName());
|
||||
}
|
||||
final List<String> prioritizerClasses = new ArrayList<>();
|
||||
final Set<String> prioritizerClasses = new HashSet<>();
|
||||
for (final Class<?> c : ExtensionManager.getExtensions(FlowFilePrioritizer.class)) {
|
||||
prioritizerClasses.add(c.getName());
|
||||
}
|
||||
final Set<String> controllerServiceClasses = new HashSet<>();
|
||||
for (final Class<?> c : ExtensionManager.getExtensions(ControllerService.class)) {
|
||||
controllerServiceClasses.add(c.getName());
|
||||
}
|
||||
|
||||
final Set<ProcessorDTO> allProcs = new HashSet<>();
|
||||
final Set<ConnectionDTO> allConns = new HashSet<>();
|
||||
|
@ -1761,6 +1817,15 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H
|
|||
}
|
||||
}
|
||||
|
||||
final Set<ControllerServiceDTO> 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<String> prioritizers = conn.getPrioritizers();
|
||||
if (prioritizers != null) {
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
@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,32 +2647,74 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H
|
|||
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, reportingTaskNode.getReportingTask(), reportingTaskNode.getConfigurationContext());
|
||||
}
|
||||
|
||||
for ( final Map.Entry<PropertyDescriptor, String> 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<ReportingTaskNode> getReportingTasks() {
|
||||
return reportingTasks.values();
|
||||
@Override
|
||||
public Set<ReportingTaskNode> 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<ControllerServiceNode> serviceNodes) {
|
||||
controllerServiceProvider.enableControllerServices(serviceNodes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void disableControllerService(final ControllerServiceNode serviceNode) {
|
||||
serviceNode.verifyCanDisable();
|
||||
|
@ -2598,8 +2722,23 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H
|
|||
}
|
||||
|
||||
@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
|
||||
|
@ -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<ControllerServiceNode> getAllControllerServices() {
|
||||
return controllerServiceProvider.getAllControllerServices();
|
||||
}
|
||||
|
||||
//
|
||||
// Counters
|
||||
//
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
@ -78,6 +80,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();
|
||||
final String groupId = getString(element, "id");
|
||||
|
@ -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<String, String> properties = new LinkedHashMap<>();
|
||||
final List<Element> 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<String> autoTerminatedRelationships = new HashSet<>();
|
||||
final List<Element> autoTerminateList = getChildrenByTagName(element, "autoTerminatedRelationship");
|
||||
|
@ -353,6 +382,17 @@ public class FlowFromDOMFactory {
|
|||
return dto;
|
||||
}
|
||||
|
||||
private static LinkedHashMap<String, String> getProperties(final Element element, final StringEncryptor encryptor) {
|
||||
final LinkedHashMap<String, String> properties = new LinkedHashMap<>();
|
||||
final List<Element> 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<Element> nodeList = getChildrenByTagName(element, childElementName);
|
||||
if (nodeList == null || nodeList.isEmpty()) {
|
||||
|
|
|
@ -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;
|
||||
|
@ -80,6 +81,18 @@ public class StandardFlowSerializer implements FlowSerializer {
|
|||
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<PropertyDescriptor, String> 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<PropertyDescriptor, String> properties, final String annotationData, final StringEncryptor encryptor) {
|
||||
final Document doc = element.getOwnerDocument();
|
||||
for (final Map.Entry<PropertyDescriptor, String> 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);
|
||||
|
|
|
@ -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();
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue