NIFI-3520 Refactoring instance class loading

- Fixing FlowController to use appropriate class loader when instantiating processor
- Updating ExtensionManager to leverage new flag in MANIFEST from NAR plugin
- Adding ReloadComponent interface and refactoring instance class loading to use it
- Fixing FetchHDFS issue with TDE by using ugi.doAs
- Refactoring nifi-nar-utils so that ExtensionManager only lives in nifi-framework
- Caching temp components found during service loader in ExtensionManager
- Updating authorizables, docs, and fingerprinting to use the cached components
- Introducing a flag on @RequiresInstanceClassLoading to indicate if ancestor resources should be cloned
- Updating developer guide regarding cloneAncestorResources flag
- This closes #1635
This commit is contained in:
Bryan Bende 2017-03-24 17:14:24 -04:00 committed by Matt Gilman
parent 8f37ad4512
commit 556f309df0
No known key found for this signature in database
GPG Key ID: DF61EC19432AEE37
98 changed files with 1411 additions and 813 deletions

View File

@ -28,10 +28,19 @@ import java.lang.annotation.Target;
* for each instance of the component, copying all resources from the component's NARClassLoader to a * for each instance of the component, copying all resources from the component's NARClassLoader to a
* new ClassLoader which will only be used by a given instance of the component. * new ClassLoader which will only be used by a given instance of the component.
* *
* This annotation is typically used when a component has one or more PropertyDescriptors which set * If cloneAncestorResources is set to true, the instance ClassLoader will include ancestor resources up to the
* dynamicallyModifiesClasspath(boolean) to true. * first ClassLoader containing a controller service API referenced by the component, or up to the Jetty NAR.
* *
* When this annotation is used it is important to note that each added instance of the component will increase * Example #1 - PutHDFS has this flag set to true and does not reference any controller services, so it will include
* resources from nifi-hadoop-nar, nifi-hadoop-libraries-nar, and nifi-standard-services-api-nar, stopping at nifi-jetty-nar.
*
* Example #2 - If PutHDFS referenced an SSLContext and has this flag set to true, then it would include
* resources from nifi-hadoop-nar, nifi-hadoop-libraries-nar, and stop before nifi-standard-services-api-nar.
*
* Example #3 - HBaseClientService_1_1_2 does not have this flag set so it defaults to false, and therefore includes
* only resources from the nifi-hbase-client-service-1_1_2-nar.
*
* NOTE: When this annotation is used it is important to note that each added instance of the component will increase
* the overall memory footprint more than that of a component without this annotation. * the overall memory footprint more than that of a component without this annotation.
*/ */
@Documented @Documented
@ -39,4 +48,7 @@ import java.lang.annotation.Target;
@Retention(RetentionPolicy.RUNTIME) @Retention(RetentionPolicy.RUNTIME)
@Inherited @Inherited
public @interface RequiresInstanceClassLoading { public @interface RequiresInstanceClassLoading {
boolean cloneAncestorResources() default false;
} }

View File

@ -2357,6 +2357,11 @@ component's NAR ClassLoader. When `@RequiresInstanceClassLoading` is not present
instance ClassLoader simply has it's parent ClassLoader set to the NAR ClassLoader, rather than instance ClassLoader simply has it's parent ClassLoader set to the NAR ClassLoader, rather than
copying resources. copying resources.
The `@RequiresInstanceClassLoading` annotation also provides an optional flag `cloneAncestorResources'. If
set to true, the instance ClassLoader will include ancestor resources up to the first ClassLoader containing a
controller service API referenced by the component, or up to the Jetty NAR. If set to false, or not specified,
only the resources from the component's NAR will be included.
Because @RequiresInstanceClassLoading copies resources from the NAR ClassLoader for each instance of the Because @RequiresInstanceClassLoading copies resources from the NAR ClassLoader for each instance of the
component, use this capability judiciously. If ten instances of one component are created, all classes component, use this capability judiciously. If ten instances of one component are created, all classes
from the component's NAR ClassLoader are loaded into memory ten times. This could eventually increase the from the component's NAR ClassLoader are loaded into memory ten times. This could eventually increase the

View File

@ -38,6 +38,8 @@ public class BundleDetailsTest {
final String buildJdk = "JDK8"; final String buildJdk = "JDK8";
final String builtBy = "bbende"; final String builtBy = "bbende";
final boolean cloneDuringInstanceClassLoading = true;
final BundleDetails bundleDetails = new BundleDetails.Builder() final BundleDetails bundleDetails = new BundleDetails.Builder()
.workingDir(workingDirectory) .workingDir(workingDirectory)
.coordinate(coordinate) .coordinate(coordinate)

View File

@ -18,10 +18,6 @@
</parent> </parent>
<artifactId>nifi-documentation</artifactId> <artifactId>nifi-documentation</artifactId>
<dependencies> <dependencies>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-nar-utils</artifactId>
</dependency>
<dependency> <dependency>
<groupId>org.apache.nifi</groupId> <groupId>org.apache.nifi</groupId>
<artifactId>nifi-api</artifactId> <artifactId>nifi-api</artifactId>
@ -34,6 +30,14 @@
<groupId>org.apache.nifi</groupId> <groupId>org.apache.nifi</groupId>
<artifactId>nifi-properties</artifactId> <artifactId>nifi-properties</artifactId>
</dependency> </dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-nar-utils</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-framework-nar-utils</artifactId>
</dependency>
<dependency> <dependency>
<groupId>org.apache.nifi</groupId> <groupId>org.apache.nifi</groupId>
<artifactId>nifi-utils</artifactId> <artifactId>nifi-utils</artifactId>

View File

@ -22,8 +22,6 @@ import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.documentation.html.HtmlDocumentationWriter; import org.apache.nifi.documentation.html.HtmlDocumentationWriter;
import org.apache.nifi.documentation.html.HtmlProcessorDocumentationWriter; import org.apache.nifi.documentation.html.HtmlProcessorDocumentationWriter;
import org.apache.nifi.init.ConfigurableComponentInitializer;
import org.apache.nifi.init.ConfigurableComponentInitializerFactory;
import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.ExtensionMapping; import org.apache.nifi.nar.ExtensionMapping;
import org.apache.nifi.processor.Processor; import org.apache.nifi.processor.Processor;
@ -93,7 +91,7 @@ public class DocGenerator {
final Class<? extends ConfigurableComponent> componentClass = extensionClass.asSubclass(ConfigurableComponent.class); final Class<? extends ConfigurableComponent> componentClass = extensionClass.asSubclass(ConfigurableComponent.class);
try { try {
logger.debug("Documenting: " + componentClass); logger.debug("Documenting: " + componentClass);
document(componentDirectory, componentClass); document(componentDirectory, componentClass, coordinate);
} catch (Exception e) { } catch (Exception e) {
logger.warn("Unable to document: " + componentClass, e); logger.warn("Unable to document: " + componentClass, e);
} }
@ -113,12 +111,12 @@ public class DocGenerator {
* @throws IOException ioe * @throws IOException ioe
* @throws InitializationException ie * @throws InitializationException ie
*/ */
private static void document(final File componentDocsDir, final Class<? extends ConfigurableComponent> componentClass) private static void document(final File componentDocsDir, final Class<? extends ConfigurableComponent> componentClass, final BundleCoordinate bundleCoordinate)
throws InstantiationException, IllegalAccessException, IOException, InitializationException { throws InstantiationException, IllegalAccessException, IOException, InitializationException {
final ConfigurableComponent component = componentClass.newInstance(); // use temp components from ExtensionManager which should always be populated before doc generation
final ConfigurableComponentInitializer initializer = ConfigurableComponentInitializerFactory.createComponentInitializer(componentClass); final String classType = componentClass.getCanonicalName();
initializer.initialize(component); final ConfigurableComponent component = ExtensionManager.getTempComponent(classType, bundleCoordinate);
final DocumentationWriter writer = getDocumentWriter(componentClass); final DocumentationWriter writer = getDocumentWriter(componentClass);
@ -130,8 +128,6 @@ public class DocGenerator {
try (final OutputStream output = new BufferedOutputStream(new FileOutputStream(baseDocumentationFile))) { try (final OutputStream output = new BufferedOutputStream(new FileOutputStream(baseDocumentationFile))) {
writer.write(component, output, hasAdditionalInfo(componentDocsDir)); writer.write(component, output, hasAdditionalInfo(componentDocsDir));
} }
initializer.teardown(component);
} }
/** /**

View File

@ -23,6 +23,7 @@ import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.ExtensionMapping; import org.apache.nifi.nar.ExtensionMapping;
import org.apache.nifi.nar.NarClassLoaders; import org.apache.nifi.nar.NarClassLoaders;
import org.apache.nifi.nar.NarUnpacker; import org.apache.nifi.nar.NarUnpacker;
import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.NiFiProperties;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -47,7 +48,7 @@ public class DocGeneratorTest {
NiFiProperties.COMPONENT_DOCS_DIRECTORY, NiFiProperties.COMPONENT_DOCS_DIRECTORY,
temporaryFolder.getRoot().getAbsolutePath()); temporaryFolder.getRoot().getAbsolutePath());
final Bundle systemBundle = ExtensionManager.createSystemBundle(properties); final Bundle systemBundle = SystemBundle.create(properties);
final ExtensionMapping mapping = NarUnpacker.unpackNars(properties, systemBundle); final ExtensionMapping mapping = NarUnpacker.unpackNars(properties, systemBundle);
NarClassLoaders.getInstance().init(properties.getFrameworkWorkingDirectory(), properties.getExtensionsWorkingDirectory()); NarClassLoaders.getInstance().init(properties.getFrameworkWorkingDirectory(), properties.getExtensionsWorkingDirectory());

View File

@ -64,6 +64,7 @@ import org.apache.nifi.events.EventReporter;
import org.apache.nifi.io.socket.ServerSocketConfiguration; import org.apache.nifi.io.socket.ServerSocketConfiguration;
import org.apache.nifi.io.socket.SocketConfiguration; import org.apache.nifi.io.socket.SocketConfiguration;
import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.registry.VariableRegistry; import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.reporting.BulletinRepository; import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.reporting.Severity; import org.apache.nifi.reporting.Severity;
@ -120,7 +121,7 @@ public class Node {
} }
}; };
final Bundle systemBundle = ExtensionManager.createSystemBundle(properties); final Bundle systemBundle = SystemBundle.create(properties);
ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet()); ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
revisionManager = Mockito.mock(RevisionManager.class); revisionManager = Mockito.mock(RevisionManager.class);

View File

@ -24,7 +24,7 @@
<dependencies> <dependencies>
<dependency> <dependency>
<groupId>org.apache.nifi</groupId> <groupId>org.apache.nifi</groupId>
<artifactId>nifi-nar-utils</artifactId> <artifactId>nifi-framework-nar-utils</artifactId>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.apache.nifi</groupId> <groupId>org.apache.nifi</groupId>

View File

@ -27,12 +27,10 @@ import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider; import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.InstanceClassLoader;
import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.nar.NarCloseable;
import org.apache.nifi.registry.VariableRegistry; import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.util.file.classloader.ClassLoaderUtils; import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
import java.net.MalformedURLException;
import java.net.URL; import java.net.URL;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
@ -60,6 +58,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
private final String componentType; private final String componentType;
private final String componentCanonicalClass; private final String componentCanonicalClass;
private final VariableRegistry variableRegistry; private final VariableRegistry variableRegistry;
private final ReloadComponent reloadComponent;
private final AtomicBoolean isExtensionMissing; private final AtomicBoolean isExtensionMissing;
@ -69,7 +68,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
public AbstractConfiguredComponent(final String id, public AbstractConfiguredComponent(final String id,
final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider, final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider,
final String componentType, final String componentCanonicalClass, final VariableRegistry variableRegistry, final String componentType, final String componentCanonicalClass, final VariableRegistry variableRegistry,
final boolean isExtensionMissing) { final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
this.id = id; this.id = id;
this.validationContextFactory = validationContextFactory; this.validationContextFactory = validationContextFactory;
this.serviceProvider = serviceProvider; this.serviceProvider = serviceProvider;
@ -78,6 +77,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
this.componentCanonicalClass = componentCanonicalClass; this.componentCanonicalClass = componentCanonicalClass;
this.variableRegistry = variableRegistry; this.variableRegistry = variableRegistry;
this.isExtensionMissing = new AtomicBoolean(isExtensionMissing); this.isExtensionMissing = new AtomicBoolean(isExtensionMissing);
this.reloadComponent = reloadComponent;
} }
@Override @Override
@ -234,37 +234,28 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
} }
/** /**
* Adds all of the modules identified by the given module paths to the InstanceClassLoader for this component. * Triggers the reloading of the underlying component using a new InstanceClassLoader that includes the additional URL resources.
* *
* @param modulePaths a list of module paths where each entry can be a comma-separated list of multiple module paths * @param modulePaths a list of module paths where each entry can be a comma-separated list of multiple module paths
*/ */
private void processClasspathModifiers(final Set<String> modulePaths) { private void processClasspathModifiers(final Set<String> modulePaths) {
try { try {
// compute the URLs from all the modules paths
final URL[] urls = ClassLoaderUtils.getURLsForClasspath(modulePaths, null, true); final URL[] urls = ClassLoaderUtils.getURLsForClasspath(modulePaths, null, true);
if (getLogger().isDebugEnabled()) { // convert to a set of URLs
getLogger().debug("Adding {} resources to the classpath for {}", new Object[] {urls.length, name}); final Set<URL> additionalUrls = new LinkedHashSet<>();
for (URL url : urls) { if (urls != null) {
getLogger().debug(url.getFile()); for (final URL url : urls) {
additionalUrls.add(url);
} }
} }
final ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); // reload the underlying component with a new InstanceClassLoader that includes the new URLs
reload(additionalUrls);
if (!(classLoader instanceof InstanceClassLoader)) { } catch (Exception e) {
// Really shouldn't happen, but if we somehow got here and don't have an InstanceClassLoader then log a warning and move on getLogger().warn("Error processing classpath resources for " + id + ": " + e.getMessage(), e);
final String classLoaderName = classLoader == null ? "null" : classLoader.getClass().getName();
if (getLogger().isWarnEnabled()) {
getLogger().warn(String.format("Unable to modify the classpath for %s, expected InstanceClassLoader, but found %s", name, classLoaderName));
}
return;
}
final InstanceClassLoader instanceClassLoader = (InstanceClassLoader) classLoader;
instanceClassLoader.setInstanceResources(urls);
} catch (MalformedURLException e) {
// Shouldn't get here since we are suppressing errors
getLogger().warn("Error processing classpath resources", e);
} }
} }
@ -506,6 +497,10 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
return this.variableRegistry; return this.variableRegistry;
} }
protected ReloadComponent getReloadComponent() {
return this.reloadComponent;
}
@Override @Override
public void verifyCanUpdateBundle(final BundleCoordinate incomingCoordinate) throws IllegalArgumentException { public void verifyCanUpdateBundle(final BundleCoordinate incomingCoordinate) throws IllegalArgumentException {
final BundleCoordinate existingCoordinate = getBundleCoordinate(); final BundleCoordinate existingCoordinate = getBundleCoordinate();

View File

@ -30,8 +30,10 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.logging.ComponentLog;
import java.net.URL;
import java.util.Collection; import java.util.Collection;
import java.util.Map; import java.util.Map;
import java.util.Set;
public interface ConfiguredComponent extends ComponentAuthorizable { public interface ConfiguredComponent extends ComponentAuthorizable {
@ -53,6 +55,8 @@ public interface ConfiguredComponent extends ComponentAuthorizable {
boolean isValid(); boolean isValid();
void reload(Set<URL> additionalUrls) throws Exception;
BundleCoordinate getBundleCoordinate(); BundleCoordinate getBundleCoordinate();
ConfigurableComponent getComponent(); ConfigurableComponent getComponent();

View File

@ -47,8 +47,8 @@ public abstract class ProcessorNode extends AbstractConfiguredComponent implemen
public ProcessorNode(final String id, public ProcessorNode(final String id,
final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider, final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider,
final String componentType, final String componentCanonicalClass, final VariableRegistry variableRegistry, final String componentType, final String componentCanonicalClass, final VariableRegistry variableRegistry,
final boolean isExtensionMissing) { final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, isExtensionMissing); super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, isExtensionMissing);
this.scheduledState = new AtomicReference<>(ScheduledState.STOPPED); this.scheduledState = new AtomicReference<>(ScheduledState.STOPPED);
} }

View File

@ -0,0 +1,69 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.controller;
import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.controller.exception.ControllerServiceInstantiationException;
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
import org.apache.nifi.controller.service.ControllerServiceNode;
import java.net.URL;
import java.util.Set;
/**
* A service used to change the type of an underlying component.
*/
public interface ReloadComponent {
/**
* Changes the underlying Processor held by the node to an instance of the new type.
*
* @param existingNode the node being being updated
* @param newType the fully qualified class name of the new type
* @param bundleCoordinate the bundle coordinate of the new type
* @param additionalUrls additional URLs to be added to the instance class loader of the new component
* @throws ControllerServiceInstantiationException if unable to create an instance of the new type
*/
void reload(ProcessorNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls)
throws ProcessorInstantiationException;
/**
* Changes the underlying ControllerService held by the node to an instance of the new type.
*
* @param existingNode the node being being updated
* @param newType the fully qualified class name of the new type
* @param bundleCoordinate the bundle coordinate of the new type
* @param additionalUrls additional URLs to be added to the instance class loader of the new component
* @throws ControllerServiceInstantiationException if unable to create an instance of the new type
*/
void reload(ControllerServiceNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls)
throws ControllerServiceInstantiationException;
/**
* Changes the underlying ReportingTask held by the node to an instance of the new type.
*
* @param existingNode the ReportingTaskNode being updated
* @param newType the fully qualified class name of the new type
* @param bundleCoordinate the bundle coordinate of the new type
* @param additionalUrls additional URLs to be added to the instance class loader of the new component
* @throws ReportingTaskInstantiationException if unable to create an instance of the new type
*/
void reload(ReportingTaskNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls)
throws ReportingTaskInstantiationException;
}

View File

@ -112,14 +112,4 @@ public interface ReportingTaskProvider {
*/ */
void disableReportingTask(ReportingTaskNode reportingTask); void disableReportingTask(ReportingTaskNode reportingTask);
/**
* Changes the underlying ReportingTask held by the node to an instance of the new type.
*
* @param reportingTask the ReportingTaskNode being updated
* @param newType the fully qualified class name of the new type
* @param bundleCoordinate the bundle coordinate of the new type
* @throws ReportingTaskInstantiationException if unable to create an instance of the new type
*/
void changeReportingTaskType(final ReportingTaskNode reportingTask, final String newType, final BundleCoordinate bundleCoordinate) throws ReportingTaskInstantiationException;
} }

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.nifi.controller.service; package org.apache.nifi.controller.service;
import java.net.URL;
import java.util.Collection; import java.util.Collection;
import java.util.Set; import java.util.Set;
@ -38,10 +39,11 @@ public interface ControllerServiceProvider extends ControllerServiceLookup {
* @param type of service * @param type of service
* @param id of service * @param id of service
* @param bundleCoordinate the coordinate of the bundle for the service * @param bundleCoordinate the coordinate of the bundle for the service
* @param additionalUrls optional additional URL resources to add to the class loader of the component
* @param firstTimeAdded for service * @param firstTimeAdded for service
* @return the service node * @return the service node
*/ */
ControllerServiceNode createControllerService(String type, String id, BundleCoordinate bundleCoordinate, boolean firstTimeAdded); ControllerServiceNode createControllerService(String type, String id, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls, boolean firstTimeAdded);
/** /**
* @param id of the service * @param id of the service

View File

@ -39,6 +39,10 @@
<groupId>org.apache.nifi</groupId> <groupId>org.apache.nifi</groupId>
<artifactId>nifi-framework-api</artifactId> <artifactId>nifi-framework-api</artifactId>
</dependency> </dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-nar-utils</artifactId>
</dependency>
<dependency> <dependency>
<groupId>org.apache.nifi</groupId> <groupId>org.apache.nifi</groupId>
<artifactId>nifi-expression-language</artifactId> <artifactId>nifi-expression-language</artifactId>
@ -51,10 +55,6 @@
<groupId>org.apache.nifi</groupId> <groupId>org.apache.nifi</groupId>
<artifactId>nifi-schema-utils</artifactId> <artifactId>nifi-schema-utils</artifactId>
</dependency> </dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-nar-utils</artifactId>
</dependency>
<dependency> <dependency>
<groupId>org.apache.nifi</groupId> <groupId>org.apache.nifi</groupId>
<artifactId>nifi-properties</artifactId> <artifactId>nifi-properties</artifactId>

View File

@ -221,6 +221,7 @@ import java.io.ByteArrayInputStream;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.OutputStream; import java.io.OutputStream;
import java.net.URL;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
@ -247,7 +248,7 @@ import java.util.stream.Collectors;
import static java.util.Objects.requireNonNull; import static java.util.Objects.requireNonNull;
public class FlowController implements EventAccess, ControllerServiceProvider, ReportingTaskProvider, public class FlowController implements EventAccess, ControllerServiceProvider, ReportingTaskProvider,
QueueProvider, Authorizable, ProvenanceAuthorizableFactory, NodeTypeProvider, IdentifierLookup { QueueProvider, Authorizable, ProvenanceAuthorizableFactory, NodeTypeProvider, IdentifierLookup, ReloadComponent {
// default repository implementations // default repository implementations
public static final String DEFAULT_FLOWFILE_REPO_IMPLEMENTATION = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository"; public static final String DEFAULT_FLOWFILE_REPO_IMPLEMENTATION = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository";
@ -1049,7 +1050,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
* instantiated for any reason * instantiated for any reason
*/ */
public ProcessorNode createProcessor(final String type, String id, final BundleCoordinate coordinate, final boolean firstTimeAdded) throws ProcessorInstantiationException { public ProcessorNode createProcessor(final String type, String id, final BundleCoordinate coordinate, final boolean firstTimeAdded) throws ProcessorInstantiationException {
return createProcessor(type, id, coordinate, firstTimeAdded, true); return createProcessor(type, id, coordinate, Collections.emptySet(), firstTimeAdded, true);
} }
/** /**
@ -1069,14 +1070,14 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
* @throws ProcessorInstantiationException if the processor cannot be * @throws ProcessorInstantiationException if the processor cannot be
* instantiated for any reason * instantiated for any reason
*/ */
public ProcessorNode createProcessor(final String type, String id, final BundleCoordinate coordinate, final boolean firstTimeAdded, final boolean registerLogObserver) public ProcessorNode createProcessor(final String type, String id, final BundleCoordinate coordinate, final Set<URL> additionalUrls,
throws ProcessorInstantiationException { final boolean firstTimeAdded, final boolean registerLogObserver) throws ProcessorInstantiationException {
id = id.intern(); id = id.intern();
boolean creationSuccessful; boolean creationSuccessful;
LoggableComponent<Processor> processor; LoggableComponent<Processor> processor;
try { try {
processor = instantiateProcessor(type, id, coordinate); processor = instantiateProcessor(type, id, coordinate, additionalUrls);
creationSuccessful = true; creationSuccessful = true;
} catch (final ProcessorInstantiationException pie) { } catch (final ProcessorInstantiationException pie) {
LOG.error("Could not create Processor of type " + type + " for ID " + id + "; creating \"Ghost\" implementation", pie); LOG.error("Could not create Processor of type " + type + " for ID " + id + "; creating \"Ghost\" implementation", pie);
@ -1090,12 +1091,12 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, variableRegistry); final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, variableRegistry);
final ProcessorNode procNode; final ProcessorNode procNode;
if (creationSuccessful) { if (creationSuccessful) {
procNode = new StandardProcessorNode(processor, id, validationContextFactory, processScheduler, controllerServiceProvider, nifiProperties, variableRegistry); procNode = new StandardProcessorNode(processor, id, validationContextFactory, processScheduler, controllerServiceProvider, nifiProperties, variableRegistry, this);
} else { } else {
final String simpleClassName = type.contains(".") ? StringUtils.substringAfterLast(type, ".") : type; final String simpleClassName = type.contains(".") ? StringUtils.substringAfterLast(type, ".") : type;
final String componentType = "(Missing) " + simpleClassName; final String componentType = "(Missing) " + simpleClassName;
procNode = new StandardProcessorNode( procNode = new StandardProcessorNode(
processor, id, validationContextFactory, processScheduler, controllerServiceProvider, componentType, type, nifiProperties, variableRegistry, true); processor, id, validationContextFactory, processScheduler, controllerServiceProvider, componentType, type, nifiProperties, variableRegistry, this, true);
} }
final LogRepository logRepository = LogRepositoryFactory.getRepository(id); final LogRepository logRepository = LogRepositoryFactory.getRepository(id);
@ -1153,7 +1154,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
return procNode; return procNode;
} }
private LoggableComponent<Processor> instantiateProcessor(final String type, final String identifier, final BundleCoordinate bundleCoordinate) throws ProcessorInstantiationException { private LoggableComponent<Processor> instantiateProcessor(final String type, final String identifier, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls)
throws ProcessorInstantiationException {
final Bundle processorBundle = ExtensionManager.getBundle(bundleCoordinate); final Bundle processorBundle = ExtensionManager.getBundle(bundleCoordinate);
if (processorBundle == null) { if (processorBundle == null) {
throw new ProcessorInstantiationException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate()); throw new ProcessorInstantiationException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate());
@ -1161,9 +1164,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader(); final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader();
try { try {
final ClassLoader detectedClassLoaderForType = ExtensionManager.createInstanceClassLoader(type, identifier, processorBundle); final ClassLoader detectedClassLoaderForInstance = ExtensionManager.createInstanceClassLoader(type, identifier, processorBundle, additionalUrls);
final Class<?> rawClass = Class.forName(type, true, processorBundle.getClassLoader()); final Class<?> rawClass = Class.forName(type, true, detectedClassLoaderForInstance);
Thread.currentThread().setContextClassLoader(detectedClassLoaderForType); Thread.currentThread().setContextClassLoader(detectedClassLoaderForInstance);
final Class<? extends Processor> processorClass = rawClass.asSubclass(Processor.class); final Class<? extends Processor> processorClass = rawClass.asSubclass(Processor.class);
final Processor processor = processorClass.newInstance(); final Processor processor = processorClass.newInstance();
@ -1184,7 +1187,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
} }
} }
public void changeProcessorType(final ProcessorNode existingNode, final String newType, final BundleCoordinate bundleCoordinate) throws ProcessorInstantiationException { @Override
public void reload(final ProcessorNode existingNode, final String newType, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls)
throws ProcessorInstantiationException {
if (existingNode == null) { if (existingNode == null) {
throw new IllegalStateException("Existing ProcessorNode cannot be null"); throw new IllegalStateException("Existing ProcessorNode cannot be null");
} }
@ -1197,13 +1202,15 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
// create a new node with firstTimeAdded as true so lifecycle methods get fired // create a new node with firstTimeAdded as true so lifecycle methods get fired
// attempt the creation to make sure it works before firing the OnRemoved methods below // attempt the creation to make sure it works before firing the OnRemoved methods below
final ProcessorNode newNode = createProcessor(newType, id, bundleCoordinate, true, false); final ProcessorNode newNode = createProcessor(newType, id, bundleCoordinate, additionalUrls, true, false);
// call OnRemoved for the existing processor using the previous instance class loader // call OnRemoved for the existing processor using the previous instance class loader
try (final NarCloseable x = NarCloseable.withComponentNarLoader(existingInstanceClassLoader)) { try (final NarCloseable x = NarCloseable.withComponentNarLoader(existingInstanceClassLoader)) {
final StandardProcessContext processContext = new StandardProcessContext( final StandardProcessContext processContext = new StandardProcessContext(
existingNode, controllerServiceProvider, encryptor, getStateManagerProvider().getStateManager(id), variableRegistry); existingNode, controllerServiceProvider, encryptor, getStateManagerProvider().getStateManager(id), variableRegistry);
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, existingNode.getProcessor(), processContext); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, existingNode.getProcessor(), processContext);
} finally {
ExtensionManager.closeURLClassLoader(id, existingInstanceClassLoader);
} }
// set the new processor in the existing node // set the new processor in the existing node
@ -1701,7 +1708,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
// //
for (final ControllerServiceDTO controllerServiceDTO : dto.getControllerServices()) { for (final ControllerServiceDTO controllerServiceDTO : dto.getControllerServices()) {
final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(controllerServiceDTO.getType(), controllerServiceDTO.getBundle()); final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(controllerServiceDTO.getType(), controllerServiceDTO.getBundle());
final ControllerServiceNode serviceNode = createControllerService(controllerServiceDTO.getType(), controllerServiceDTO.getId(), bundleCoordinate, true); final ControllerServiceNode serviceNode = createControllerService(controllerServiceDTO.getType(), controllerServiceDTO.getId(), bundleCoordinate, Collections.emptySet(),true);
serviceNode.setAnnotationData(controllerServiceDTO.getAnnotationData()); serviceNode.setAnnotationData(controllerServiceDTO.getAnnotationData());
serviceNode.setComments(controllerServiceDTO.getComments()); serviceNode.setComments(controllerServiceDTO.getComments());
@ -2942,12 +2949,13 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
} }
@Override @Override
public ReportingTaskNode createReportingTask(final String type, final String id, final BundleCoordinate bundleCoordinate,final boolean firstTimeAdded) throws ReportingTaskInstantiationException { public ReportingTaskNode createReportingTask(final String type, final String id, final BundleCoordinate bundleCoordinate,final boolean firstTimeAdded)
return createReportingTask(type, id, bundleCoordinate, firstTimeAdded, true); throws ReportingTaskInstantiationException {
return createReportingTask(type, id, bundleCoordinate, Collections.emptySet(), firstTimeAdded, true);
} }
public ReportingTaskNode createReportingTask(final String type, final String id, final BundleCoordinate bundleCoordinate, final boolean firstTimeAdded, final boolean register) public ReportingTaskNode createReportingTask(final String type, final String id, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls,
throws ReportingTaskInstantiationException { final boolean firstTimeAdded, final boolean register) throws ReportingTaskInstantiationException {
if (type == null || id == null || bundleCoordinate == null) { if (type == null || id == null || bundleCoordinate == null) {
throw new NullPointerException(); throw new NullPointerException();
} }
@ -2955,7 +2963,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
LoggableComponent<ReportingTask> task = null; LoggableComponent<ReportingTask> task = null;
boolean creationSuccessful = true; boolean creationSuccessful = true;
try { try {
task = instantiateReportingTask(type, id, bundleCoordinate); task = instantiateReportingTask(type, id, bundleCoordinate, additionalUrls);
} catch (final Exception e) { } catch (final Exception e) {
LOG.error("Could not create Reporting Task of type " + type + " for ID " + id + "; creating \"Ghost\" implementation", e); LOG.error("Could not create Reporting Task of type " + type + " for ID " + id + "; creating \"Ghost\" implementation", e);
final GhostReportingTask ghostTask = new GhostReportingTask(); final GhostReportingTask ghostTask = new GhostReportingTask();
@ -2968,12 +2976,12 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, variableRegistry); final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, variableRegistry);
final ReportingTaskNode taskNode; final ReportingTaskNode taskNode;
if (creationSuccessful) { if (creationSuccessful) {
taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, variableRegistry); taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, variableRegistry, this);
} else { } else {
final String simpleClassName = type.contains(".") ? StringUtils.substringAfterLast(type, ".") : type; final String simpleClassName = type.contains(".") ? StringUtils.substringAfterLast(type, ".") : type;
final String componentType = "(Missing) " + simpleClassName; final String componentType = "(Missing) " + simpleClassName;
taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, componentType, type, variableRegistry, true); taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, componentType, type, variableRegistry, this, true);
} }
taskNode.setName(taskNode.getReportingTask().getClass().getSimpleName()); taskNode.setName(taskNode.getReportingTask().getClass().getSimpleName());
@ -3008,7 +3016,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
return taskNode; return taskNode;
} }
private LoggableComponent<ReportingTask> instantiateReportingTask(final String type, final String id, final BundleCoordinate bundleCoordinate) private LoggableComponent<ReportingTask> instantiateReportingTask(final String type, final String id, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls)
throws ReportingTaskInstantiationException { throws ReportingTaskInstantiationException {
final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader(); final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader();
@ -3018,7 +3026,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
throw new IllegalStateException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate()); throw new IllegalStateException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate());
} }
final ClassLoader detectedClassLoader = ExtensionManager.createInstanceClassLoader(type, id, reportingTaskBundle); final ClassLoader detectedClassLoader = ExtensionManager.createInstanceClassLoader(type, id, reportingTaskBundle, additionalUrls);
final Class<?> rawClass = Class.forName(type, false, detectedClassLoader); final Class<?> rawClass = Class.forName(type, false, detectedClassLoader);
Thread.currentThread().setContextClassLoader(detectedClassLoader); Thread.currentThread().setContextClassLoader(detectedClassLoader);
@ -3039,7 +3047,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
} }
@Override @Override
public void changeReportingTaskType(final ReportingTaskNode existingNode, final String newType, final BundleCoordinate bundleCoordinate) throws ReportingTaskInstantiationException { public void reload(final ReportingTaskNode existingNode, final String newType, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls)
throws ReportingTaskInstantiationException {
if (existingNode == null) { if (existingNode == null) {
throw new IllegalStateException("Existing ReportingTaskNode cannot be null"); throw new IllegalStateException("Existing ReportingTaskNode cannot be null");
} }
@ -3052,11 +3061,13 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
// set firstTimeAdded to true so lifecycle annotations get fired, but don't register this node // set firstTimeAdded to true so lifecycle annotations get fired, but don't register this node
// attempt the creation to make sure it works before firing the OnRemoved methods below // attempt the creation to make sure it works before firing the OnRemoved methods below
final ReportingTaskNode newNode = createReportingTask(newType, id, bundleCoordinate, true, false); final ReportingTaskNode newNode = createReportingTask(newType, id, bundleCoordinate, additionalUrls, true, false);
// call OnRemoved for the existing reporting task using the previous instance class loader // call OnRemoved for the existing reporting task using the previous instance class loader
try (final NarCloseable x = NarCloseable.withComponentNarLoader(existingInstanceClassLoader)) { try (final NarCloseable x = NarCloseable.withComponentNarLoader(existingInstanceClassLoader)) {
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, existingNode.getReportingTask(), existingNode.getConfigurationContext()); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, existingNode.getReportingTask(), existingNode.getConfigurationContext());
} finally {
ExtensionManager.closeURLClassLoader(id, existingInstanceClassLoader);
} }
// set the new reporting task into the existing node // set the new reporting task into the existing node
@ -3117,7 +3128,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
} }
reportingTasks.remove(reportingTaskNode.getIdentifier()); reportingTasks.remove(reportingTaskNode.getIdentifier());
ExtensionManager.removeInstanceClassLoaderIfExists(reportingTaskNode.getIdentifier()); ExtensionManager.removeInstanceClassLoader(reportingTaskNode.getIdentifier());
} }
@Override @Override
@ -3126,8 +3137,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
} }
@Override @Override
public ControllerServiceNode createControllerService(final String type, final String id, final BundleCoordinate bundleCoordinate, final boolean firstTimeAdded) { public ControllerServiceNode createControllerService(final String type, final String id, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls, final boolean firstTimeAdded) {
final ControllerServiceNode serviceNode = controllerServiceProvider.createControllerService(type, id, bundleCoordinate, firstTimeAdded); final ControllerServiceNode serviceNode = controllerServiceProvider.createControllerService(type, id, bundleCoordinate, additionalUrls, firstTimeAdded);
// Register log observer to provide bulletins when reporting task logs anything at WARN level or above // Register log observer to provide bulletins when reporting task logs anything at WARN level or above
final LogRepository logRepository = LogRepositoryFactory.getRepository(id); final LogRepository logRepository = LogRepositoryFactory.getRepository(id);
@ -3145,7 +3156,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
return serviceNode; return serviceNode;
} }
public void changeControllerServiceType(final ControllerServiceNode existingNode, final String newType, final BundleCoordinate bundleCoordinate) @Override
public void reload(final ControllerServiceNode existingNode, final String newType, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls)
throws ControllerServiceInstantiationException { throws ControllerServiceInstantiationException {
if (existingNode == null) { if (existingNode == null) {
throw new IllegalStateException("Existing ControllerServiceNode cannot be null"); throw new IllegalStateException("Existing ControllerServiceNode cannot be null");
@ -3159,12 +3171,14 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
// create a new node with firstTimeAdded as true so lifecycle methods get called // create a new node with firstTimeAdded as true so lifecycle methods get called
// attempt the creation to make sure it works before firing the OnRemoved methods below // attempt the creation to make sure it works before firing the OnRemoved methods below
final ControllerServiceNode newNode = controllerServiceProvider.createControllerService(newType, id, bundleCoordinate, true); final ControllerServiceNode newNode = controllerServiceProvider.createControllerService(newType, id, bundleCoordinate, additionalUrls, true);
// call OnRemoved for the existing service using the previous instance class loader // call OnRemoved for the existing service using the previous instance class loader
try (final NarCloseable x = NarCloseable.withComponentNarLoader(existingInstanceClassLoader)) { try (final NarCloseable x = NarCloseable.withComponentNarLoader(existingInstanceClassLoader)) {
final ConfigurationContext configurationContext = new StandardConfigurationContext(existingNode, controllerServiceProvider, null, variableRegistry); final ConfigurationContext configurationContext = new StandardConfigurationContext(existingNode, controllerServiceProvider, null, variableRegistry);
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, existingNode.getControllerServiceImplementation(), configurationContext); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, existingNode.getControllerServiceImplementation(), configurationContext);
} finally {
ExtensionManager.closeURLClassLoader(id, existingInstanceClassLoader);
} }
// take the invocation handler that was created for new proxy and is set to look at the new node, // take the invocation handler that was created for new proxy and is set to look at the new node,
@ -3313,7 +3327,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
rootControllerServices.remove(service.getIdentifier()); rootControllerServices.remove(service.getIdentifier());
getStateManagerProvider().onComponentRemoved(service.getIdentifier()); getStateManagerProvider().onComponentRemoved(service.getIdentifier());
ExtensionManager.removeInstanceClassLoaderIfExists(service.getIdentifier()); ExtensionManager.removeInstanceClassLoader(service.getIdentifier());
LOG.info("{} removed from Flow Controller", service, this); LOG.info("{} removed from Flow Controller", service, this);
} }

View File

@ -40,6 +40,7 @@ import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.ConnectableType;
import org.apache.nifi.connectable.Connection; import org.apache.nifi.connectable.Connection;
import org.apache.nifi.connectable.Position; import org.apache.nifi.connectable.Position;
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
import org.apache.nifi.controller.scheduling.ScheduleState; import org.apache.nifi.controller.scheduling.ScheduleState;
import org.apache.nifi.controller.scheduling.SchedulingAgent; import org.apache.nifi.controller.scheduling.SchedulingAgent;
import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceNode;
@ -66,6 +67,7 @@ import org.slf4j.LoggerFactory;
import org.springframework.util.Assert; import org.springframework.util.Assert;
import java.lang.reflect.InvocationTargetException; import java.lang.reflect.InvocationTargetException;
import java.net.URL;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
@ -132,19 +134,19 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
public StandardProcessorNode(final LoggableComponent<Processor> processor, final String uuid, public StandardProcessorNode(final LoggableComponent<Processor> processor, final String uuid,
final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler, final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler,
final ControllerServiceProvider controllerServiceProvider, final NiFiProperties nifiProperties, final ControllerServiceProvider controllerServiceProvider, final NiFiProperties nifiProperties,
final VariableRegistry variableRegistry) { final VariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
this(processor, uuid, validationContextFactory, scheduler, controllerServiceProvider, this(processor, uuid, validationContextFactory, scheduler, controllerServiceProvider,
processor.getComponent().getClass().getSimpleName(), processor.getComponent().getClass().getCanonicalName(), nifiProperties, variableRegistry, false); processor.getComponent().getClass().getSimpleName(), processor.getComponent().getClass().getCanonicalName(), nifiProperties, variableRegistry, reloadComponent, false);
} }
public StandardProcessorNode(final LoggableComponent<Processor> processor, final String uuid, public StandardProcessorNode(final LoggableComponent<Processor> processor, final String uuid,
final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler, final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler,
final ControllerServiceProvider controllerServiceProvider, final ControllerServiceProvider controllerServiceProvider,
final String componentType, final String componentCanonicalClass, final NiFiProperties nifiProperties, final String componentType, final String componentCanonicalClass, final NiFiProperties nifiProperties,
final VariableRegistry variableRegistry, final boolean isExtensionMissing) { final VariableRegistry variableRegistry, final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
super(uuid, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, isExtensionMissing); super(uuid, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, isExtensionMissing);
final ProcessorDetails processorDetails = new ProcessorDetails(processor); final ProcessorDetails processorDetails = new ProcessorDetails(processor);
this.processorRef = new AtomicReference<>(processorDetails); this.processorRef = new AtomicReference<>(processorDetails);
@ -867,6 +869,15 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
processorRef.set(processorDetails); processorRef.set(processorDetails);
} }
@Override
public synchronized void reload(final Set<URL> additionalUrls) throws ProcessorInstantiationException {
if (isRunning()) {
throw new IllegalStateException("Cannot reload Processor while the Processor is running");
}
getReloadComponent().reload(this, getCanonicalClassName(), getBundleCoordinate(), additionalUrls);
}
/** /**
* @return the Set of destination processors for all relationships excluding * @return the Set of destination processors for all relationships excluding
* any destinations that are this processor itself (self-loops) * any destinations that are this processor itself (self-loops)

View File

@ -21,6 +21,7 @@ import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.components.ConfigurableComponent; import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.controller.AbstractConfiguredComponent; import org.apache.nifi.controller.AbstractConfiguredComponent;
import org.apache.nifi.controller.ReloadComponent;
import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.controller.ControllerServiceLookup; import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.controller.LoggableComponent; import org.apache.nifi.controller.LoggableComponent;
@ -37,6 +38,7 @@ import org.apache.nifi.reporting.ReportingTask;
import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.scheduling.SchedulingStrategy;
import org.apache.nifi.util.FormatUtils; import org.apache.nifi.util.FormatUtils;
import java.net.URL;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.HashSet; import java.util.HashSet;
@ -64,19 +66,21 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon
public AbstractReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id, public AbstractReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id,
final ControllerServiceProvider controllerServiceProvider, final ProcessScheduler processScheduler, final ControllerServiceProvider controllerServiceProvider, final ProcessScheduler processScheduler,
final ValidationContextFactory validationContextFactory, final VariableRegistry variableRegistry) { final ValidationContextFactory validationContextFactory, final VariableRegistry variableRegistry,
final ReloadComponent reloadComponent) {
this(reportingTask, id, controllerServiceProvider, processScheduler, validationContextFactory, this(reportingTask, id, controllerServiceProvider, processScheduler, validationContextFactory,
reportingTask.getComponent().getClass().getSimpleName(), reportingTask.getComponent().getClass().getCanonicalName(),variableRegistry, false); reportingTask.getComponent().getClass().getSimpleName(), reportingTask.getComponent().getClass().getCanonicalName(),
variableRegistry, reloadComponent, false);
} }
public AbstractReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id, final ControllerServiceProvider controllerServiceProvider, public AbstractReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id, final ControllerServiceProvider controllerServiceProvider,
final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory, final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory,
final String componentType, final String componentCanonicalClass, final VariableRegistry variableRegistry, final String componentType, final String componentCanonicalClass, final VariableRegistry variableRegistry,
final boolean isExtensionMissing) { final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
super(id, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, isExtensionMissing); super(id, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, isExtensionMissing);
this.reportingTaskRef = new AtomicReference<>(new ReportingTaskDetails(reportingTask)); this.reportingTaskRef = new AtomicReference<>(new ReportingTaskDetails(reportingTask));
this.processScheduler = processScheduler; this.processScheduler = processScheduler;
this.serviceLookup = controllerServiceProvider; this.serviceLookup = controllerServiceProvider;
@ -152,6 +156,15 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon
this.reportingTaskRef.set(new ReportingTaskDetails(reportingTask)); this.reportingTaskRef.set(new ReportingTaskDetails(reportingTask));
} }
@Override
public void reload(final Set<URL> additionalUrls) throws ReportingTaskInstantiationException {
if (isRunning()) {
throw new IllegalStateException("Cannot reload Reporting Task while Reporting Task is running");
}
getReloadComponent().reload(this, getCanonicalClassName(), getBundleCoordinate(), additionalUrls);
}
@Override @Override
public boolean isRunning() { public boolean isRunning() {
return processScheduler.isScheduled(this) || processScheduler.getActiveThreadCount(this) > 0; return processScheduler.isScheduled(this) || processScheduler.getActiveThreadCount(this) > 0;

View File

@ -21,6 +21,7 @@ import org.apache.nifi.authorization.Resource;
import org.apache.nifi.authorization.resource.Authorizable; import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.authorization.resource.ResourceFactory; import org.apache.nifi.authorization.resource.ResourceFactory;
import org.apache.nifi.authorization.resource.ResourceType; import org.apache.nifi.authorization.resource.ResourceType;
import org.apache.nifi.controller.ReloadComponent;
import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.LoggableComponent; import org.apache.nifi.controller.LoggableComponent;
import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.controller.ProcessScheduler;
@ -36,16 +37,17 @@ public class StandardReportingTaskNode extends AbstractReportingTaskNode impleme
public StandardReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id, final FlowController controller, public StandardReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id, final FlowController controller,
final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory, final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory,
final VariableRegistry variableRegistry) { final VariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
super(reportingTask, id, controller, processScheduler, validationContextFactory, variableRegistry); super(reportingTask, id, controller, processScheduler, validationContextFactory, variableRegistry, reloadComponent);
this.flowController = controller; this.flowController = controller;
} }
public StandardReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id, final FlowController controller, public StandardReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id, final FlowController controller,
final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory, final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory,
final String componentType, final String canonicalClassName, final VariableRegistry variableRegistry, final String componentType, final String canonicalClassName, final VariableRegistry variableRegistry,
final boolean isExtensionMissing) { final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
super(reportingTask, id, controller, processScheduler, validationContextFactory, componentType, canonicalClassName,variableRegistry, isExtensionMissing); super(reportingTask, id, controller, processScheduler, validationContextFactory, componentType, canonicalClassName,
variableRegistry, reloadComponent, isExtensionMissing);
this.flowController = controller; this.flowController = controller;
} }

View File

@ -43,6 +43,7 @@ import java.io.InputStream;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
@ -163,7 +164,8 @@ public class ControllerServiceLoader {
// create a new id for the clone seeded from the original id so that it is consistent in a cluster // create a new id for the clone seeded from the original id so that it is consistent in a cluster
final UUID id = UUID.nameUUIDFromBytes(controllerService.getIdentifier().getBytes(StandardCharsets.UTF_8)); final UUID id = UUID.nameUUIDFromBytes(controllerService.getIdentifier().getBytes(StandardCharsets.UTF_8));
final ControllerServiceNode clone = provider.createControllerService(controllerService.getCanonicalClassName(), id.toString(), controllerService.getBundleCoordinate(), false); final ControllerServiceNode clone = provider.createControllerService(controllerService.getCanonicalClassName(), id.toString(),
controllerService.getBundleCoordinate(), Collections.emptySet(), false);
clone.setName(controllerService.getName()); clone.setName(controllerService.getName());
clone.setComments(controllerService.getComments()); clone.setComments(controllerService.getComments());
@ -193,7 +195,7 @@ public class ControllerServiceLoader {
} }
} }
final ControllerServiceNode node = provider.createControllerService(dto.getType(), dto.getId(), coordinate, false); final ControllerServiceNode node = provider.createControllerService(dto.getType(), dto.getId(), coordinate, Collections.emptySet(), false);
node.setName(dto.getName()); node.setName(dto.getName());
node.setComments(dto.getComments()); node.setComments(dto.getComments());
return node; return node;

View File

@ -29,11 +29,13 @@ import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.controller.AbstractConfiguredComponent; import org.apache.nifi.controller.AbstractConfiguredComponent;
import org.apache.nifi.controller.ReloadComponent;
import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.controller.ConfiguredComponent; import org.apache.nifi.controller.ConfiguredComponent;
import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.LoggableComponent; import org.apache.nifi.controller.LoggableComponent;
import org.apache.nifi.controller.ValidationContextFactory; import org.apache.nifi.controller.ValidationContextFactory;
import org.apache.nifi.controller.exception.ControllerServiceInstantiationException;
import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.nar.NarCloseable;
@ -44,6 +46,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.lang.reflect.InvocationTargetException; import java.lang.reflect.InvocationTargetException;
import java.net.URL;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
@ -79,18 +82,18 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
public StandardControllerServiceNode(final LoggableComponent<ControllerService> implementation, final LoggableComponent<ControllerService> proxiedControllerService, public StandardControllerServiceNode(final LoggableComponent<ControllerService> implementation, final LoggableComponent<ControllerService> proxiedControllerService,
final ControllerServiceInvocationHandler invocationHandler, final String id, final ValidationContextFactory validationContextFactory, final ControllerServiceInvocationHandler invocationHandler, final String id, final ValidationContextFactory validationContextFactory,
final ControllerServiceProvider serviceProvider, final VariableRegistry variableRegistry) { final ControllerServiceProvider serviceProvider, final VariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
this(implementation, proxiedControllerService, invocationHandler, id, validationContextFactory, serviceProvider, this(implementation, proxiedControllerService, invocationHandler, id, validationContextFactory, serviceProvider,
implementation.getComponent().getClass().getSimpleName(), implementation.getComponent().getClass().getCanonicalName(), variableRegistry, false); implementation.getComponent().getClass().getSimpleName(), implementation.getComponent().getClass().getCanonicalName(), variableRegistry, reloadComponent, false);
} }
public StandardControllerServiceNode(final LoggableComponent<ControllerService> implementation, final LoggableComponent<ControllerService> proxiedControllerService, public StandardControllerServiceNode(final LoggableComponent<ControllerService> implementation, final LoggableComponent<ControllerService> proxiedControllerService,
final ControllerServiceInvocationHandler invocationHandler, final String id, final ValidationContextFactory validationContextFactory, final ControllerServiceInvocationHandler invocationHandler, final String id, final ValidationContextFactory validationContextFactory,
final ControllerServiceProvider serviceProvider, final String componentType, final String componentCanonicalClass, final ControllerServiceProvider serviceProvider, final String componentType, final String componentCanonicalClass,
final VariableRegistry variableRegistry, final boolean isExtensionMissing) { final VariableRegistry variableRegistry, final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, isExtensionMissing); super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, isExtensionMissing);
this.serviceProvider = serviceProvider; this.serviceProvider = serviceProvider;
this.active = new AtomicBoolean(); this.active = new AtomicBoolean();
setControllerServiceAndProxy(implementation, proxiedControllerService, invocationHandler); setControllerServiceAndProxy(implementation, proxiedControllerService, invocationHandler);
@ -171,6 +174,17 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
} }
} }
@Override
public void reload(final Set<URL> additionalUrls) throws ControllerServiceInstantiationException {
synchronized (this.active) {
if (isActive()) {
throw new IllegalStateException("Cannot reload Controller Service while service is active");
}
getReloadComponent().reload(this, getCanonicalClassName(), getBundleCoordinate(), additionalUrls);
}
}
@Override @Override
public ProcessGroup getProcessGroup() { public ProcessGroup getProcessGroup() {
readLock.lock(); readLock.lock();

View File

@ -53,6 +53,7 @@ import org.slf4j.LoggerFactory;
import java.lang.reflect.Method; import java.lang.reflect.Method;
import java.lang.reflect.Proxy; import java.lang.reflect.Proxy;
import java.net.URL;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
@ -91,7 +92,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
} }
@Override @Override
public ControllerServiceNode createControllerService(final String type, final String id, final BundleCoordinate bundleCoordinate, final boolean firstTimeAdded) { public ControllerServiceNode createControllerService(final String type, final String id, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls, final boolean firstTimeAdded) {
if (type == null || id == null || bundleCoordinate == null) { if (type == null || id == null || bundleCoordinate == null) {
throw new NullPointerException(); throw new NullPointerException();
} }
@ -106,7 +107,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
throw new ControllerServiceInstantiationException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate()); throw new ControllerServiceInstantiationException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate());
} }
cl = ExtensionManager.createInstanceClassLoader(type, id, csBundle); cl = ExtensionManager.createInstanceClassLoader(type, id, csBundle, additionalUrls);
Thread.currentThread().setContextClassLoader(cl); Thread.currentThread().setContextClassLoader(cl);
rawClass = Class.forName(type, false, cl); rawClass = Class.forName(type, false, cl);
} catch (final Exception e) { } catch (final Exception e) {
@ -141,7 +142,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
final LoggableComponent<ControllerService> proxiedLoggableComponent = new LoggableComponent<>(proxiedService, bundleCoordinate, serviceLogger); final LoggableComponent<ControllerService> proxiedLoggableComponent = new LoggableComponent<>(proxiedService, bundleCoordinate, serviceLogger);
final ControllerServiceNode serviceNode = new StandardControllerServiceNode(originalLoggableComponent, proxiedLoggableComponent, invocationHandler, final ControllerServiceNode serviceNode = new StandardControllerServiceNode(originalLoggableComponent, proxiedLoggableComponent, invocationHandler,
id, validationContextFactory, this, variableRegistry); id, validationContextFactory, this, variableRegistry, flowController);
serviceNode.setName(rawClass.getSimpleName()); serviceNode.setName(rawClass.getSimpleName());
invocationHandler.setServiceNode(serviceNode); invocationHandler.setServiceNode(serviceNode);
@ -217,7 +218,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
final LoggableComponent<ControllerService> proxiedLoggableComponent = new LoggableComponent<>(proxiedService, bundleCoordinate, null); final LoggableComponent<ControllerService> proxiedLoggableComponent = new LoggableComponent<>(proxiedService, bundleCoordinate, null);
final ControllerServiceNode serviceNode = new StandardControllerServiceNode(proxiedLoggableComponent, proxiedLoggableComponent, invocationHandler, id, final ControllerServiceNode serviceNode = new StandardControllerServiceNode(proxiedLoggableComponent, proxiedLoggableComponent, invocationHandler, id,
new StandardValidationContextFactory(this, variableRegistry), this, componentType, type, variableRegistry, true); new StandardValidationContextFactory(this, variableRegistry), this, componentType, type, variableRegistry, flowController, true);
return serviceNode; return serviceNode;
} }
@ -537,7 +538,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
} }
group.removeControllerService(serviceNode); group.removeControllerService(serviceNode);
ExtensionManager.removeInstanceClassLoaderIfExists(serviceNode.getIdentifier()); ExtensionManager.removeInstanceClassLoader(serviceNode.getIdentifier());
} }
@Override @Override

View File

@ -20,17 +20,13 @@ import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.bundle.BundleCoordinate; import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.components.ConfigurableComponent; import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
import org.apache.nifi.controller.serialization.FlowFromDOMFactory; import org.apache.nifi.controller.serialization.FlowFromDOMFactory;
import org.apache.nifi.util.LoggingXmlParserErrorHandler;
import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.encrypt.StringEncryptor;
import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.reporting.ReportingTask;
import org.apache.nifi.util.BundleUtils; import org.apache.nifi.util.BundleUtils;
import org.apache.nifi.util.DomUtils; import org.apache.nifi.util.DomUtils;
import org.apache.nifi.util.LoggingXmlParserErrorHandler;
import org.apache.nifi.web.api.dto.BundleDTO; import org.apache.nifi.web.api.dto.BundleDTO;
import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.ControllerServiceDTO;
import org.apache.nifi.web.api.dto.ReportingTaskDTO; import org.apache.nifi.web.api.dto.ReportingTaskDTO;
@ -58,7 +54,6 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.SortedMap; import java.util.SortedMap;
import java.util.TreeMap; import java.util.TreeMap;
import java.util.UUID;
/** /**
* <p>Creates a fingerprint of a flow.xml. The order of elements or attributes in the flow.xml does not influence the fingerprint generation. * <p>Creates a fingerprint of a flow.xml. The order of elements or attributes in the flow.xml does not influence the fingerprint generation.
@ -232,7 +227,7 @@ public class FingerprintFactory {
}); });
for (final ControllerServiceDTO dto : serviceDtos) { for (final ControllerServiceDTO dto : serviceDtos) {
addControllerServiceFingerprint(builder, dto, controller); addControllerServiceFingerprint(builder, dto);
} }
} }
@ -262,7 +257,7 @@ public class FingerprintFactory {
}); });
for (final ReportingTaskDTO dto : reportingTaskDtos) { for (final ReportingTaskDTO dto : reportingTaskDtos) {
addReportingTaskFingerprint(builder, dto, controller); addReportingTaskFingerprint(builder, dto);
} }
} }
@ -277,7 +272,7 @@ public class FingerprintFactory {
final List<Element> processorElems = DomUtils.getChildElementsByTagName(processGroupElem, "processor"); final List<Element> processorElems = DomUtils.getChildElementsByTagName(processGroupElem, "processor");
Collections.sort(processorElems, getIdsComparator()); Collections.sort(processorElems, getIdsComparator());
for (final Element processorElem : processorElems) { for (final Element processorElem : processorElems) {
addFlowFileProcessorFingerprint(builder, processorElem, controller); addFlowFileProcessorFingerprint(builder, processorElem);
} }
// input ports // input ports
@ -332,7 +327,7 @@ public class FingerprintFactory {
return builder; return builder;
} }
private StringBuilder addFlowFileProcessorFingerprint(final StringBuilder builder, final Element processorElem, final FlowController controller) throws FingerprintException { private StringBuilder addFlowFileProcessorFingerprint(final StringBuilder builder, final Element processorElem) throws FingerprintException {
// id // id
appendFirstValue(builder, DomUtils.getChildNodesByTagName(processorElem, "id")); appendFirstValue(builder, DomUtils.getChildNodesByTagName(processorElem, "id"));
// class // class
@ -346,24 +341,11 @@ public class FingerprintFactory {
final BundleDTO bundle = FlowFromDOMFactory.getBundle(DomUtils.getChild(processorElem, "bundle")); final BundleDTO bundle = FlowFromDOMFactory.getBundle(DomUtils.getChild(processorElem, "bundle"));
addBundleFingerprint(builder, bundle); addBundleFingerprint(builder, bundle);
// create an instance of the Processor so that we know the default property values // get the temp instance of the Processor so that we know the default property values
Processor processor = null; final BundleCoordinate coordinate = getCoordinate(className, bundle);
try { final ConfigurableComponent configurableComponent = ExtensionManager.getTempComponent(className, coordinate);
if (controller != null) { if (configurableComponent == null) {
final BundleCoordinate coordinate = getCoordinate(className, bundle); logger.warn("Unable to get Processor of type {}; its default properties will be fingerprinted instead of being ignored.", className);
processor = controller.createProcessor(className, UUID.randomUUID().toString(), coordinate, false).getProcessor();
}
} catch (ProcessorInstantiationException | IllegalStateException e) {
logger.warn("Unable to create Processor of type {} due to {}; its default properties will be fingerprinted instead of being ignored.", className, e.toString());
if (logger.isDebugEnabled()) {
logger.warn("", e);
}
} finally {
// The processor instance is only for fingerprinting so we can remove the InstanceClassLoader here
// since otherwise it will stick around in the map forever
if (processor != null) {
ExtensionManager.removeInstanceClassLoaderIfExists(processor.getIdentifier());
}
} }
// properties // properties
@ -372,7 +354,7 @@ public class FingerprintFactory {
for (final Element propertyElem : sortedPropertyElems) { for (final Element propertyElem : sortedPropertyElems) {
final String propName = DomUtils.getChildElementsByTagName(propertyElem, "name").get(0).getTextContent(); final String propName = DomUtils.getChildElementsByTagName(propertyElem, "name").get(0).getTextContent();
String propValue = getFirstValue(DomUtils.getChildNodesByTagName(propertyElem, "value"), null); String propValue = getFirstValue(DomUtils.getChildNodesByTagName(propertyElem, "value"), null);
addPropertyFingerprint(builder, processor, propName, propValue); addPropertyFingerprint(builder, configurableComponent, propName, propValue);
} }
final NodeList autoTerminateElems = DomUtils.getChildNodesByTagName(processorElem, "autoTerminatedRelationship"); final NodeList autoTerminateElems = DomUtils.getChildNodesByTagName(processorElem, "autoTerminatedRelationship");
@ -571,7 +553,7 @@ public class FingerprintFactory {
return builder; return builder;
} }
private void addControllerServiceFingerprint(final StringBuilder builder, final ControllerServiceDTO dto, final FlowController controller) { private void addControllerServiceFingerprint(final StringBuilder builder, final ControllerServiceDTO dto) {
builder.append(dto.getId()); builder.append(dto.getId());
builder.append(dto.getType()); builder.append(dto.getType());
builder.append(dto.getName()); builder.append(dto.getName());
@ -582,21 +564,14 @@ public class FingerprintFactory {
builder.append(dto.getAnnotationData()); builder.append(dto.getAnnotationData());
builder.append(dto.getState()); builder.append(dto.getState());
// create an instance of the ControllerService so that we know the default property values // get the temp instance of the ControllerService so that we know the default property values
ControllerService controllerService = null; final BundleCoordinate coordinate = getCoordinate(dto.getType(), dto.getBundle());
try { final ConfigurableComponent configurableComponent = ExtensionManager.getTempComponent(dto.getType(), coordinate);
if (controller != null) { if (configurableComponent == null) {
final BundleCoordinate coordinate = getCoordinate(dto.getType(), dto.getBundle()); logger.warn("Unable to get ControllerService of type {}; its default properties will be fingerprinted instead of being ignored.", dto.getType());
controllerService = controller.createControllerService(dto.getType(), UUID.randomUUID().toString(), coordinate, false).getControllerServiceImplementation();
}
} catch (Exception e) {
logger.warn("Unable to create ControllerService of type {} due to {}; its default properties will be fingerprinted instead of being ignored.", dto.getType(), e.toString());
if (logger.isDebugEnabled()) {
logger.warn("", e);
}
} }
addPropertiesFingerprint(builder, controllerService, dto.getProperties()); addPropertiesFingerprint(builder, configurableComponent, dto.getProperties());
} }
private void addPropertiesFingerprint(final StringBuilder builder, final ConfigurableComponent component, final Map<String, String> properties) { private void addPropertiesFingerprint(final StringBuilder builder, final ConfigurableComponent component, final Map<String, String> properties) {
@ -634,7 +609,7 @@ public class FingerprintFactory {
return coordinate; return coordinate;
} }
private void addReportingTaskFingerprint(final StringBuilder builder, final ReportingTaskDTO dto, final FlowController controller) { private void addReportingTaskFingerprint(final StringBuilder builder, final ReportingTaskDTO dto) {
builder.append(dto.getId()); builder.append(dto.getId());
builder.append(dto.getType()); builder.append(dto.getType());
builder.append(dto.getName()); builder.append(dto.getName());
@ -646,21 +621,14 @@ public class FingerprintFactory {
builder.append(dto.getSchedulingStrategy()); builder.append(dto.getSchedulingStrategy());
builder.append(dto.getAnnotationData()); builder.append(dto.getAnnotationData());
// create an instance of the ReportingTask so that we know the default property values // get the temp instance of the ReportingTask so that we know the default property values
ReportingTask reportingTask = null; final BundleCoordinate coordinate = getCoordinate(dto.getType(), dto.getBundle());
try { final ConfigurableComponent configurableComponent = ExtensionManager.getTempComponent(dto.getType(), coordinate);
if (controller != null) { if (configurableComponent == null) {
final BundleCoordinate coordinate = getCoordinate(dto.getType(), dto.getBundle()); logger.warn("Unable to get ReportingTask of type {}; its default properties will be fingerprinted instead of being ignored.", dto.getType());
reportingTask = controller.createReportingTask(dto.getType(), UUID.randomUUID().toString(), coordinate, false, false).getReportingTask();
}
} catch (Exception e) {
logger.warn("Unable to create ReportingTask of type {} due to {}; its default properties will be fingerprinted instead of being ignored.", dto.getType(), e.toString());
if (logger.isDebugEnabled()) {
logger.warn("", e);
}
} }
addPropertiesFingerprint(builder, reportingTask, dto.getProperties()); addPropertiesFingerprint(builder, configurableComponent, dto.getProperties());
} }
private Comparator<Element> getIdsComparator() { private Comparator<Element> getIdsComparator() {

View File

@ -767,7 +767,7 @@ public final class StandardProcessGroup implements ProcessGroup {
} finally { } finally {
if (removed) { if (removed) {
try { try {
ExtensionManager.removeInstanceClassLoaderIfExists(id); ExtensionManager.removeInstanceClassLoader(id);
} catch (Throwable t) { } catch (Throwable t) {
} }
} }
@ -1914,7 +1914,7 @@ public final class StandardProcessGroup implements ProcessGroup {
} finally { } finally {
if (removed) { if (removed) {
try { try {
ExtensionManager.removeInstanceClassLoaderIfExists(service.getIdentifier()); ExtensionManager.removeInstanceClassLoader(service.getIdentifier());
} catch (Throwable t) { } catch (Throwable t) {
} }
} }

View File

@ -26,6 +26,7 @@ import org.apache.nifi.controller.queue.FlowFileQueue
import org.apache.nifi.groups.ProcessGroup import org.apache.nifi.groups.ProcessGroup
import org.apache.nifi.groups.RemoteProcessGroup import org.apache.nifi.groups.RemoteProcessGroup
import org.apache.nifi.nar.ExtensionManager import org.apache.nifi.nar.ExtensionManager
import org.apache.nifi.nar.SystemBundle
import org.apache.nifi.processor.Relationship import org.apache.nifi.processor.Relationship
import org.apache.nifi.reporting.BulletinRepository import org.apache.nifi.reporting.BulletinRepository
import org.apache.nifi.util.NiFiProperties import org.apache.nifi.util.NiFiProperties
@ -43,7 +44,7 @@ class StandardFlowSynchronizerSpec extends Specification {
System.setProperty NiFiProperties.PROPERTIES_FILE_PATH, propFile System.setProperty NiFiProperties.PROPERTIES_FILE_PATH, propFile
def niFiProperties = NiFiProperties.createBasicNiFiProperties(null, null); def niFiProperties = NiFiProperties.createBasicNiFiProperties(null, null);
systemBundle = ExtensionManager.createSystemBundle(niFiProperties); systemBundle = SystemBundle.create(niFiProperties);
ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet()); ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
} }

View File

@ -44,6 +44,8 @@ import org.apache.nifi.logging.LogLevel;
import org.apache.nifi.logging.LogRepository; import org.apache.nifi.logging.LogRepository;
import org.apache.nifi.logging.LogRepositoryFactory; import org.apache.nifi.logging.LogRepositoryFactory;
import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.InstanceClassLoader;
import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.Relationship;
import org.apache.nifi.provenance.MockProvenanceRepository; import org.apache.nifi.provenance.MockProvenanceRepository;
import org.apache.nifi.registry.VariableRegistry; import org.apache.nifi.registry.VariableRegistry;
@ -67,6 +69,8 @@ import java.io.File;
import java.io.FileInputStream; import java.io.FileInputStream;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.net.MalformedURLException;
import java.net.URL;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
@ -115,7 +119,7 @@ public class TestFlowController {
encryptor = StringEncryptor.createEncryptor(nifiProperties); encryptor = StringEncryptor.createEncryptor(nifiProperties);
// use the system bundle // use the system bundle
systemBundle = ExtensionManager.createSystemBundle(nifiProperties); systemBundle = SystemBundle.create(nifiProperties);
ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet()); ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
User user1 = new User.Builder().identifier("user-id-1").identity("user-1").build(); User user1 = new User.Builder().identifier("user-id-1").identity("user-1").build();
@ -465,7 +469,7 @@ public class TestFlowController {
@Test @Test
public void testCreateMissingControllerService() throws ProcessorInstantiationException { public void testCreateMissingControllerService() throws ProcessorInstantiationException {
final ControllerServiceNode serviceNode = controller.createControllerService("org.apache.nifi.NonExistingControllerService", "1234-Controller-Service", final ControllerServiceNode serviceNode = controller.createControllerService("org.apache.nifi.NonExistingControllerService", "1234-Controller-Service",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
assertNotNull(serviceNode); assertNotNull(serviceNode);
assertEquals("org.apache.nifi.NonExistingControllerService", serviceNode.getCanonicalClassName()); assertEquals("org.apache.nifi.NonExistingControllerService", serviceNode.getCanonicalClassName());
assertEquals("(Missing) NonExistingControllerService", serviceNode.getComponentType()); assertEquals("(Missing) NonExistingControllerService", serviceNode.getComponentType());
@ -518,7 +522,7 @@ public class TestFlowController {
ProcessGroup pg = controller.createProcessGroup("my-process-group"); ProcessGroup pg = controller.createProcessGroup("my-process-group");
pg.setName("my-process-group"); pg.setName("my-process-group");
ControllerServiceNode cs = controller.createControllerService("org.apache.nifi.NonExistingControllerService", "my-controller-service", ControllerServiceNode cs = controller.createControllerService("org.apache.nifi.NonExistingControllerService", "my-controller-service",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
pg.addControllerService(cs); pg.addControllerService(cs);
controller.getRootGroup().addProcessGroup(pg); controller.getRootGroup().addProcessGroup(pg);
controller.getRootGroup().removeProcessGroup(pg); controller.getRootGroup().removeProcessGroup(pg);
@ -527,7 +531,7 @@ public class TestFlowController {
} }
@Test @Test
public void testChangeProcessorType() throws ProcessorInstantiationException { public void testReloadProcessor() throws ProcessorInstantiationException {
final String id = "1234-ScheduledProcessor" + System.currentTimeMillis(); final String id = "1234-ScheduledProcessor" + System.currentTimeMillis();
final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate(); final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate();
final ProcessorNode processorNode = controller.createProcessor(DummyScheduledProcessor.class.getName(), id, coordinate); final ProcessorNode processorNode = controller.createProcessor(DummyScheduledProcessor.class.getName(), id, coordinate);
@ -548,7 +552,7 @@ public class TestFlowController {
assertEquals(LogLevel.WARN, processorNode.getBulletinLevel()); assertEquals(LogLevel.WARN, processorNode.getBulletinLevel());
// now change the type of the processor from DummyScheduledProcessor to DummySettingsProcessor // now change the type of the processor from DummyScheduledProcessor to DummySettingsProcessor
controller.changeProcessorType(processorNode, DummySettingsProcessor.class.getName(), coordinate); controller.reload(processorNode, DummySettingsProcessor.class.getName(), coordinate, Collections.emptySet());
// ids and coordinate should stay the same // ids and coordinate should stay the same
assertEquals(id, processorNode.getIdentifier()); assertEquals(id, processorNode.getIdentifier());
@ -573,10 +577,42 @@ public class TestFlowController {
} }
@Test @Test
public void testChangeControllerServiceType() { public void testReloadProcessorWithAdditionalResources() throws ProcessorInstantiationException, MalformedURLException {
final URL resource1 = new File("src/test/resources/TestClasspathResources/resource1.txt").toURI().toURL();
final URL resource2 = new File("src/test/resources/TestClasspathResources/resource2.txt").toURI().toURL();
final URL resource3 = new File("src/test/resources/TestClasspathResources/resource3.txt").toURI().toURL();
final Set<URL> additionalUrls = new LinkedHashSet<>(Arrays.asList(resource1, resource2, resource3));
final String id = "1234-ScheduledProcessor" + System.currentTimeMillis();
final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate();
final ProcessorNode processorNode = controller.createProcessor(DummyScheduledProcessor.class.getName(), id, coordinate);
final String originalName = processorNode.getName();
// the instance class loader shouldn't have any of the resources yet
InstanceClassLoader instanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
assertNotNull(instanceClassLoader);
assertFalse(containsResource(instanceClassLoader.getURLs(), resource1));
assertFalse(containsResource(instanceClassLoader.getURLs(), resource2));
assertFalse(containsResource(instanceClassLoader.getURLs(), resource3));
assertTrue(instanceClassLoader.getAdditionalResourceUrls().isEmpty());
// now change the type of the processor from DummyScheduledProcessor to DummySettingsProcessor
controller.reload(processorNode, DummySettingsProcessor.class.getName(), coordinate, additionalUrls);
// the instance class loader shouldn't have any of the resources yet
instanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
assertNotNull(instanceClassLoader);
assertTrue(containsResource(instanceClassLoader.getURLs(), resource1));
assertTrue(containsResource(instanceClassLoader.getURLs(), resource2));
assertTrue(containsResource(instanceClassLoader.getURLs(), resource3));
assertEquals(3, instanceClassLoader.getAdditionalResourceUrls().size());
}
@Test
public void testReloadControllerService() {
final String id = "ServiceA" + System.currentTimeMillis(); final String id = "ServiceA" + System.currentTimeMillis();
final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate(); final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate();
final ControllerServiceNode controllerServiceNode = controller.createControllerService(ServiceA.class.getName(), id, coordinate, true); final ControllerServiceNode controllerServiceNode = controller.createControllerService(ServiceA.class.getName(), id, coordinate, null, true);
final String originalName = controllerServiceNode.getName(); final String originalName = controllerServiceNode.getName();
assertEquals(id, controllerServiceNode.getIdentifier()); assertEquals(id, controllerServiceNode.getIdentifier());
@ -586,7 +622,7 @@ public class TestFlowController {
assertEquals(ServiceA.class.getSimpleName(), controllerServiceNode.getComponentType()); assertEquals(ServiceA.class.getSimpleName(), controllerServiceNode.getComponentType());
assertEquals(ServiceA.class.getCanonicalName(), controllerServiceNode.getComponent().getClass().getCanonicalName()); assertEquals(ServiceA.class.getCanonicalName(), controllerServiceNode.getComponent().getClass().getCanonicalName());
controller.changeControllerServiceType(controllerServiceNode, ServiceB.class.getName(), coordinate); controller.reload(controllerServiceNode, ServiceB.class.getName(), coordinate, Collections.emptySet());
// ids and coordinate should stay the same // ids and coordinate should stay the same
assertEquals(id, controllerServiceNode.getIdentifier()); assertEquals(id, controllerServiceNode.getIdentifier());
@ -603,7 +639,38 @@ public class TestFlowController {
} }
@Test @Test
public void testChangeReportingTaskType() throws ReportingTaskInstantiationException { public void testReloadControllerServiceWithAdditionalResources() throws MalformedURLException {
final URL resource1 = new File("src/test/resources/TestClasspathResources/resource1.txt").toURI().toURL();
final URL resource2 = new File("src/test/resources/TestClasspathResources/resource2.txt").toURI().toURL();
final URL resource3 = new File("src/test/resources/TestClasspathResources/resource3.txt").toURI().toURL();
final Set<URL> additionalUrls = new LinkedHashSet<>(Arrays.asList(resource1, resource2, resource3));
final String id = "ServiceA" + System.currentTimeMillis();
final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate();
final ControllerServiceNode controllerServiceNode = controller.createControllerService(ServiceA.class.getName(), id, coordinate, null, true);
final String originalName = controllerServiceNode.getName();
// the instance class loader shouldn't have any of the resources yet
InstanceClassLoader instanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
assertNotNull(instanceClassLoader);
assertFalse(containsResource(instanceClassLoader.getURLs(), resource1));
assertFalse(containsResource(instanceClassLoader.getURLs(), resource2));
assertFalse(containsResource(instanceClassLoader.getURLs(), resource3));
assertTrue(instanceClassLoader.getAdditionalResourceUrls().isEmpty());
controller.reload(controllerServiceNode, ServiceB.class.getName(), coordinate, additionalUrls);
// the instance class loader shouldn't have any of the resources yet
instanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
assertNotNull(instanceClassLoader);
assertTrue(containsResource(instanceClassLoader.getURLs(), resource1));
assertTrue(containsResource(instanceClassLoader.getURLs(), resource2));
assertTrue(containsResource(instanceClassLoader.getURLs(), resource3));
assertEquals(3, instanceClassLoader.getAdditionalResourceUrls().size());
}
@Test
public void testReloadReportingTask() throws ReportingTaskInstantiationException {
final String id = "ReportingTask" + System.currentTimeMillis(); final String id = "ReportingTask" + System.currentTimeMillis();
final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate(); final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate();
final ReportingTaskNode node = controller.createReportingTask(DummyReportingTask.class.getName(), id, coordinate, true); final ReportingTaskNode node = controller.createReportingTask(DummyReportingTask.class.getName(), id, coordinate, true);
@ -616,7 +683,7 @@ public class TestFlowController {
assertEquals(DummyReportingTask.class.getSimpleName(), node.getComponentType()); assertEquals(DummyReportingTask.class.getSimpleName(), node.getComponentType());
assertEquals(DummyReportingTask.class.getCanonicalName(), node.getComponent().getClass().getCanonicalName()); assertEquals(DummyReportingTask.class.getCanonicalName(), node.getComponent().getClass().getCanonicalName());
controller.changeReportingTaskType(node, DummyScheduledReportingTask.class.getName(), coordinate); controller.reload(node, DummyScheduledReportingTask.class.getName(), coordinate, Collections.emptySet());
// ids and coordinate should stay the same // ids and coordinate should stay the same
assertEquals(id, node.getIdentifier()); assertEquals(id, node.getIdentifier());
@ -630,7 +697,45 @@ public class TestFlowController {
assertEquals(DummyReportingTask.class.getCanonicalName(), node.getCanonicalClassName()); assertEquals(DummyReportingTask.class.getCanonicalName(), node.getCanonicalClassName());
assertEquals(DummyReportingTask.class.getSimpleName(), node.getComponentType()); assertEquals(DummyReportingTask.class.getSimpleName(), node.getComponentType());
assertEquals(DummyScheduledReportingTask.class.getCanonicalName(), node.getComponent().getClass().getCanonicalName()); assertEquals(DummyScheduledReportingTask.class.getCanonicalName(), node.getComponent().getClass().getCanonicalName());
}
@Test
public void testReloadReportingTaskWithAdditionalResources() throws ReportingTaskInstantiationException, MalformedURLException {
final URL resource1 = new File("src/test/resources/TestClasspathResources/resource1.txt").toURI().toURL();
final URL resource2 = new File("src/test/resources/TestClasspathResources/resource2.txt").toURI().toURL();
final URL resource3 = new File("src/test/resources/TestClasspathResources/resource3.txt").toURI().toURL();
final Set<URL> additionalUrls = new LinkedHashSet<>(Arrays.asList(resource1, resource2, resource3));
final String id = "ReportingTask" + System.currentTimeMillis();
final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate();
final ReportingTaskNode node = controller.createReportingTask(DummyReportingTask.class.getName(), id, coordinate, true);
// the instance class loader shouldn't have any of the resources yet
InstanceClassLoader instanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
assertNotNull(instanceClassLoader);
assertFalse(containsResource(instanceClassLoader.getURLs(), resource1));
assertFalse(containsResource(instanceClassLoader.getURLs(), resource2));
assertFalse(containsResource(instanceClassLoader.getURLs(), resource3));
assertTrue(instanceClassLoader.getAdditionalResourceUrls().isEmpty());
controller.reload(node, DummyScheduledReportingTask.class.getName(), coordinate, additionalUrls);
// the instance class loader shouldn't have any of the resources yet
instanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
assertNotNull(instanceClassLoader);
assertTrue(containsResource(instanceClassLoader.getURLs(), resource1));
assertTrue(containsResource(instanceClassLoader.getURLs(), resource2));
assertTrue(containsResource(instanceClassLoader.getURLs(), resource3));
assertEquals(3, instanceClassLoader.getAdditionalResourceUrls().size());
}
private boolean containsResource(URL[] resources, URL resourceToFind) {
for (URL resource : resources) {
if (resourceToFind.getPath().equals(resource.getPath())) {
return true;
}
}
return false;
} }
@Test(expected = IllegalArgumentException.class) @Test(expected = IllegalArgumentException.class)
@ -744,7 +849,7 @@ public class TestFlowController {
public void testInstantiateSnippetWhenControllerServiceMissingBundle() throws ProcessorInstantiationException { public void testInstantiateSnippetWhenControllerServiceMissingBundle() throws ProcessorInstantiationException {
final String id = UUID.randomUUID().toString(); final String id = UUID.randomUUID().toString();
final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate(); final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate();
final ControllerServiceNode controllerServiceNode = controller.createControllerService(ServiceA.class.getName(), id, coordinate, true); final ControllerServiceNode controllerServiceNode = controller.createControllerService(ServiceA.class.getName(), id, coordinate, null, true);
// create the controller service dto // create the controller service dto
final ControllerServiceDTO csDto = new ControllerServiceDTO(); final ControllerServiceDTO csDto = new ControllerServiceDTO();
@ -775,7 +880,7 @@ public class TestFlowController {
public void testInstantiateSnippetWithControllerService() throws ProcessorInstantiationException { public void testInstantiateSnippetWithControllerService() throws ProcessorInstantiationException {
final String id = UUID.randomUUID().toString(); final String id = UUID.randomUUID().toString();
final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate(); final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate();
final ControllerServiceNode controllerServiceNode = controller.createControllerService(ServiceA.class.getName(), id, coordinate, true); final ControllerServiceNode controllerServiceNode = controller.createControllerService(ServiceA.class.getName(), id, coordinate, null, true);
// create the controller service dto // create the controller service dto
final ControllerServiceDTO csDto = new ControllerServiceDTO(); final ControllerServiceDTO csDto = new ControllerServiceDTO();

View File

@ -25,12 +25,16 @@ import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue; import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.controller.exception.ControllerServiceInstantiationException;
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.engine.FlowEngine; import org.apache.nifi.engine.FlowEngine;
import org.apache.nifi.expression.ExpressionLanguageCompiler; import org.apache.nifi.expression.ExpressionLanguageCompiler;
import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.InstanceClassLoader;
import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.nar.NarCloseable;
import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSession;
@ -55,10 +59,10 @@ import org.mockito.Mockito;
import java.io.File; import java.io.File;
import java.net.MalformedURLException; import java.net.MalformedURLException;
import java.net.URL; import java.net.URL;
import java.net.URLClassLoader;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.LinkedHashSet;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.UUID; import java.util.UUID;
@ -88,11 +92,12 @@ public class TestStandardProcessorNode {
ProcessorInitializationContext initContext = new StandardProcessorInitializationContext(uuid, null, null, null, null); ProcessorInitializationContext initContext = new StandardProcessorInitializationContext(uuid, null, null, null, null);
processor.initialize(initContext); processor.initialize(initContext);
final ReloadComponent reloadComponent = Mockito.mock(ReloadComponent.class);
final BundleCoordinate coordinate = Mockito.mock(BundleCoordinate.class); final BundleCoordinate coordinate = Mockito.mock(BundleCoordinate.class);
final LoggableComponent<Processor> loggableComponent = new LoggableComponent<>(processor, coordinate, null); final LoggableComponent<Processor> loggableComponent = new LoggableComponent<>(processor, coordinate, null);
final StandardProcessorNode procNode = new StandardProcessorNode(loggableComponent, uuid, createValidationContextFactory(), null, null, final StandardProcessorNode procNode = new StandardProcessorNode(loggableComponent, uuid, createValidationContextFactory(), null, null,
NiFiProperties.createBasicNiFiProperties(null, null), VariableRegistry.EMPTY_REGISTRY); NiFiProperties.createBasicNiFiProperties(null, null), VariableRegistry.EMPTY_REGISTRY, reloadComponent);
final ScheduledExecutorService taskScheduler = new FlowEngine(2, "TestClasspathResources", true); final ScheduledExecutorService taskScheduler = new FlowEngine(2, "TestClasspathResources", true);
final StandardProcessContext processContext = new StandardProcessContext(procNode, null, null, null, null); final StandardProcessContext processContext = new StandardProcessContext(procNode, null, null, null, null);
@ -122,8 +127,9 @@ public class TestStandardProcessorNode {
@Test @Test
public void testDisabledValidationErrors() { public void testDisabledValidationErrors() {
final MockReloadComponent reloadComponent = new MockReloadComponent();
final ModifiesClasspathNoAnnotationProcessor processor = new ModifiesClasspathNoAnnotationProcessor(); final ModifiesClasspathNoAnnotationProcessor processor = new ModifiesClasspathNoAnnotationProcessor();
final StandardProcessorNode procNode = createProcessorNode(processor); final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
// Set a property to an invalid value // Set a property to an invalid value
final Map<String, String> properties = new HashMap<>(); final Map<String, String> properties = new HashMap<>();
@ -138,22 +144,19 @@ public class TestStandardProcessorNode {
@Test @Test
public void testSinglePropertyDynamicallyModifiesClasspath() throws MalformedURLException { public void testSinglePropertyDynamicallyModifiesClasspath() throws MalformedURLException {
final MockReloadComponent reloadComponent = new MockReloadComponent();
final PropertyDescriptor classpathProp = new PropertyDescriptor.Builder().name("Classpath Resources") final PropertyDescriptor classpathProp = new PropertyDescriptor.Builder().name("Classpath Resources")
.dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build(); .dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
final ModifiesClasspathProcessor processor = new ModifiesClasspathProcessor(Arrays.asList(classpathProp)); final ModifiesClasspathProcessor processor = new ModifiesClasspathProcessor(Arrays.asList(classpathProp));
final StandardProcessorNode procNode = createProcessorNode(processor); final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){ try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){
// Should have an InstanceClassLoader here
final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader();
assertTrue(contextClassLoader instanceof InstanceClassLoader);
final InstanceClassLoader instanceClassLoader = (InstanceClassLoader) contextClassLoader;
// Should not have any of the test resources loaded at this point // Should not have any of the test resources loaded at this point
final URL[] testResources = getTestResources(); final URL[] testResources = getTestResources();
for (URL testResource : testResources) { for (URL testResource : testResources) {
if (containsResource(instanceClassLoader.getInstanceResources(), testResource)) { if (containsResource(reloadComponent.getAdditionalUrls(), testResource)) {
fail("found resource that should not have been loaded"); fail("found resource that should not have been loaded");
} }
} }
@ -165,18 +168,22 @@ public class TestStandardProcessorNode {
// Should have all of the resources loaded into the InstanceClassLoader now // Should have all of the resources loaded into the InstanceClassLoader now
for (URL testResource : testResources) { for (URL testResource : testResources) {
assertTrue(containsResource(instanceClassLoader.getInstanceResources(), testResource)); assertTrue(containsResource(reloadComponent.getAdditionalUrls(), testResource));
} }
assertEquals(ModifiesClasspathProcessor.class.getCanonicalName(), reloadComponent.getNewType());
// Should pass validation // Should pass validation
assertTrue(procNode.isValid()); assertTrue(procNode.isValid());
} finally { } finally {
ExtensionManager.removeInstanceClassLoaderIfExists(procNode.getIdentifier()); ExtensionManager.removeInstanceClassLoader(procNode.getIdentifier());
} }
} }
@Test @Test
public void testUpdateOtherPropertyDoesNotImpactClasspath() throws MalformedURLException { public void testUpdateOtherPropertyDoesNotImpactClasspath() throws MalformedURLException {
final MockReloadComponent reloadComponent = new MockReloadComponent();
final PropertyDescriptor classpathProp = new PropertyDescriptor.Builder().name("Classpath Resources") final PropertyDescriptor classpathProp = new PropertyDescriptor.Builder().name("Classpath Resources")
.dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build(); .dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
@ -184,19 +191,13 @@ public class TestStandardProcessorNode {
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build(); .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
final ModifiesClasspathProcessor processor = new ModifiesClasspathProcessor(Arrays.asList(classpathProp, otherProp)); final ModifiesClasspathProcessor processor = new ModifiesClasspathProcessor(Arrays.asList(classpathProp, otherProp));
final StandardProcessorNode procNode = createProcessorNode(processor); final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){ try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){
// Should have an InstanceClassLoader here
final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader();
assertTrue(contextClassLoader instanceof InstanceClassLoader);
final InstanceClassLoader instanceClassLoader = (InstanceClassLoader) contextClassLoader;
// Should not have any of the test resources loaded at this point // Should not have any of the test resources loaded at this point
final URL[] testResources = getTestResources(); final URL[] testResources = getTestResources();
for (URL testResource : testResources) { for (URL testResource : testResources) {
if (containsResource(instanceClassLoader.getInstanceResources(), testResource)) { if (containsResource(reloadComponent.getAdditionalUrls(), testResource)) {
fail("found resource that should not have been loaded"); fail("found resource that should not have been loaded");
} }
} }
@ -208,7 +209,7 @@ public class TestStandardProcessorNode {
// Should have all of the resources loaded into the InstanceClassLoader now // Should have all of the resources loaded into the InstanceClassLoader now
for (URL testResource : testResources) { for (URL testResource : testResources) {
assertTrue(containsResource(instanceClassLoader.getInstanceResources(), testResource)); assertTrue(containsResource(reloadComponent.getAdditionalUrls(), testResource));
} }
// Should pass validation // Should pass validation
@ -221,7 +222,7 @@ public class TestStandardProcessorNode {
// Should STILL have all of the resources loaded into the InstanceClassLoader now // Should STILL have all of the resources loaded into the InstanceClassLoader now
for (URL testResource : testResources) { for (URL testResource : testResources) {
assertTrue(containsResource(instanceClassLoader.getInstanceResources(), testResource)); assertTrue(containsResource(reloadComponent.getAdditionalUrls(), testResource));
} }
// Should STILL pass validation // Should STILL pass validation
@ -233,38 +234,37 @@ public class TestStandardProcessorNode {
procNode.setProperties(newClasspathProperties); procNode.setProperties(newClasspathProperties);
// Should only have resource1 loaded now // Should only have resource1 loaded now
assertTrue(containsResource(instanceClassLoader.getInstanceResources(), testResources[0])); assertTrue(containsResource(reloadComponent.getAdditionalUrls(), testResources[0]));
assertFalse(containsResource(instanceClassLoader.getInstanceResources(), testResources[1])); assertFalse(containsResource(reloadComponent.getAdditionalUrls(), testResources[1]));
assertFalse(containsResource(instanceClassLoader.getInstanceResources(), testResources[2])); assertFalse(containsResource(reloadComponent.getAdditionalUrls(), testResources[2]));
assertEquals(ModifiesClasspathProcessor.class.getCanonicalName(), reloadComponent.getNewType());
// Should STILL pass validation // Should STILL pass validation
assertTrue(procNode.isValid()); assertTrue(procNode.isValid());
} finally { } finally {
ExtensionManager.removeInstanceClassLoaderIfExists(procNode.getIdentifier()); ExtensionManager.removeInstanceClassLoader(procNode.getIdentifier());
} }
} }
@Test @Test
public void testMultiplePropertiesDynamicallyModifyClasspathWithExpressionLanguage() throws MalformedURLException { public void testMultiplePropertiesDynamicallyModifyClasspathWithExpressionLanguage() throws MalformedURLException {
final MockReloadComponent reloadComponent = new MockReloadComponent();
final PropertyDescriptor classpathProp1 = new PropertyDescriptor.Builder().name("Classpath Resource 1") final PropertyDescriptor classpathProp1 = new PropertyDescriptor.Builder().name("Classpath Resource 1")
.dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build(); .dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
final PropertyDescriptor classpathProp2 = new PropertyDescriptor.Builder().name("Classpath Resource 2") final PropertyDescriptor classpathProp2 = new PropertyDescriptor.Builder().name("Classpath Resource 2")
.dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build(); .dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
final ModifiesClasspathProcessor processor = new ModifiesClasspathProcessor(Arrays.asList(classpathProp1, classpathProp2)); final ModifiesClasspathProcessor processor = new ModifiesClasspathProcessor(Arrays.asList(classpathProp1, classpathProp2));
final StandardProcessorNode procNode = createProcessorNode(processor); final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){ try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){
// Should have an InstanceClassLoader here
final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader();
assertTrue(contextClassLoader instanceof InstanceClassLoader);
final InstanceClassLoader instanceClassLoader = (InstanceClassLoader) contextClassLoader;
// Should not have any of the test resources loaded at this point // Should not have any of the test resources loaded at this point
final URL[] testResources = getTestResources(); final URL[] testResources = getTestResources();
for (URL testResource : testResources) { for (URL testResource : testResources) {
if (containsResource(instanceClassLoader.getInstanceResources(), testResource)) { if (containsResource(reloadComponent.getAdditionalUrls(), testResource)) {
fail("found resource that should not have been loaded"); fail("found resource that should not have been loaded");
} }
} }
@ -279,38 +279,37 @@ public class TestStandardProcessorNode {
procNode.setProperties(properties); procNode.setProperties(properties);
// Should have resources 1 and 3 loaded into the InstanceClassLoader now // Should have resources 1 and 3 loaded into the InstanceClassLoader now
assertTrue(containsResource(instanceClassLoader.getInstanceResources(), testResources[0])); assertTrue(containsResource(reloadComponent.getAdditionalUrls(), testResources[0]));
assertTrue(containsResource(instanceClassLoader.getInstanceResources(), testResources[2])); assertTrue(containsResource(reloadComponent.getAdditionalUrls(), testResources[2]));
assertFalse(containsResource(instanceClassLoader.getInstanceResources(), testResources[1])); assertFalse(containsResource(reloadComponent.getAdditionalUrls(), testResources[1]));
assertEquals(ModifiesClasspathProcessor.class.getCanonicalName(), reloadComponent.getNewType());
// Should pass validation // Should pass validation
assertTrue(procNode.isValid()); assertTrue(procNode.isValid());
} finally { } finally {
ExtensionManager.removeInstanceClassLoaderIfExists(procNode.getIdentifier()); ExtensionManager.removeInstanceClassLoader(procNode.getIdentifier());
} }
} }
@Test @Test
public void testSomeNonExistentPropertiesDynamicallyModifyClasspath() throws MalformedURLException { public void testSomeNonExistentPropertiesDynamicallyModifyClasspath() throws MalformedURLException {
final MockReloadComponent reloadComponent = new MockReloadComponent();
final PropertyDescriptor classpathProp1 = new PropertyDescriptor.Builder().name("Classpath Resource 1") final PropertyDescriptor classpathProp1 = new PropertyDescriptor.Builder().name("Classpath Resource 1")
.dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build(); .dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
final PropertyDescriptor classpathProp2 = new PropertyDescriptor.Builder().name("Classpath Resource 2") final PropertyDescriptor classpathProp2 = new PropertyDescriptor.Builder().name("Classpath Resource 2")
.dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build(); .dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
final ModifiesClasspathProcessor processor = new ModifiesClasspathProcessor(Arrays.asList(classpathProp1, classpathProp2)); final ModifiesClasspathProcessor processor = new ModifiesClasspathProcessor(Arrays.asList(classpathProp1, classpathProp2));
final StandardProcessorNode procNode = createProcessorNode(processor); final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){ try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){
// Should have an InstanceClassLoader here
final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader();
assertTrue(contextClassLoader instanceof InstanceClassLoader);
final InstanceClassLoader instanceClassLoader = (InstanceClassLoader) contextClassLoader;
// Should not have any of the test resources loaded at this point // Should not have any of the test resources loaded at this point
final URL[] testResources = getTestResources(); final URL[] testResources = getTestResources();
for (URL testResource : testResources) { for (URL testResource : testResources) {
if (containsResource(instanceClassLoader.getInstanceResources(), testResource)) { if (containsResource(reloadComponent.getAdditionalUrls(), testResource)) {
fail("found resource that should not have been loaded"); fail("found resource that should not have been loaded");
} }
} }
@ -322,53 +321,52 @@ public class TestStandardProcessorNode {
procNode.setProperties(properties); procNode.setProperties(properties);
// Should have resources 1 and 3 loaded into the InstanceClassLoader now // Should have resources 1 and 3 loaded into the InstanceClassLoader now
assertTrue(containsResource(instanceClassLoader.getInstanceResources(), testResources[0])); assertTrue(containsResource(reloadComponent.getAdditionalUrls(), testResources[0]));
assertFalse(containsResource(instanceClassLoader.getInstanceResources(), testResources[1])); assertFalse(containsResource(reloadComponent.getAdditionalUrls(), testResources[1]));
assertFalse(containsResource(instanceClassLoader.getInstanceResources(), testResources[2])); assertFalse(containsResource(reloadComponent.getAdditionalUrls(), testResources[2]));
assertEquals(ModifiesClasspathProcessor.class.getCanonicalName(), reloadComponent.getNewType());
// Should pass validation // Should pass validation
assertTrue(procNode.isValid()); assertTrue(procNode.isValid());
} finally { } finally {
ExtensionManager.removeInstanceClassLoaderIfExists(procNode.getIdentifier()); ExtensionManager.removeInstanceClassLoader(procNode.getIdentifier());
} }
} }
@Test @Test
public void testPropertyModifiesClasspathWhenProcessorMissingAnnotation() throws MalformedURLException { public void testPropertyModifiesClasspathWhenProcessorMissingAnnotation() throws MalformedURLException {
final MockReloadComponent reloadComponent = new MockReloadComponent();
final ModifiesClasspathNoAnnotationProcessor processor = new ModifiesClasspathNoAnnotationProcessor(); final ModifiesClasspathNoAnnotationProcessor processor = new ModifiesClasspathNoAnnotationProcessor();
final StandardProcessorNode procNode = createProcessorNode(processor); final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){ try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){
// Can't validate the ClassLoader here b/c the class is missing the annotation
// Simulate setting the properties pointing to two of the resources
final Map<String, String> properties = new HashMap<>(); final Map<String, String> properties = new HashMap<>();
properties.put(ModifiesClasspathNoAnnotationProcessor.CLASSPATH_RESOURCE.getName(), properties.put(ModifiesClasspathNoAnnotationProcessor.CLASSPATH_RESOURCE.getName(),
"src/test/resources/TestClasspathResources/resource1.txt"); "src/test/resources/TestClasspathResources/resource1.txt");
procNode.setProperties(properties); procNode.setProperties(properties);
// Should not have loaded any of the resources
final ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
assertTrue(classLoader instanceof URLClassLoader);
final URL[] testResources = getTestResources(); final URL[] testResources = getTestResources();
final URLClassLoader urlClassLoader = (URLClassLoader) classLoader; assertTrue(containsResource(reloadComponent.getAdditionalUrls(), testResources[0]));
assertFalse(containsResource(urlClassLoader.getURLs(), testResources[0])); assertFalse(containsResource(reloadComponent.getAdditionalUrls(), testResources[1]));
assertFalse(containsResource(urlClassLoader.getURLs(), testResources[1])); assertFalse(containsResource(reloadComponent.getAdditionalUrls(), testResources[2]));
assertFalse(containsResource(urlClassLoader.getURLs(), testResources[2]));
assertEquals(ModifiesClasspathNoAnnotationProcessor.class.getCanonicalName(), reloadComponent.getNewType());
// Should pass validation // Should pass validation
assertTrue(procNode.isValid()); assertTrue(procNode.isValid());
} finally { } finally {
ExtensionManager.removeInstanceClassLoaderIfExists(procNode.getIdentifier()); ExtensionManager.removeInstanceClassLoader(procNode.getIdentifier());
} }
} }
@Test @Test
public void testVerifyCanUpdateBundle() { public void testVerifyCanUpdateBundle() {
final ReloadComponent reloadComponent = new MockReloadComponent();
final ModifiesClasspathNoAnnotationProcessor processor = new ModifiesClasspathNoAnnotationProcessor(); final ModifiesClasspathNoAnnotationProcessor processor = new ModifiesClasspathNoAnnotationProcessor();
final StandardProcessorNode procNode = createProcessorNode(processor); final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
final BundleCoordinate existingCoordinate = procNode.getBundleCoordinate(); final BundleCoordinate existingCoordinate = procNode.getBundleCoordinate();
// should be allowed to update when the bundle is the same // should be allowed to update when the bundle is the same
@ -400,30 +398,68 @@ public class TestStandardProcessorNode {
} }
} }
@Test private StandardProcessorNode createProcessorNode(final Processor processor, final ReloadComponent reloadComponent) {
public void testValidateControllerServiceApiRequired() {
}
private StandardProcessorNode createProcessorNode(Processor processor) {
final String uuid = UUID.randomUUID().toString(); final String uuid = UUID.randomUUID().toString();
final ValidationContextFactory validationContextFactory = createValidationContextFactory(); final ValidationContextFactory validationContextFactory = createValidationContextFactory();
final NiFiProperties niFiProperties = NiFiProperties.createBasicNiFiProperties("src/test/resources/conf/nifi.properties", null); final NiFiProperties niFiProperties = NiFiProperties.createBasicNiFiProperties("src/test/resources/conf/nifi.properties", null);
final ProcessScheduler processScheduler = Mockito.mock(ProcessScheduler.class); final ProcessScheduler processScheduler = Mockito.mock(ProcessScheduler.class);
final ComponentLog componentLog = Mockito.mock(ComponentLog.class); final ComponentLog componentLog = Mockito.mock(ComponentLog.class);
final Bundle systemBundle = ExtensionManager.createSystemBundle(niFiProperties); final Bundle systemBundle = SystemBundle.create(niFiProperties);
ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet()); ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
ExtensionManager.createInstanceClassLoader(processor.getClass().getName(), uuid, systemBundle); ExtensionManager.createInstanceClassLoader(processor.getClass().getName(), uuid, systemBundle, null);
ProcessorInitializationContext initContext = new StandardProcessorInitializationContext(uuid, componentLog, null, null, null); ProcessorInitializationContext initContext = new StandardProcessorInitializationContext(uuid, componentLog, null, null, null);
processor.initialize(initContext); processor.initialize(initContext);
final LoggableComponent<Processor> loggableComponent = new LoggableComponent<>(processor, systemBundle.getBundleDetails().getCoordinate(), componentLog); final LoggableComponent<Processor> loggableComponent = new LoggableComponent<>(processor, systemBundle.getBundleDetails().getCoordinate(), componentLog);
return new StandardProcessorNode(loggableComponent, uuid, validationContextFactory, processScheduler, null, niFiProperties, variableRegistry); return new StandardProcessorNode(loggableComponent, uuid, validationContextFactory, processScheduler, null, niFiProperties, variableRegistry, reloadComponent);
} }
private boolean containsResource(URL[] resources, URL resourceToFind) { private static class MockReloadComponent implements ReloadComponent {
private String newType;
private BundleCoordinate bundleCoordinate;
private final Set<URL> additionalUrls = new LinkedHashSet<>();
public Set<URL> getAdditionalUrls() {
return this.additionalUrls;
}
public String getNewType() {
return newType;
}
public BundleCoordinate getBundleCoordinate() {
return bundleCoordinate;
}
@Override
public void reload(ProcessorNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls) throws ProcessorInstantiationException {
reload(newType, bundleCoordinate, additionalUrls);
}
@Override
public void reload(ControllerServiceNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls) throws ControllerServiceInstantiationException {
reload(newType, bundleCoordinate, additionalUrls);
}
@Override
public void reload(ReportingTaskNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls) throws ReportingTaskInstantiationException {
reload(newType, bundleCoordinate, additionalUrls);
}
private void reload(String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls) {
this.newType = newType;
this.bundleCoordinate = bundleCoordinate;
this.additionalUrls.clear();
if (additionalUrls != null) {
this.additionalUrls.addAll(additionalUrls);
}
}
}
private boolean containsResource(Set<URL> resources, URL resourceToFind) {
for (URL resource : resources) { for (URL resource : resources) {
if (resourceToFind.getPath().equals(resource.getPath())) { if (resourceToFind.getPath().equals(resource.getPath())) {
return true; return true;

View File

@ -39,6 +39,7 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.events.VolatileBulletinRepository; import org.apache.nifi.events.VolatileBulletinRepository;
import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSession;
@ -542,7 +543,7 @@ public class TestProcessorLifecycle {
this.setControllerRootGroup(fc, testGroup); this.setControllerRootGroup(fc, testGroup);
ControllerServiceNode testServiceNode = fc.createControllerService(TestService.class.getName(), "serv", ControllerServiceNode testServiceNode = fc.createControllerService(TestService.class.getName(), "serv",
fcsb.getSystemBundle().getBundleDetails().getCoordinate(), true); fcsb.getSystemBundle().getBundleDetails().getCoordinate(), null, true);
ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString(), ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString(),
fcsb.getSystemBundle().getBundleDetails().getCoordinate()); fcsb.getSystemBundle().getBundleDetails().getCoordinate());
@ -569,7 +570,7 @@ public class TestProcessorLifecycle {
this.setControllerRootGroup(fc, testGroup); this.setControllerRootGroup(fc, testGroup);
ControllerServiceNode testServiceNode = fc.createControllerService(TestService.class.getName(), "foo", ControllerServiceNode testServiceNode = fc.createControllerService(TestService.class.getName(), "foo",
fcsb.getSystemBundle().getBundleDetails().getCoordinate(), true); fcsb.getSystemBundle().getBundleDetails().getCoordinate(), null, true);
testGroup.addControllerService(testServiceNode); testGroup.addControllerService(testServiceNode);
ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString(), ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString(),
@ -731,7 +732,7 @@ public class TestProcessorLifecycle {
} }
final NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(null, addProps); final NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(null, addProps);
final Bundle systemBundle = ExtensionManager.createSystemBundle(nifiProperties); final Bundle systemBundle = SystemBundle.create(nifiProperties);
ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet()); ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
final FlowController flowController = FlowController.createStandaloneInstance(mock(FlowFileEventRepository.class), nifiProperties, final FlowController flowController = FlowController.createStandaloneInstance(mock(FlowFileEventRepository.class), nifiProperties,

View File

@ -28,6 +28,7 @@ import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.LoggableComponent; import org.apache.nifi.controller.LoggableComponent;
import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.controller.ProcessScheduler;
import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.ReloadComponent;
import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.StandardProcessorNode; import org.apache.nifi.controller.StandardProcessorNode;
import org.apache.nifi.controller.ValidationContextFactory; import org.apache.nifi.controller.ValidationContextFactory;
@ -43,6 +44,7 @@ import org.apache.nifi.controller.service.mock.MockProcessGroup;
import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSession;
@ -98,7 +100,7 @@ public class TestStandardProcessScheduler {
this.nifiProperties = NiFiProperties.createBasicNiFiProperties(null, null); this.nifiProperties = NiFiProperties.createBasicNiFiProperties(null, null);
// load the system bundle // load the system bundle
systemBundle = ExtensionManager.createSystemBundle(nifiProperties); systemBundle = SystemBundle.create(nifiProperties);
ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet()); ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
scheduler = new StandardProcessScheduler(Mockito.mock(ControllerServiceProvider.class), null, stateMgrProvider, variableRegistry, nifiProperties); scheduler = new StandardProcessScheduler(Mockito.mock(ControllerServiceProvider.class), null, stateMgrProvider, variableRegistry, nifiProperties);
@ -111,8 +113,9 @@ public class TestStandardProcessScheduler {
final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(null, variableRegistry); final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(null, variableRegistry);
final ComponentLog logger = Mockito.mock(ComponentLog.class); final ComponentLog logger = Mockito.mock(ComponentLog.class);
final ReloadComponent reloadComponent = Mockito.mock(ReloadComponent.class);
final LoggableComponent<ReportingTask> loggableComponent = new LoggableComponent<>(reportingTask, systemBundle.getBundleDetails().getCoordinate(), logger); final LoggableComponent<ReportingTask> loggableComponent = new LoggableComponent<>(reportingTask, systemBundle.getBundleDetails().getCoordinate(), logger);
taskNode = new StandardReportingTaskNode(loggableComponent, UUID.randomUUID().toString(), null, scheduler, validationContextFactory, variableRegistry); taskNode = new StandardReportingTaskNode(loggableComponent, UUID.randomUUID().toString(), null, scheduler, validationContextFactory, variableRegistry, reloadComponent);
controller = Mockito.mock(FlowController.class); controller = Mockito.mock(FlowController.class);
rootGroup = new MockProcessGroup(); rootGroup = new MockProcessGroup();
@ -150,16 +153,18 @@ public class TestStandardProcessScheduler {
final Processor proc = new ServiceReferencingProcessor(); final Processor proc = new ServiceReferencingProcessor();
proc.initialize(new StandardProcessorInitializationContext(uuid, null, null, null, null)); proc.initialize(new StandardProcessorInitializationContext(uuid, null, null, null, null));
final ReloadComponent reloadComponent = Mockito.mock(ReloadComponent.class);
final StandardControllerServiceProvider serviceProvider = final StandardControllerServiceProvider serviceProvider =
new StandardControllerServiceProvider(controller, scheduler, null, Mockito.mock(StateManagerProvider.class), variableRegistry, nifiProperties); new StandardControllerServiceProvider(controller, scheduler, null, Mockito.mock(StateManagerProvider.class), variableRegistry, nifiProperties);
final ControllerServiceNode service = serviceProvider.createControllerService(NoStartServiceImpl.class.getName(), "service", final ControllerServiceNode service = serviceProvider.createControllerService(NoStartServiceImpl.class.getName(), "service",
systemBundle.getBundleDetails().getCoordinate(), true); systemBundle.getBundleDetails().getCoordinate(), null, true);
rootGroup.addControllerService(service); rootGroup.addControllerService(service);
final LoggableComponent<Processor> loggableComponent = new LoggableComponent<>(proc, systemBundle.getBundleDetails().getCoordinate(), null); final LoggableComponent<Processor> loggableComponent = new LoggableComponent<>(proc, systemBundle.getBundleDetails().getCoordinate(), null);
final ProcessorNode procNode = new StandardProcessorNode(loggableComponent, uuid, final ProcessorNode procNode = new StandardProcessorNode(loggableComponent, uuid,
new StandardValidationContextFactory(serviceProvider, variableRegistry), new StandardValidationContextFactory(serviceProvider, variableRegistry),
scheduler, serviceProvider, nifiProperties, VariableRegistry.EMPTY_REGISTRY); scheduler, serviceProvider, nifiProperties, VariableRegistry.EMPTY_REGISTRY, reloadComponent);
rootGroup.addProcessor(procNode); rootGroup.addProcessor(procNode);
Map<String,String> procProps = new HashMap<>(); Map<String,String> procProps = new HashMap<>();
@ -233,7 +238,7 @@ public class TestStandardProcessScheduler {
final ProcessScheduler scheduler = createScheduler(); final ProcessScheduler scheduler = createScheduler();
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties); final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(), final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(),
"1", systemBundle.getBundleDetails().getCoordinate(), false); "1", systemBundle.getBundleDetails().getCoordinate(), null, false);
assertFalse(serviceNode.isActive()); assertFalse(serviceNode.isActive());
final SimpleTestService ts = (SimpleTestService) serviceNode.getControllerServiceImplementation(); final SimpleTestService ts = (SimpleTestService) serviceNode.getControllerServiceImplementation();
final ExecutorService executor = Executors.newCachedThreadPool(); final ExecutorService executor = Executors.newCachedThreadPool();
@ -272,7 +277,7 @@ public class TestStandardProcessScheduler {
final ProcessScheduler scheduler = createScheduler(); final ProcessScheduler scheduler = createScheduler();
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties); final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(), final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(),
"1", systemBundle.getBundleDetails().getCoordinate(), false); "1", systemBundle.getBundleDetails().getCoordinate(), null, false);
final SimpleTestService ts = (SimpleTestService) serviceNode.getControllerServiceImplementation(); final SimpleTestService ts = (SimpleTestService) serviceNode.getControllerServiceImplementation();
final ExecutorService executor = Executors.newCachedThreadPool(); final ExecutorService executor = Executors.newCachedThreadPool();
@ -310,7 +315,7 @@ public class TestStandardProcessScheduler {
final ProcessScheduler scheduler = createScheduler(); final ProcessScheduler scheduler = createScheduler();
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties); final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(), final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(),
"1", systemBundle.getBundleDetails().getCoordinate(), false); "1", systemBundle.getBundleDetails().getCoordinate(), null, false);
final SimpleTestService ts = (SimpleTestService) serviceNode.getControllerServiceImplementation(); final SimpleTestService ts = (SimpleTestService) serviceNode.getControllerServiceImplementation();
scheduler.enableControllerService(serviceNode); scheduler.enableControllerService(serviceNode);
assertTrue(serviceNode.isActive()); assertTrue(serviceNode.isActive());
@ -344,7 +349,7 @@ public class TestStandardProcessScheduler {
final ProcessScheduler scheduler = createScheduler(); final ProcessScheduler scheduler = createScheduler();
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties); final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
final ControllerServiceNode serviceNode = provider.createControllerService(FailingService.class.getName(), final ControllerServiceNode serviceNode = provider.createControllerService(FailingService.class.getName(),
"1", systemBundle.getBundleDetails().getCoordinate(), false); "1", systemBundle.getBundleDetails().getCoordinate(), null, false);
scheduler.enableControllerService(serviceNode); scheduler.enableControllerService(serviceNode);
Thread.sleep(1000); Thread.sleep(1000);
scheduler.shutdown(); scheduler.shutdown();
@ -378,7 +383,7 @@ public class TestStandardProcessScheduler {
final ExecutorService executor = Executors.newCachedThreadPool(); final ExecutorService executor = Executors.newCachedThreadPool();
for (int i = 0; i < 200; i++) { for (int i = 0; i < 200; i++) {
final ControllerServiceNode serviceNode = provider.createControllerService(RandomShortDelayEnablingService.class.getName(), "1", final ControllerServiceNode serviceNode = provider.createControllerService(RandomShortDelayEnablingService.class.getName(), "1",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
executor.execute(new Runnable() { executor.execute(new Runnable() {
@Override @Override
@ -419,7 +424,7 @@ public class TestStandardProcessScheduler {
final ProcessScheduler scheduler = createScheduler(); final ProcessScheduler scheduler = createScheduler();
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties); final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
final ControllerServiceNode serviceNode = provider.createControllerService(LongEnablingService.class.getName(), final ControllerServiceNode serviceNode = provider.createControllerService(LongEnablingService.class.getName(),
"1", systemBundle.getBundleDetails().getCoordinate(), false); "1", systemBundle.getBundleDetails().getCoordinate(), null, false);
final LongEnablingService ts = (LongEnablingService) serviceNode.getControllerServiceImplementation(); final LongEnablingService ts = (LongEnablingService) serviceNode.getControllerServiceImplementation();
ts.setLimit(Long.MAX_VALUE); ts.setLimit(Long.MAX_VALUE);
scheduler.enableControllerService(serviceNode); scheduler.enableControllerService(serviceNode);
@ -445,7 +450,7 @@ public class TestStandardProcessScheduler {
final ProcessScheduler scheduler = createScheduler(); final ProcessScheduler scheduler = createScheduler();
final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties); final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
final ControllerServiceNode serviceNode = provider.createControllerService(LongEnablingService.class.getName(), final ControllerServiceNode serviceNode = provider.createControllerService(LongEnablingService.class.getName(),
"1", systemBundle.getBundleDetails().getCoordinate(), false); "1", systemBundle.getBundleDetails().getCoordinate(), null, false);
final LongEnablingService ts = (LongEnablingService) serviceNode.getControllerServiceImplementation(); final LongEnablingService ts = (LongEnablingService) serviceNode.getControllerServiceImplementation();
ts.setLimit(3000); ts.setLimit(3000);
scheduler.enableControllerService(serviceNode); scheduler.enableControllerService(serviceNode);

View File

@ -22,6 +22,7 @@ import org.apache.nifi.components.state.StateManagerProvider;
import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.NarClassLoaders; import org.apache.nifi.nar.NarClassLoaders;
import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.registry.VariableRegistry; import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.reporting.InitializationException; import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.util.FileBasedVariableRegistry; import org.apache.nifi.util.FileBasedVariableRegistry;
@ -48,7 +49,7 @@ public class StandardControllerServiceProviderTest {
NarClassLoaders.getInstance().init(nifiProperties.getFrameworkWorkingDirectory(), nifiProperties.getExtensionsWorkingDirectory()); NarClassLoaders.getInstance().init(nifiProperties.getFrameworkWorkingDirectory(), nifiProperties.getExtensionsWorkingDirectory());
// load the system bundle // load the system bundle
systemBundle = ExtensionManager.createSystemBundle(nifiProperties); systemBundle = SystemBundle.create(nifiProperties);
ExtensionManager.discoverExtensions(systemBundle, NarClassLoaders.getInstance().getBundles()); ExtensionManager.discoverExtensions(systemBundle, NarClassLoaders.getInstance().getBundles());
variableRegistry = new FileBasedVariableRegistry(nifiProperties.getVariableRegistryPropertiesPaths()); variableRegistry = new FileBasedVariableRegistry(nifiProperties.getVariableRegistryPropertiesPaths());
@ -80,7 +81,7 @@ public class StandardControllerServiceProviderTest {
public void onComponentRemoved(String componentId) { public void onComponentRemoved(String componentId) {
} }
}, variableRegistry, nifiProperties); }, variableRegistry, nifiProperties);
ControllerServiceNode node = provider.createControllerService(clazz, id, systemBundle.getBundleDetails().getCoordinate(), true); ControllerServiceNode node = provider.createControllerService(clazz, id, systemBundle.getBundleDetails().getCoordinate(), null, true);
proxied = node.getProxiedControllerService(); proxied = node.getProxiedControllerService();
implementation = node.getControllerServiceImplementation(); implementation = node.getControllerServiceImplementation();
} }

View File

@ -24,6 +24,7 @@ import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.LoggableComponent; import org.apache.nifi.controller.LoggableComponent;
import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.controller.ProcessScheduler;
import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.ReloadComponent;
import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.StandardProcessorNode; import org.apache.nifi.controller.StandardProcessorNode;
import org.apache.nifi.controller.scheduling.StandardProcessScheduler; import org.apache.nifi.controller.scheduling.StandardProcessScheduler;
@ -35,6 +36,7 @@ import org.apache.nifi.controller.service.mock.ServiceC;
import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.StandardProcessGroup; import org.apache.nifi.groups.StandardProcessGroup;
import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.processor.Processor; import org.apache.nifi.processor.Processor;
import org.apache.nifi.processor.StandardValidationContextFactory; import org.apache.nifi.processor.StandardValidationContextFactory;
import org.apache.nifi.registry.VariableRegistry; import org.apache.nifi.registry.VariableRegistry;
@ -93,7 +95,7 @@ public class TestStandardControllerServiceProvider {
niFiProperties = NiFiProperties.createBasicNiFiProperties(null, null); niFiProperties = NiFiProperties.createBasicNiFiProperties(null, null);
// load the system bundle // load the system bundle
systemBundle = ExtensionManager.createSystemBundle(niFiProperties); systemBundle = SystemBundle.create(niFiProperties);
ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet()); ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
} }
@ -118,7 +120,7 @@ public class TestStandardControllerServiceProvider {
new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider, variableRegistry, niFiProperties); new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider, variableRegistry, niFiProperties);
final ControllerServiceNode serviceNode = provider.createControllerService(ServiceB.class.getName(), "B", final ControllerServiceNode serviceNode = provider.createControllerService(ServiceB.class.getName(), "B",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null,false);
provider.enableControllerService(serviceNode); provider.enableControllerService(serviceNode);
provider.disableControllerService(serviceNode); provider.disableControllerService(serviceNode);
} }
@ -134,9 +136,9 @@ public class TestStandardControllerServiceProvider {
new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider, variableRegistry, niFiProperties); new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider, variableRegistry, niFiProperties);
final ControllerServiceNode serviceNodeB = provider.createControllerService(ServiceB.class.getName(), "B", final ControllerServiceNode serviceNodeB = provider.createControllerService(ServiceB.class.getName(), "B",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
final ControllerServiceNode serviceNodeA = provider.createControllerService(ServiceA.class.getName(), "A", final ControllerServiceNode serviceNodeA = provider.createControllerService(ServiceA.class.getName(), "A",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
group.addControllerService(serviceNodeA); group.addControllerService(serviceNodeA);
group.addControllerService(serviceNodeB); group.addControllerService(serviceNodeB);
@ -208,13 +210,13 @@ public class TestStandardControllerServiceProvider {
// we enable C and B, even if we attempt to enable C before B... i.e., if we try to enable C, we cannot do so // we enable C and B, even if we attempt to enable C before B... i.e., if we try to enable C, we cannot do so
// until B is first enabled so ensure that we enable B first. // until B is first enabled so ensure that we enable B first.
final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1", final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceA.class.getName(), "2", final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceA.class.getName(), "2",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
final ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3", final ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
final ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4", final ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
procGroup.addControllerService(serviceNode1); procGroup.addControllerService(serviceNode1);
procGroup.addControllerService(serviceNode2); procGroup.addControllerService(serviceNode2);
@ -251,9 +253,9 @@ public class TestStandardControllerServiceProvider {
final StandardControllerServiceProvider provider = final StandardControllerServiceProvider provider =
new StandardControllerServiceProvider(controller, null, null, stateManagerProvider, variableRegistry, niFiProperties); new StandardControllerServiceProvider(controller, null, null, stateManagerProvider, variableRegistry, niFiProperties);
final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1", final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceB.class.getName(), "2", final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceB.class.getName(), "2",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
setProperty(serviceNode1, ServiceA.OTHER_SERVICE.getName(), "2"); setProperty(serviceNode1, ServiceA.OTHER_SERVICE.getName(), "2");
@ -312,7 +314,7 @@ public class TestStandardControllerServiceProvider {
// like that. // like that.
nodeMap.clear(); nodeMap.clear();
final ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3", final ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
setProperty(serviceNode1, ServiceA.OTHER_SERVICE.getName(), "3"); setProperty(serviceNode1, ServiceA.OTHER_SERVICE.getName(), "3");
setProperty(serviceNode3, ServiceA.OTHER_SERVICE.getName(), "1"); setProperty(serviceNode3, ServiceA.OTHER_SERVICE.getName(), "1");
nodeMap.put("1", serviceNode1); nodeMap.put("1", serviceNode1);
@ -338,9 +340,9 @@ public class TestStandardControllerServiceProvider {
nodeMap.clear(); nodeMap.clear();
setProperty(serviceNode1, ServiceA.OTHER_SERVICE.getName(), "2"); setProperty(serviceNode1, ServiceA.OTHER_SERVICE.getName(), "2");
final ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4", final ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
final ControllerServiceNode serviceNode5 = provider.createControllerService(ServiceB.class.getName(), "5", final ControllerServiceNode serviceNode5 = provider.createControllerService(ServiceB.class.getName(), "5",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
setProperty(serviceNode3, ServiceA.OTHER_SERVICE.getName(), "4"); setProperty(serviceNode3, ServiceA.OTHER_SERVICE.getName(), "4");
nodeMap.put("1", serviceNode1); nodeMap.put("1", serviceNode1);
nodeMap.put("2", serviceNode2); nodeMap.put("2", serviceNode2);
@ -397,10 +399,11 @@ public class TestStandardControllerServiceProvider {
} }
private ProcessorNode createProcessor(final StandardProcessScheduler scheduler, final ControllerServiceProvider serviceProvider) { private ProcessorNode createProcessor(final StandardProcessScheduler scheduler, final ControllerServiceProvider serviceProvider) {
final ReloadComponent reloadComponent = Mockito.mock(ReloadComponent.class);
final LoggableComponent<Processor> dummyProcessor = new LoggableComponent<>(new DummyProcessor(), systemBundle.getBundleDetails().getCoordinate(), null); final LoggableComponent<Processor> dummyProcessor = new LoggableComponent<>(new DummyProcessor(), systemBundle.getBundleDetails().getCoordinate(), null);
final ProcessorNode procNode = new StandardProcessorNode(dummyProcessor, UUID.randomUUID().toString(), final ProcessorNode procNode = new StandardProcessorNode(dummyProcessor, UUID.randomUUID().toString(),
new StandardValidationContextFactory(serviceProvider, null), scheduler, serviceProvider, niFiProperties, new StandardValidationContextFactory(serviceProvider, null), scheduler, serviceProvider, niFiProperties,
VariableRegistry.EMPTY_REGISTRY); VariableRegistry.EMPTY_REGISTRY, reloadComponent);
final ProcessGroup group = new StandardProcessGroup(UUID.randomUUID().toString(), serviceProvider, scheduler, null, null, null, variableRegistry); final ProcessGroup group = new StandardProcessGroup(UUID.randomUUID().toString(), serviceProvider, scheduler, null, null, null, variableRegistry);
group.addProcessor(procNode); group.addProcessor(procNode);
@ -419,7 +422,7 @@ public class TestStandardControllerServiceProvider {
final StandardControllerServiceProvider provider = final StandardControllerServiceProvider provider =
new StandardControllerServiceProvider(controller, null, null, stateManagerProvider, variableRegistry, niFiProperties); new StandardControllerServiceProvider(controller, null, null, stateManagerProvider, variableRegistry, niFiProperties);
final ControllerServiceNode serviceNode = provider.createControllerService(ServiceA.class.getName(), "1", final ControllerServiceNode serviceNode = provider.createControllerService(ServiceA.class.getName(), "1",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
final ProcessorNode procNode = createProcessor(scheduler, provider); final ProcessorNode procNode = createProcessor(scheduler, provider);
serviceNode.addReference(procNode); serviceNode.addReference(procNode);
@ -443,17 +446,17 @@ public class TestStandardControllerServiceProvider {
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup); Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
ControllerServiceNode A = provider.createControllerService(ServiceA.class.getName(), "A", ControllerServiceNode A = provider.createControllerService(ServiceA.class.getName(), "A",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
ControllerServiceNode B = provider.createControllerService(ServiceA.class.getName(), "B", ControllerServiceNode B = provider.createControllerService(ServiceA.class.getName(), "B",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
ControllerServiceNode C = provider.createControllerService(ServiceA.class.getName(), "C", ControllerServiceNode C = provider.createControllerService(ServiceA.class.getName(), "C",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
ControllerServiceNode D = provider.createControllerService(ServiceB.class.getName(), "D", ControllerServiceNode D = provider.createControllerService(ServiceB.class.getName(), "D",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
ControllerServiceNode E = provider.createControllerService(ServiceA.class.getName(), "E", ControllerServiceNode E = provider.createControllerService(ServiceA.class.getName(), "E",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
ControllerServiceNode F = provider.createControllerService(ServiceB.class.getName(), "F", ControllerServiceNode F = provider.createControllerService(ServiceB.class.getName(), "F",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
procGroup.addControllerService(A); procGroup.addControllerService(A);
procGroup.addControllerService(B); procGroup.addControllerService(B);
@ -494,15 +497,15 @@ public class TestStandardControllerServiceProvider {
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup); Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
ControllerServiceNode A = provider.createControllerService(ServiceC.class.getName(), "A", ControllerServiceNode A = provider.createControllerService(ServiceC.class.getName(), "A",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
ControllerServiceNode B = provider.createControllerService(ServiceA.class.getName(), "B", ControllerServiceNode B = provider.createControllerService(ServiceA.class.getName(), "B",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
ControllerServiceNode C = provider.createControllerService(ServiceB.class.getName(), "C", ControllerServiceNode C = provider.createControllerService(ServiceB.class.getName(), "C",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
ControllerServiceNode D = provider.createControllerService(ServiceA.class.getName(), "D", ControllerServiceNode D = provider.createControllerService(ServiceA.class.getName(), "D",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
ControllerServiceNode F = provider.createControllerService(ServiceA.class.getName(), "F", ControllerServiceNode F = provider.createControllerService(ServiceA.class.getName(), "F",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
procGroup.addControllerService(A); procGroup.addControllerService(A);
procGroup.addControllerService(B); procGroup.addControllerService(B);
@ -536,19 +539,19 @@ public class TestStandardControllerServiceProvider {
Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup); Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1", ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceA.class.getName(), "2", ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceA.class.getName(), "2",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3", ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4", ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
ControllerServiceNode serviceNode5 = provider.createControllerService(ServiceA.class.getName(), "5", ControllerServiceNode serviceNode5 = provider.createControllerService(ServiceA.class.getName(), "5",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
ControllerServiceNode serviceNode6 = provider.createControllerService(ServiceB.class.getName(), "6", ControllerServiceNode serviceNode6 = provider.createControllerService(ServiceB.class.getName(), "6",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
ControllerServiceNode serviceNode7 = provider.createControllerService(ServiceC.class.getName(), "7", ControllerServiceNode serviceNode7 = provider.createControllerService(ServiceC.class.getName(), "7",
systemBundle.getBundleDetails().getCoordinate(), false); systemBundle.getBundleDetails().getCoordinate(), null, false);
procGroup.addControllerService(serviceNode1); procGroup.addControllerService(serviceNode1);
procGroup.addControllerService(serviceNode2); procGroup.addControllerService(serviceNode2);

View File

@ -0,0 +1,42 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<!--
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.
-->
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-framework</artifactId>
<version>1.2.0-SNAPSHOT</version>
</parent>
<artifactId>nifi-framework-nar-utils</artifactId>
<packaging>jar</packaging>
<dependencies>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-nar-utils</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-properties</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-framework-api</artifactId>
</dependency>
</dependencies>
</project>

View File

@ -53,7 +53,7 @@ public class ControllerServiceInitializer implements ConfigurableComponentInitia
final MockConfigurationContext context = new MockConfigurationContext(); final MockConfigurationContext context = new MockConfigurationContext();
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, controllerService, logger, context); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, controllerService, logger, context);
} finally { } finally {
ExtensionManager.removeInstanceClassLoaderIfExists(component.getIdentifier()); ExtensionManager.removeInstanceClassLoader(component.getIdentifier());
} }
} }
} }

View File

@ -52,7 +52,7 @@ public class ProcessorInitializer implements ConfigurableComponentInitializer {
final MockProcessContext context = new MockProcessContext(); final MockProcessContext context = new MockProcessContext();
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, processor, logger, context); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, processor, logger, context);
} finally { } finally {
ExtensionManager.removeInstanceClassLoaderIfExists(component.getIdentifier()); ExtensionManager.removeInstanceClassLoader(component.getIdentifier());
} }
} }
} }

View File

@ -51,7 +51,7 @@ public class ReportingTaskingInitializer implements ConfigurableComponentInitial
final MockConfigurationContext context = new MockConfigurationContext(); final MockConfigurationContext context = new MockConfigurationContext();
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, reportingTask, new MockComponentLogger(), context); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, reportingTask, new MockComponentLogger(), context);
} finally { } finally {
ExtensionManager.removeInstanceClassLoaderIfExists(component.getIdentifier()); ExtensionManager.removeInstanceClassLoader(component.getIdentifier());
} }
} }
} }

View File

@ -21,7 +21,6 @@ import org.apache.nifi.authentication.LoginIdentityProvider;
import org.apache.nifi.authorization.Authorizer; import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.bundle.Bundle; import org.apache.nifi.bundle.Bundle;
import org.apache.nifi.bundle.BundleCoordinate; import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.bundle.BundleDetails;
import org.apache.nifi.components.ConfigurableComponent; import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.StateProvider; import org.apache.nifi.components.state.StateProvider;
@ -37,12 +36,10 @@ import org.apache.nifi.processor.Processor;
import org.apache.nifi.provenance.ProvenanceRepository; import org.apache.nifi.provenance.ProvenanceRepository;
import org.apache.nifi.reporting.InitializationException; import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.reporting.ReportingTask; import org.apache.nifi.reporting.ReportingTask;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.StringUtils; import org.apache.nifi.util.StringUtils;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.net.URL; import java.net.URL;
import java.net.URLClassLoader; import java.net.URLClassLoader;
@ -50,6 +47,7 @@ import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.LinkedHashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.ServiceLoader; import java.util.ServiceLoader;
@ -67,18 +65,16 @@ public class ExtensionManager {
private static final Logger logger = LoggerFactory.getLogger(ExtensionManager.class); private static final Logger logger = LoggerFactory.getLogger(ExtensionManager.class);
public static final BundleCoordinate SYSTEM_BUNDLE_COORDINATE = new BundleCoordinate(
BundleCoordinate.DEFAULT_GROUP, "system", BundleCoordinate.DEFAULT_VERSION);
// Maps a service definition (interface) to those classes that implement the interface // Maps a service definition (interface) to those classes that implement the interface
private static final Map<Class, Set<Class>> definitionMap = new HashMap<>(); private static final Map<Class, Set<Class>> definitionMap = new HashMap<>();
private static final Map<String, List<Bundle>> classNameBundleLookup = new HashMap<>(); private static final Map<String, List<Bundle>> classNameBundleLookup = new HashMap<>();
private static final Map<BundleCoordinate, Bundle> bundleCoordinateBundleLookup = new HashMap<>(); private static final Map<BundleCoordinate, Bundle> bundleCoordinateBundleLookup = new HashMap<>();
private static final Map<ClassLoader, Bundle> classLoaderBundleLookup = new HashMap<>(); private static final Map<ClassLoader, Bundle> classLoaderBundleLookup = new HashMap<>();
private static final Map<String, ConfigurableComponent> tempComponentLookup = new HashMap<>();
private static final Set<String> requiresInstanceClassLoading = new HashSet<>(); private static final Map<String, Class<?>> requiresInstanceClassLoading = new HashMap<>();
private static final Map<String, ClassLoader> instanceClassloaderLookup = new ConcurrentHashMap<>(); private static final Map<String, InstanceClassLoader> instanceClassloaderLookup = new ConcurrentHashMap<>();
static { static {
definitionMap.put(Processor.class, new HashSet<>()); definitionMap.put(Processor.class, new HashSet<>());
@ -126,29 +122,6 @@ public class ExtensionManager {
} }
} }
/**
* Returns a bundle representing the system class loader.
*
* @param niFiProperties a NiFiProperties instance which will be used to obtain the default NAR library path,
* which will become the working directory of the returned bundle
* @return a bundle for the system class loader
*/
public static Bundle createSystemBundle(final NiFiProperties niFiProperties) {
final ClassLoader systemClassLoader = ClassLoader.getSystemClassLoader();
final String narLibraryDirectory = niFiProperties.getProperty(NiFiProperties.NAR_LIBRARY_DIRECTORY);
if (StringUtils.isBlank(narLibraryDirectory)) {
throw new IllegalStateException("Unable to create system bundle because " + NiFiProperties.NAR_LIBRARY_DIRECTORY + " was null or empty");
}
final BundleDetails systemBundleDetails = new BundleDetails.Builder()
.workingDir(new File(narLibraryDirectory))
.coordinate(SYSTEM_BUNDLE_COORDINATE)
.build();
return new Bundle(systemBundleDetails, systemClassLoader);
}
/** /**
* Loads extensions from the specified bundle. * Loads extensions from the specified bundle.
* *
@ -163,6 +136,15 @@ public class ExtensionManager {
final ServiceLoader<?> serviceLoader = ServiceLoader.load(entry.getKey(), bundle.getClassLoader()); final ServiceLoader<?> serviceLoader = ServiceLoader.load(entry.getKey(), bundle.getClassLoader());
for (final Object o : serviceLoader) { for (final Object o : serviceLoader) {
// create a cache of temp ConfigurableComponent instances, the initialize here has to happen before the checks below
if ((isControllerService || isProcessor || isReportingTask) && o instanceof ConfigurableComponent) {
final ConfigurableComponent configurableComponent = (ConfigurableComponent) o;
initializeTempComponent(configurableComponent);
final String cacheKey = getClassBundleKey(o.getClass().getCanonicalName(), bundle.getBundleDetails().getCoordinate());
tempComponentLookup.put(cacheKey, (ConfigurableComponent)o);
}
// only consider extensions discovered directly in this bundle // only consider extensions discovered directly in this bundle
boolean registerExtension = bundle.getClassLoader().equals(o.getClass().getClassLoader()); boolean registerExtension = bundle.getClassLoader().equals(o.getClass().getClassLoader());
@ -185,50 +167,48 @@ public class ExtensionManager {
registerServiceClass(o.getClass(), classNameBundleLookup, bundle, entry.getValue()); registerServiceClass(o.getClass(), classNameBundleLookup, bundle, entry.getValue());
} }
} }
} }
classLoaderBundleLookup.put(bundle.getClassLoader(), bundle); classLoaderBundleLookup.put(bundle.getClassLoader(), bundle);
} }
} }
private static void initializeTempComponent(final ConfigurableComponent configurableComponent) {
ConfigurableComponentInitializer initializer = null;
try {
initializer = ConfigurableComponentInitializerFactory.createComponentInitializer(configurableComponent.getClass());
initializer.initialize(configurableComponent);
} catch (final InitializationException e) {
logger.warn(String.format("Unable to initialize component %s due to %s", configurableComponent.getClass().getName(), e.getMessage()));
}
}
private static boolean checkControllerServiceReferenceEligibility(final ConfigurableComponent component, final ClassLoader classLoader) { private static boolean checkControllerServiceReferenceEligibility(final ConfigurableComponent component, final ClassLoader classLoader) {
// if the extension does not require instance classloading, its eligible // if the extension does not require instance classloading, its eligible
final boolean requiresInstanceClassLoading = component.getClass().isAnnotationPresent(RequiresInstanceClassLoading.class); final boolean requiresInstanceClassLoading = component.getClass().isAnnotationPresent(RequiresInstanceClassLoading.class);
ConfigurableComponentInitializer initializer = null; final Set<Class> cobundledApis = new HashSet<>();
try { try (final NarCloseable closeable = NarCloseable.withComponentNarLoader(component.getClass().getClassLoader())) {
initializer = ConfigurableComponentInitializerFactory.createComponentInitializer(component.getClass()); final List<PropertyDescriptor> descriptors = component.getPropertyDescriptors();
initializer.initialize(component); if (descriptors != null && !descriptors.isEmpty()) {
for (final PropertyDescriptor descriptor : descriptors) {
final Set<Class> cobundledApis = new HashSet<>(); final Class<? extends ControllerService> serviceApi = descriptor.getControllerServiceDefinition();
try (final NarCloseable closeable = NarCloseable.withComponentNarLoader(component.getClass().getClassLoader())) { if (serviceApi != null && classLoader.equals(serviceApi.getClassLoader())) {
final List<PropertyDescriptor> descriptors = component.getPropertyDescriptors(); cobundledApis.add(serviceApi);
if (descriptors != null && !descriptors.isEmpty()) {
for (final PropertyDescriptor descriptor : descriptors) {
final Class<? extends ControllerService> serviceApi = descriptor.getControllerServiceDefinition();
if (serviceApi != null && classLoader.equals(serviceApi.getClassLoader())) {
cobundledApis.add(serviceApi);
}
} }
} }
} }
if (!cobundledApis.isEmpty()) {
logger.warn(String.format(
"Component %s is bundled with its referenced Controller Service APIs %s. The service APIs should not be bundled with component implementations that reference it.",
component.getClass().getName(), StringUtils.join(cobundledApis.stream().map(cls -> cls.getName()).collect(Collectors.toSet()), ", ")));
}
// the component is eligible when it does not require instance classloading or when the supporting APIs are bundled in a parent NAR
return requiresInstanceClassLoading == false || cobundledApis.isEmpty();
} catch (final InitializationException e) {
logger.warn(String.format("Unable to verify if component %s references any bundled Controller Service APIs due to %s", component.getClass().getName(), e.getMessage()));
return true;
} finally {
if (initializer != null) {
initializer.teardown(component);
}
} }
if (!cobundledApis.isEmpty()) {
logger.warn(String.format(
"Component %s is bundled with its referenced Controller Service APIs %s. The service APIs should not be bundled with component implementations that reference it.",
component.getClass().getName(), StringUtils.join(cobundledApis.stream().map(cls -> cls.getName()).collect(Collectors.toSet()), ", ")));
}
// the component is eligible when it does not require instance classloading or when the supporting APIs are bundled in a parent NAR
return requiresInstanceClassLoading == false || cobundledApis.isEmpty();
} }
private static boolean checkControllerServiceEligibility(Class extensionType) { private static boolean checkControllerServiceEligibility(Class extensionType) {
@ -304,7 +284,8 @@ public class ExtensionManager {
classes.add(type); classes.add(type);
if (type.isAnnotationPresent(RequiresInstanceClassLoading.class)) { if (type.isAnnotationPresent(RequiresInstanceClassLoading.class)) {
requiresInstanceClassLoading.add(className); final String cacheKey = getClassBundleKey(className, bundle.getBundleDetails().getCoordinate());
requiresInstanceClassLoading.put(cacheKey, type);
} }
} }
@ -324,9 +305,10 @@ public class ExtensionManager {
* @param classType the type of class to lookup the ClassLoader for * @param classType the type of class to lookup the ClassLoader for
* @param instanceIdentifier the identifier of the specific instance of the classType to look up the ClassLoader for * @param instanceIdentifier the identifier of the specific instance of the classType to look up the ClassLoader for
* @param bundle the bundle where the classType exists * @param bundle the bundle where the classType exists
* @param additionalUrls additional URLs to add to the instance class loader
* @return the ClassLoader for the given instance of the given type, or null if the type is not a detected extension type * @return the ClassLoader for the given instance of the given type, or null if the type is not a detected extension type
*/ */
public static ClassLoader createInstanceClassLoader(final String classType, final String instanceIdentifier, final Bundle bundle) { public static InstanceClassLoader createInstanceClassLoader(final String classType, final String instanceIdentifier, final Bundle bundle, final Set<URL> additionalUrls) {
if (StringUtils.isEmpty(classType)) { if (StringUtils.isEmpty(classType)) {
throw new IllegalArgumentException("Class-Type is required"); throw new IllegalArgumentException("Class-Type is required");
} }
@ -339,67 +321,128 @@ public class ExtensionManager {
throw new IllegalArgumentException("Bundle is required"); throw new IllegalArgumentException("Bundle is required");
} }
final ClassLoader bundleClassLoader = bundle.getClassLoader();
// If the class is annotated with @RequiresInstanceClassLoading and the registered ClassLoader is a URLClassLoader // If the class is annotated with @RequiresInstanceClassLoading and the registered ClassLoader is a URLClassLoader
// then make a new InstanceClassLoader that is a full copy of the NAR Class Loader, otherwise create an empty // then make a new InstanceClassLoader that is a full copy of the NAR Class Loader, otherwise create an empty
// InstanceClassLoader that has the NAR ClassLoader as a parent // InstanceClassLoader that has the NAR ClassLoader as a parent
ClassLoader instanceClassLoader;
if (requiresInstanceClassLoading.contains(classType) && (bundleClassLoader instanceof URLClassLoader)) { InstanceClassLoader instanceClassLoader;
final URLClassLoader registeredUrlClassLoader = (URLClassLoader) bundleClassLoader; final ClassLoader bundleClassLoader = bundle.getClassLoader();
instanceClassLoader = new InstanceClassLoader(instanceIdentifier, classType, registeredUrlClassLoader.getURLs(), registeredUrlClassLoader.getParent()); final String key = getClassBundleKey(classType, bundle.getBundleDetails().getCoordinate());
if (requiresInstanceClassLoading.containsKey(key) && bundleClassLoader instanceof NarClassLoader) {
final Class<?> type = requiresInstanceClassLoading.get(key);
final RequiresInstanceClassLoading requiresInstanceClassLoading = type.getAnnotation(RequiresInstanceClassLoading.class);
final NarClassLoader narBundleClassLoader = (NarClassLoader) bundleClassLoader;
logger.debug("Including ClassLoader resources from {} for component {}", new Object[] {bundle.getBundleDetails(), instanceIdentifier});
final Set<URL> instanceUrls = new LinkedHashSet<>();
for (final URL url : narBundleClassLoader.getURLs()) {
instanceUrls.add(url);
}
ClassLoader ancestorClassLoader = narBundleClassLoader.getParent();
if (requiresInstanceClassLoading.cloneAncestorResources()) {
final ConfigurableComponent component = getTempComponent(classType, bundle.getBundleDetails().getCoordinate());
final Set<BundleCoordinate> reachableApiBundles = findReachableApiBundles(component);
while (ancestorClassLoader != null && ancestorClassLoader instanceof NarClassLoader) {
final Bundle ancestorNarBundle = classLoaderBundleLookup.get(ancestorClassLoader);
// stop including ancestor resources when we reach one of the APIs, or when we hit the Jetty NAR
if (ancestorNarBundle == null || reachableApiBundles.contains(ancestorNarBundle.getBundleDetails().getCoordinate())
|| ancestorNarBundle.getBundleDetails().getCoordinate().getId().equals(NarClassLoaders.JETTY_NAR_ID)) {
break;
}
final NarClassLoader ancestorNarClassLoader = (NarClassLoader) ancestorClassLoader;
for (final URL url : ancestorNarClassLoader.getURLs()) {
instanceUrls.add(url);
}
ancestorClassLoader = ancestorNarClassLoader.getParent();
}
}
instanceClassLoader = new InstanceClassLoader(instanceIdentifier, classType, instanceUrls, additionalUrls, ancestorClassLoader);
} else { } else {
instanceClassLoader = new InstanceClassLoader(instanceIdentifier, classType, new URL[0], bundleClassLoader); instanceClassLoader = new InstanceClassLoader(instanceIdentifier, classType, Collections.emptySet(), additionalUrls, bundleClassLoader);
}
if (logger.isTraceEnabled()) {
for (URL url : instanceClassLoader.getURLs()) {
logger.trace("URL resource {} for {}...", new Object[]{url.toExternalForm(), instanceIdentifier});
}
} }
instanceClassloaderLookup.put(instanceIdentifier, instanceClassLoader); instanceClassloaderLookup.put(instanceIdentifier, instanceClassLoader);
return instanceClassLoader; return instanceClassLoader;
} }
/**
* Find the bundle coordinates for any service APIs that are referenced by this component and not part of the same bundle.
*
* @param component the component being instantiated
*/
protected static Set<BundleCoordinate> findReachableApiBundles(final ConfigurableComponent component) {
final Set<BundleCoordinate> reachableApiBundles = new HashSet<>();
try (final NarCloseable closeable = NarCloseable.withComponentNarLoader(component.getClass().getClassLoader())) {
final List<PropertyDescriptor> descriptors = component.getPropertyDescriptors();
if (descriptors != null && !descriptors.isEmpty()) {
for (final PropertyDescriptor descriptor : descriptors) {
final Class<? extends ControllerService> serviceApi = descriptor.getControllerServiceDefinition();
if (serviceApi != null && !component.getClass().getClassLoader().equals(serviceApi.getClassLoader())) {
final Bundle apiBundle = classLoaderBundleLookup.get(serviceApi.getClassLoader());
reachableApiBundles.add(apiBundle.getBundleDetails().getCoordinate());
}
}
}
}
return reachableApiBundles;
}
/** /**
* Retrieves the InstanceClassLoader for the component with the given identifier. * Retrieves the InstanceClassLoader for the component with the given identifier.
* *
* @param instanceIdentifier the identifier of a component * @param instanceIdentifier the identifier of a component
* @return the instance class loader for the component * @return the instance class loader for the component
*/ */
public static ClassLoader getInstanceClassLoader(final String instanceIdentifier) { public static InstanceClassLoader getInstanceClassLoader(final String instanceIdentifier) {
return instanceClassloaderLookup.get(instanceIdentifier); return instanceClassloaderLookup.get(instanceIdentifier);
} }
/** /**
* Removes the ClassLoader for the given instance and closes it if necessary. * Removes the InstanceClassLoader for a given component.
* *
* @param instanceIdentifier the identifier of a component to remove the ClassLoader for * @param instanceIdentifier the of a component
* @return the removed ClassLoader for the given instance, or null if not found
*/ */
public static ClassLoader removeInstanceClassLoaderIfExists(final String instanceIdentifier) { public static InstanceClassLoader removeInstanceClassLoader(final String instanceIdentifier) {
if (instanceIdentifier == null) { if (instanceIdentifier == null) {
return null; return null;
} }
final ClassLoader classLoader = instanceClassloaderLookup.remove(instanceIdentifier); final InstanceClassLoader classLoader = instanceClassloaderLookup.remove(instanceIdentifier);
closeURLClassLoader(instanceIdentifier, classLoader);
return classLoader;
}
/**
* Closes the given ClassLoader if it is an instance of URLClassLoader.
*
* @param instanceIdentifier the instance id the class loader corresponds to
* @param classLoader the class loader to close
*/
public static void closeURLClassLoader(final String instanceIdentifier, final ClassLoader classLoader) {
if (classLoader != null && (classLoader instanceof URLClassLoader)) { if (classLoader != null && (classLoader instanceof URLClassLoader)) {
final URLClassLoader urlClassLoader = (URLClassLoader) classLoader; final URLClassLoader urlClassLoader = (URLClassLoader) classLoader;
try { try {
urlClassLoader.close(); urlClassLoader.close();
} catch (IOException e) { } catch (IOException e) {
logger.warn("Unable to class URLClassLoader for " + instanceIdentifier); logger.warn("Unable to close URLClassLoader for " + instanceIdentifier);
} }
} }
return classLoader;
}
/**
* Checks if the given class type requires per-instance class loading (i.e. contains the @RequiresInstanceClassLoading annotation)
*
* @param classType the class to check
* @return true if the class is found in the set of classes requiring instance level class loading, false otherwise
*/
public static boolean requiresInstanceClassLoading(final String classType) {
if (classType == null) {
throw new IllegalArgumentException("Class type cannot be null");
}
return requiresInstanceClassLoading.contains(classType);
} }
/** /**
@ -450,6 +493,22 @@ public class ExtensionManager {
return (extensions == null) ? Collections.<Class>emptySet() : extensions; return (extensions == null) ? Collections.<Class>emptySet() : extensions;
} }
public static ConfigurableComponent getTempComponent(final String classType, final BundleCoordinate bundleCoordinate) {
if (classType == null) {
throw new IllegalArgumentException("Class type cannot be null");
}
if (bundleCoordinate == null) {
throw new IllegalArgumentException("Bundle Coordinate cannot be null");
}
return tempComponentLookup.get(getClassBundleKey(classType, bundleCoordinate));
}
private static String getClassBundleKey(final String classType, final BundleCoordinate bundleCoordinate) {
return classType + "_" + bundleCoordinate.getCoordinate();
}
public static void logClassLoaderMapping() { public static void logClassLoaderMapping() {
final StringBuilder builder = new StringBuilder(); final StringBuilder builder = new StringBuilder();

View File

@ -0,0 +1,89 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.nar;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.net.URL;
import java.net.URLClassLoader;
import java.util.Collections;
import java.util.LinkedHashSet;
import java.util.Set;
/**
* Each processor, controller service, and reporting task will have an InstanceClassLoader.
*
* The InstanceClassLoader will either be an empty pass-through to the NARClassLoader, or will contain a
* copy of all the NAR's resources in the case of components that @RequireInstanceClassLoading.
*/
public class InstanceClassLoader extends URLClassLoader {
private static final Logger logger = LoggerFactory.getLogger(InstanceClassLoader.class);
private final String identifier;
private final String instanceType;
private final Set<URL> instanceUrls;
private final Set<URL> additionalResourceUrls;
/**
* @param identifier the id of the component this ClassLoader was created for
* @param instanceUrls the urls for the instance, will either be empty or a copy of the NARs urls
* @param additionalResourceUrls the urls that came from runtime properties of the component
* @param parent the parent ClassLoader
*/
public InstanceClassLoader(final String identifier, final String type, final Set<URL> instanceUrls, final Set<URL> additionalResourceUrls, final ClassLoader parent) {
super(combineURLs(instanceUrls, additionalResourceUrls), parent);
this.identifier = identifier;
this.instanceType = type;
this.instanceUrls = Collections.unmodifiableSet(
instanceUrls == null ? Collections.emptySet() : new LinkedHashSet<>(instanceUrls));
this.additionalResourceUrls = Collections.unmodifiableSet(
additionalResourceUrls == null ? Collections.emptySet() : new LinkedHashSet<>(additionalResourceUrls));
}
private static URL[] combineURLs(final Set<URL> instanceUrls, final Set<URL> additionalResourceUrls) {
final Set<URL> allUrls = new LinkedHashSet<>();
if (instanceUrls != null) {
allUrls.addAll(instanceUrls);
}
if (additionalResourceUrls != null) {
allUrls.addAll(additionalResourceUrls);
}
return allUrls.toArray(new URL[allUrls.size()]);
}
public String getIdentifier() {
return identifier;
}
public String getInstanceType() {
return instanceType;
}
public Set<URL> getInstanceUrls() {
return instanceUrls;
}
public Set<URL> getAdditionalResourceUrls() {
return additionalResourceUrls;
}
}

View File

@ -36,7 +36,7 @@ public class NarThreadContextClassLoaderTest {
@Test @Test
public void validateWithPropertiesConstructor() throws Exception { public void validateWithPropertiesConstructor() throws Exception {
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", null); NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", null);
Bundle systemBundle = ExtensionManager.createSystemBundle(properties); Bundle systemBundle = SystemBundle.create(properties);
ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet()); ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
Object obj = NarThreadContextClassLoader.createInstance(WithPropertiesConstructor.class.getName(), Object obj = NarThreadContextClassLoader.createInstance(WithPropertiesConstructor.class.getName(),
@ -51,7 +51,7 @@ public class NarThreadContextClassLoaderTest {
Map<String, String> additionalProperties = new HashMap<>(); Map<String, String> additionalProperties = new HashMap<>();
additionalProperties.put("fail", "true"); additionalProperties.put("fail", "true");
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", additionalProperties); NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", additionalProperties);
Bundle systemBundle = ExtensionManager.createSystemBundle(properties); Bundle systemBundle = SystemBundle.create(properties);
ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet()); ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
NarThreadContextClassLoader.createInstance(WithPropertiesConstructor.class.getName(), WithPropertiesConstructor.class, properties); NarThreadContextClassLoader.createInstance(WithPropertiesConstructor.class.getName(), WithPropertiesConstructor.class, properties);
} }
@ -59,7 +59,7 @@ public class NarThreadContextClassLoaderTest {
@Test @Test
public void validateWithDefaultConstructor() throws Exception { public void validateWithDefaultConstructor() throws Exception {
NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", null); NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", null);
Bundle systemBundle = ExtensionManager.createSystemBundle(properties); Bundle systemBundle = SystemBundle.create(properties);
ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet()); ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
assertTrue(NarThreadContextClassLoader.createInstance(WithDefaultConstructor.class.getName(), assertTrue(NarThreadContextClassLoader.createInstance(WithDefaultConstructor.class.getName(),
WithDefaultConstructor.class, properties) instanceof WithDefaultConstructor); WithDefaultConstructor.class, properties) instanceof WithDefaultConstructor);

View File

@ -88,7 +88,7 @@ public class NarUnpackerTest {
assertEquals("./target/NarUnpacker/lib2/", assertEquals("./target/NarUnpacker/lib2/",
properties.getProperty("nifi.nar.library.directory.alt")); properties.getProperty("nifi.nar.library.directory.alt"));
final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, ExtensionManager.createSystemBundle(properties)); final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties));
assertEquals(2, extensionMapping.getAllExtensionNames().size()); assertEquals(2, extensionMapping.getAllExtensionNames().size());
@ -119,7 +119,7 @@ public class NarUnpackerTest {
others.put("nifi.nar.library.directory.alt", emptyDir.toString()); others.put("nifi.nar.library.directory.alt", emptyDir.toString());
NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others); NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, ExtensionManager.createSystemBundle(properties)); final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties));
assertEquals(1, extensionMapping.getAllExtensionNames().size()); assertEquals(1, extensionMapping.getAllExtensionNames().size());
assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.one")); assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.one"));
@ -142,7 +142,7 @@ public class NarUnpackerTest {
others.put("nifi.nar.library.directory.alt", nonExistantDir.toString()); others.put("nifi.nar.library.directory.alt", nonExistantDir.toString());
NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others); NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, ExtensionManager.createSystemBundle(properties)); final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties));
assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.one")); assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.one"));
@ -166,7 +166,7 @@ public class NarUnpackerTest {
others.put("nifi.nar.library.directory.alt", nonDir.toString()); others.put("nifi.nar.library.directory.alt", nonDir.toString());
NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others); NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, ExtensionManager.createSystemBundle(properties)); final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties));
assertNull(extensionMapping); assertNull(extensionMapping);
} }

View File

@ -30,6 +30,7 @@ nifi.nar.library.directory=./target/NarUnpacker/lib/
nifi.nar.library.directory.alt=./target/NarUnpacker/lib2/ nifi.nar.library.directory.alt=./target/NarUnpacker/lib2/
nifi.nar.working.directory=./target/work/nar/ nifi.nar.working.directory=./target/work/nar/
nifi.documentation.working.directory=./target/work/docs/components
# H2 Settings # H2 Settings
nifi.database.directory=./target/database_repository nifi.database.directory=./target/database_repository

View File

@ -0,0 +1,198 @@
# 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.
# Core Properties #
nifi.flow.configuration.file=./conf/flow.xml.gz
nifi.flow.configuration.archive.enabled=true
nifi.flow.configuration.archive.dir=./conf/archive/
nifi.flow.configuration.archive.max.time=30 days
nifi.flow.configuration.archive.max.storage=500 MB
nifi.flowcontroller.autoResumeState=true
nifi.flowcontroller.graceful.shutdown.period=10 sec
nifi.flowservice.writedelay.interval=500 ms
nifi.administrative.yield.duration=30 sec
# If a component has no work to do (is "bored"), how long should we wait before checking again for work?
nifi.bored.yield.duration=10 millis
nifi.authorizer.configuration.file=./conf/authorizers.xml
nifi.login.identity.provider.configuration.file=./conf/login-identity-providers.xml
nifi.templates.directory=./conf/templates
nifi.ui.banner.text=
nifi.ui.autorefresh.interval=30 sec
nifi.nar.library.directory=./lib
nifi.nar.working.directory=./target/work/nar/
nifi.documentation.working.directory=./target/work/docs/components
####################
# State Management #
####################
nifi.state.management.configuration.file=./conf/state-management.xml
# The ID of the local state provider
nifi.state.management.provider.local=local-provider
# The ID of the cluster-wide state provider. This will be ignored if NiFi is not clustered but must be populated if running in a cluster.
nifi.state.management.provider.cluster=zk-provider
# Specifies whether or not this instance of NiFi should run an embedded ZooKeeper server
nifi.state.management.embedded.zookeeper.start=false
# Properties file that provides the ZooKeeper properties to use if <nifi.state.management.embedded.zookeeper.start> is set to true
nifi.state.management.embedded.zookeeper.properties=./conf/zookeeper.properties
# H2 Settings
nifi.database.directory=./database_repository
nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE
# FlowFile Repository
nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository
nifi.flowfile.repository.directory=./flowfile_repository
nifi.flowfile.repository.partitions=256
nifi.flowfile.repository.checkpoint.interval=2 mins
nifi.flowfile.repository.always.sync=false
nifi.swap.manager.implementation=org.apache.nifi.controller.FileSystemSwapManager
nifi.queue.swap.threshold=20000
nifi.swap.in.period=5 sec
nifi.swap.in.threads=1
nifi.swap.out.period=5 sec
nifi.swap.out.threads=4
# Content Repository
nifi.content.repository.implementation=org.apache.nifi.controller.repository.FileSystemRepository
nifi.content.claim.max.appendable.size=10 MB
nifi.content.claim.max.flow.files=100
nifi.content.repository.directory.default=./content_repository
nifi.content.repository.directory.content1=/tmp/foo/repo
nifi.content.repository.archive.max.retention.period=12 hours
nifi.content.repository.archive.max.usage.percentage=50%
nifi.content.repository.archive.enabled=true
nifi.content.repository.always.sync=false
nifi.content.viewer.url=/nifi-content-viewer/
# Provenance Repository Properties
nifi.provenance.repository.implementation=org.apache.nifi.provenance.PersistentProvenanceRepository
# Persistent Provenance Repository Properties
nifi.provenance.repository.directory.default=./provenance_repository
nifi.provenance.repository.max.storage.time=24 hours
nifi.provenance.repository.max.storage.size=1 GB
nifi.provenance.repository.rollover.time=30 secs
nifi.provenance.repository.rollover.size=100 MB
nifi.provenance.repository.query.threads=2
nifi.provenance.repository.index.threads=1
nifi.provenance.repository.compress.on.rollover=true
nifi.provenance.repository.always.sync=false
nifi.provenance.repository.journal.count=16
# Comma-separated list of fields. Fields that are not indexed will not be searchable. Valid fields are:
# EventType, FlowFileUUID, Filename, TransitURI, ProcessorID, AlternateIdentifierURI, Relationship, Details
nifi.provenance.repository.indexed.fields=EventType, FlowFileUUID, Filename, ProcessorID, Relationship
# FlowFile Attributes that should be indexed and made searchable. Some examples to consider are filename, uuid, mime.type
nifi.provenance.repository.indexed.attributes=
# Large values for the shard size will result in more Java heap usage when searching the Provenance Repository
# but should provide better performance
nifi.provenance.repository.index.shard.size=500 MB
# Indicates the maximum length that a FlowFile attribute can be when retrieving a Provenance Event from
# the repository. If the length of any attribute exceeds this value, it will be truncated when the event is retrieved.
nifi.provenance.repository.max.attribute.length=65536
# Volatile Provenance Respository Properties
nifi.provenance.repository.buffer.size=100000
# Component Status Repository
nifi.components.status.repository.implementation=org.apache.nifi.controller.status.history.VolatileComponentStatusRepository
nifi.components.status.repository.buffer.size=1440
nifi.components.status.snapshot.frequency=1 min
# Site to Site properties
nifi.remote.input.host=
nifi.remote.input.secure=false
nifi.remote.input.socket.port=
nifi.remote.input.http.enabled=true
nifi.remote.input.http.transaction.ttl=30 sec
# web properties #
nifi.web.war.directory=./lib
nifi.web.http.host=
nifi.web.http.port=8080
nifi.web.https.host=
nifi.web.https.port=
nifi.web.jetty.working.directory=./work/jetty
nifi.web.jetty.threads=200
# security properties #
nifi.sensitive.props.key=
nifi.sensitive.props.key.protected=
nifi.sensitive.props.algorithm=PBEWITHMD5AND256BITAES-CBC-OPENSSL
nifi.sensitive.props.provider=BC
nifi.sensitive.props.additional.keys=
nifi.security.keystore=
nifi.security.keystoreType=
nifi.security.keystorePasswd=
nifi.security.keyPasswd=
nifi.security.truststore=
nifi.security.truststoreType=
nifi.security.truststorePasswd=
nifi.security.needClientAuth=
nifi.security.user.authorizer=file-provider
nifi.security.user.login.identity.provider=
nifi.security.ocsp.responder.url=
nifi.security.ocsp.responder.certificate=
# Identity Mapping Properties #
# These properties allow normalizing user identities such that identities coming from different identity providers
# (certificates, LDAP, Kerberos) can be treated the same internally in NiFi. The following example demonstrates normalizing
# DNs from certificates and principals from Kerberos into a common identity string:
#
# nifi.security.identity.mapping.pattern.dn=^CN=(.*?), OU=(.*?), O=(.*?), L=(.*?), ST=(.*?), C=(.*?)$
# nifi.security.identity.mapping.value.dn=$1@$2
# nifi.security.identity.mapping.pattern.kerb=^(.*?)/instance@(.*?)$
# nifi.security.identity.mapping.value.kerb=$1@$2
# cluster common properties (all nodes must have same values) #
nifi.cluster.protocol.heartbeat.interval=5 sec
nifi.cluster.protocol.is.secure=false
# cluster node properties (only configure for cluster nodes) #
nifi.cluster.is.node=false
nifi.cluster.node.address=
nifi.cluster.node.protocol.port=
nifi.cluster.node.protocol.threads=10
nifi.cluster.node.event.history.size=25
nifi.cluster.node.connection.timeout=5 sec
nifi.cluster.node.read.timeout=5 sec
nifi.cluster.firewall.file=
nifi.cluster.flow.election.max.wait.time=5 mins
nifi.cluster.flow.election.max.candidates=
# zookeeper properties, used for cluster management #
nifi.zookeeper.connect.string=
nifi.zookeeper.connect.timeout=3 secs
nifi.zookeeper.session.timeout=3 secs
nifi.zookeeper.root.node=/nifi
# kerberos #
nifi.kerberos.krb5.file=
# kerberos service principal #
nifi.kerberos.service.principal=
nifi.kerberos.service.keytab.location=
# kerberos spnego principal #
nifi.kerberos.spnego.principal=
nifi.kerberos.spnego.keytab.location=
nifi.kerberos.spnego.authentication.expiration=12 hours
# external properties files for variable registry
# supports a comma delimited list of file locations
nifi.variable.registry.properties=

View File

@ -45,6 +45,7 @@
<exclude>src/test/resources/nars/nar-with-versioning/META-INF/MANIFEST.MF</exclude> <exclude>src/test/resources/nars/nar-with-versioning/META-INF/MANIFEST.MF</exclude>
<exclude>src/test/resources/nars/nar-without-versioning/META-INF/MANIFEST.MF</exclude> <exclude>src/test/resources/nars/nar-without-versioning/META-INF/MANIFEST.MF</exclude>
<exclude>src/test/resources/nars/nar-without-dependency/META-INF/MANIFEST.MF</exclude> <exclude>src/test/resources/nars/nar-without-dependency/META-INF/MANIFEST.MF</exclude>
<exclude>src/test/resources/nars/nar-requires-cloning/META-INF/MANIFEST.MF</exclude>
</excludes> </excludes>
</configuration> </configuration>
</plugin> </plugin>

View File

@ -1,160 +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.nar;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.URL;
import java.net.URLClassLoader;
/**
* A ClassLoader created for an instance of a component which lets a client add resources to an intermediary ClassLoader
* that will be checked first when loading/finding classes.
*
* Typically an instance of this ClassLoader will be created by passing in the URLs and parent from a NARClassLoader in
* order to create a copy of the NARClassLoader without modifying it.
*/
public class InstanceClassLoader extends URLClassLoader {
private static final Logger logger = LoggerFactory.getLogger(InstanceClassLoader.class);
private final String identifier;
private final String instanceType;
private ShimClassLoader shimClassLoader;
/**
* @param identifier the id of the component this ClassLoader was created for
* @param urls the URLs for the ClassLoader
* @param parent the parent ClassLoader
*/
public InstanceClassLoader(final String identifier, final String type, final URL[] urls, final ClassLoader parent) {
super(urls, parent);
this.identifier = identifier;
this.instanceType = type;
}
/**
* Initializes a new ShimClassLoader for the provided resources, closing the previous ShimClassLoader if one existed.
*
* @param urls the URLs for the ShimClassLoader
* @throws IOException if the previous ShimClassLoader existed and couldn't be closed
*/
public synchronized void setInstanceResources(final URL[] urls) {
if (shimClassLoader != null) {
try {
shimClassLoader.close();
} catch (IOException e) {
logger.warn("Unable to close inner URLClassLoader for " + identifier);
}
}
shimClassLoader = new ShimClassLoader(urls, getParent());
}
/**
* @return the URLs for the instance resources that have been set
*/
public synchronized URL[] getInstanceResources() {
if (shimClassLoader != null) {
return shimClassLoader.getURLs();
}
return new URL[0];
}
@Override
public Class<?> loadClass(String name) throws ClassNotFoundException {
return this.loadClass(name, false);
}
@Override
protected Class<?> loadClass(String name, boolean resolve) throws ClassNotFoundException {
Class<?> c = null;
// first try the shim
if (shimClassLoader != null) {
try {
c = shimClassLoader.loadClass(name, resolve);
} catch (ClassNotFoundException e) {
c = null;
}
}
// if it wasn't in the shim try our self
if (c == null) {
return super.loadClass(name, resolve);
} else {
return c;
}
}
@Override
protected Class<?> findClass(String name) throws ClassNotFoundException {
Class<?> c = null;
// first try the shim
if (shimClassLoader != null) {
try {
c = shimClassLoader.findClass(name);
} catch (ClassNotFoundException cnf) {
c = null;
}
}
// if it wasn't in the shim try our self
if (c == null) {
return super.findClass(name);
} else {
return c;
}
}
@Override
public void close() throws IOException {
if (shimClassLoader != null) {
try {
shimClassLoader.close();
} catch (IOException e) {
logger.warn("Unable to close inner URLClassLoader for " + identifier);
}
}
super.close();
}
/**
* Extend URLClassLoader to increase visibility of protected methods so that InstanceClassLoader can delegate.
*/
private static class ShimClassLoader extends URLClassLoader {
public ShimClassLoader(URL[] urls, ClassLoader parent) {
super(urls, parent);
}
public ShimClassLoader(URL[] urls) {
super(urls);
}
@Override
public Class<?> findClass(String name) throws ClassNotFoundException {
return super.findClass(name);
}
@Override
public Class<?> loadClass(String name, boolean resolve) throws ClassNotFoundException {
return super.loadClass(name, resolve);
}
}
}

View File

@ -184,6 +184,7 @@ public final class NarClassLoaders {
jettyClassLoader = createNarClassLoader(narDetail.getWorkingDirectory(), systemClassLoader); jettyClassLoader = createNarClassLoader(narDetail.getWorkingDirectory(), systemClassLoader);
// remove the jetty nar since its already loaded // remove the jetty nar since its already loaded
narDirectoryBundleLookup.put(narDetail.getWorkingDirectory().getCanonicalPath(), new Bundle(narDetail, jettyClassLoader));
narCoordinateClassLoaderLookup.put(narDetail.getCoordinate().getCoordinate(), jettyClassLoader); narCoordinateClassLoaderLookup.put(narDetail.getCoordinate().getCoordinate(), jettyClassLoader);
narDetailsIter.remove(); narDetailsIter.remove();
} }

View File

@ -32,7 +32,7 @@ public enum NarManifestEntry {
BUILD_BRANCH("Build-Branch"), BUILD_BRANCH("Build-Branch"),
BUILD_TIMESTAMP("Build-Timestamp"), BUILD_TIMESTAMP("Build-Timestamp"),
BUILD_JDK("Build-Jdk"), BUILD_JDK("Build-Jdk"),
BUILT_BY("Built-By") BUILT_BY("Built-By"),
; ;
final String manifestName; final String manifestName;

View File

@ -0,0 +1,57 @@
/*
* 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.nar;
import org.apache.nifi.bundle.Bundle;
import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.bundle.BundleDetails;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.StringUtils;
import java.io.File;
/**
* Utility to create the system bundle.
*/
public final class SystemBundle {
public static final BundleCoordinate SYSTEM_BUNDLE_COORDINATE = new BundleCoordinate(
BundleCoordinate.DEFAULT_GROUP, "system", BundleCoordinate.DEFAULT_VERSION);
/**
* Returns a bundle representing the system class loader.
*
* @param niFiProperties a NiFiProperties instance which will be used to obtain the default NAR library path,
* which will become the working directory of the returned bundle
* @return a bundle for the system class loader
*/
public static Bundle create(final NiFiProperties niFiProperties) {
final ClassLoader systemClassLoader = ClassLoader.getSystemClassLoader();
final String narLibraryDirectory = niFiProperties.getProperty(NiFiProperties.NAR_LIBRARY_DIRECTORY);
if (StringUtils.isBlank(narLibraryDirectory)) {
throw new IllegalStateException("Unable to create system bundle because " + NiFiProperties.NAR_LIBRARY_DIRECTORY + " was null or empty");
}
final BundleDetails systemBundleDetails = new BundleDetails.Builder()
.workingDir(new File(narLibraryDirectory))
.coordinate(SYSTEM_BUNDLE_COORDINATE)
.build();
return new Bundle(systemBundleDetails, systemClassLoader);
}
}

View File

@ -16,9 +16,6 @@
*/ */
package org.apache.nifi.nar; package org.apache.nifi.nar;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import org.apache.nifi.bundle.BundleCoordinate; import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.bundle.BundleDetails; import org.apache.nifi.bundle.BundleDetails;
import org.junit.Test; import org.junit.Test;
@ -26,6 +23,9 @@ import org.junit.Test;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
public class NarBundleUtilTest { public class NarBundleUtilTest {
@Test @Test

View File

@ -43,11 +43,6 @@
<!-- The dependency is required to be made available by the binary build in the bootstrap folder --> <!-- The dependency is required to be made available by the binary build in the bootstrap folder -->
<scope>provided</scope> <scope>provided</scope>
</dependency> </dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-documentation</artifactId>
<scope>compile</scope>
</dependency>
<dependency> <dependency>
<groupId>org.slf4j</groupId> <groupId>org.slf4j</groupId>
<artifactId>jul-to-slf4j</artifactId> <artifactId>jul-to-slf4j</artifactId>

View File

@ -17,11 +17,10 @@
package org.apache.nifi; package org.apache.nifi;
import org.apache.nifi.bundle.Bundle; import org.apache.nifi.bundle.Bundle;
import org.apache.nifi.documentation.DocGenerator;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.ExtensionMapping; import org.apache.nifi.nar.ExtensionMapping;
import org.apache.nifi.nar.NarClassLoaders; import org.apache.nifi.nar.NarClassLoaders;
import org.apache.nifi.nar.NarUnpacker; import org.apache.nifi.nar.NarUnpacker;
import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.util.FileUtils; import org.apache.nifi.util.FileUtils;
import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.NiFiProperties;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -126,7 +125,7 @@ public class NiFi {
SLF4JBridgeHandler.removeHandlersForRootLogger(); SLF4JBridgeHandler.removeHandlersForRootLogger();
SLF4JBridgeHandler.install(); SLF4JBridgeHandler.install();
final Bundle systemBundle = ExtensionManager.createSystemBundle(properties); final Bundle systemBundle = SystemBundle.create(properties);
// expand the nars // expand the nars
final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, systemBundle); final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, systemBundle);
@ -140,12 +139,7 @@ public class NiFi {
throw new IllegalStateException("Unable to find the framework NAR ClassLoader."); throw new IllegalStateException("Unable to find the framework NAR ClassLoader.");
} }
// discover the extensions
final Set<Bundle> narBundles = NarClassLoaders.getInstance().getBundles(); final Set<Bundle> narBundles = NarClassLoaders.getInstance().getBundles();
ExtensionManager.discoverExtensions(systemBundle, narBundles);
ExtensionManager.logClassLoaderMapping();
DocGenerator.generate(properties, extensionMapping);
// load the server from the framework classloader // load the server from the framework classloader
Thread.currentThread().setContextClassLoader(frameworkClassLoader); Thread.currentThread().setContextClassLoader(frameworkClassLoader);
@ -155,6 +149,7 @@ public class NiFi {
final long startTime = System.nanoTime(); final long startTime = System.nanoTime();
nifiServer = (NiFiServer) jettyConstructor.newInstance(properties, narBundles); nifiServer = (NiFiServer) jettyConstructor.newInstance(properties, narBundles);
nifiServer.setExtensionMapping(extensionMapping); nifiServer.setExtensionMapping(extensionMapping);
nifiServer.setBundles(systemBundle, narBundles);
if (shutdown) { if (shutdown) {
LOGGER.info("NiFi has been shutdown via NiFi Bootstrap. Will not start Controller"); LOGGER.info("NiFi has been shutdown via NiFi Bootstrap. Will not start Controller");

View File

@ -16,8 +16,11 @@
*/ */
package org.apache.nifi; package org.apache.nifi;
import org.apache.nifi.bundle.Bundle;
import org.apache.nifi.nar.ExtensionMapping; import org.apache.nifi.nar.ExtensionMapping;
import java.util.Set;
/** /**
* *
*/ */
@ -27,5 +30,7 @@ public interface NiFiServer {
void setExtensionMapping(ExtensionMapping extensionMapping); void setExtensionMapping(ExtensionMapping extensionMapping);
void setBundles(Bundle systemBundle, Set<Bundle> bundles);
void stop(); void stop();
} }

View File

@ -33,6 +33,11 @@
<artifactId>nifi-runtime</artifactId> <artifactId>nifi-runtime</artifactId>
<scope>compile</scope> <scope>compile</scope>
</dependency> </dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-documentation</artifactId>
<scope>compile</scope>
</dependency>
<dependency> <dependency>
<groupId>org.apache.nifi</groupId> <groupId>org.apache.nifi</groupId>
<artifactId>nifi-nar-utils</artifactId> <artifactId>nifi-nar-utils</artifactId>

View File

@ -26,7 +26,9 @@ import org.apache.nifi.bundle.BundleDetails;
import org.apache.nifi.controller.UninheritableFlowException; import org.apache.nifi.controller.UninheritableFlowException;
import org.apache.nifi.controller.serialization.FlowSerializationException; import org.apache.nifi.controller.serialization.FlowSerializationException;
import org.apache.nifi.controller.serialization.FlowSynchronizationException; import org.apache.nifi.controller.serialization.FlowSynchronizationException;
import org.apache.nifi.documentation.DocGenerator;
import org.apache.nifi.lifecycle.LifeCycleStartException; import org.apache.nifi.lifecycle.LifeCycleStartException;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.ExtensionMapping; import org.apache.nifi.nar.ExtensionMapping;
import org.apache.nifi.security.util.KeyStoreUtils; import org.apache.nifi.security.util.KeyStoreUtils;
import org.apache.nifi.services.FlowService; import org.apache.nifi.services.FlowService;
@ -113,7 +115,10 @@ public class JettyServer implements NiFiServer {
private final Server server; private final Server server;
private final NiFiProperties props; private final NiFiProperties props;
private Bundle systemBundle;
private Set<Bundle> bundles;
private ExtensionMapping extensionMapping; private ExtensionMapping extensionMapping;
private WebAppContext webApiContext; private WebAppContext webApiContext;
private WebAppContext webDocsContext; private WebAppContext webDocsContext;
@ -681,6 +686,11 @@ public class JettyServer implements NiFiServer {
@Override @Override
public void start() { public void start() {
try { try {
ExtensionManager.discoverExtensions(systemBundle, bundles);
ExtensionManager.logClassLoaderMapping();
DocGenerator.generate(props, extensionMapping);
// start the server // start the server
server.start(); server.start();
@ -858,6 +868,12 @@ public class JettyServer implements NiFiServer {
this.extensionMapping = extensionMapping; this.extensionMapping = extensionMapping;
} }
@Override
public void setBundles(Bundle systemBundle, Set<Bundle> bundles) {
this.systemBundle = systemBundle;
this.bundles = bundles;
}
@Override @Override
public void stop() { public void stop() {
try { try {

View File

@ -29,23 +29,24 @@ public interface AuthorizableLookup {
*/ */
Authorizable getController(); Authorizable getController();
/**
* Get the authorizable for the given type and bundle. This will use a dummy instance of the
* component. The intent of this method is to provide access to the PropertyDescriptors
* prior to the component being created.
*
* @param type component type
* @param bundle the bundle for the component
* @return authorizable
*/
ComponentAuthorizable getConfigurableComponent(String type, BundleDTO bundle);
/** /**
* Get the authorizable Processor. * Get the authorizable Processor.
* *
* @param id processor id * @param id processor id
* @return authorizable * @return authorizable
*/ */
ConfigurableComponentAuthorizable getProcessor(String id); ComponentAuthorizable getProcessor(String id);
/**
* Get the authorizable for this Processor. This will create a dummy instance of the
* processor. The intent of this method is to provide access to the PropertyDescriptors
* prior to the component being created.
*
* @param type processor type
* @return authorizable
*/
ConfigurableComponentAuthorizable getProcessorByType(String type, BundleDTO bundle);
/** /**
* Get the authorizable for querying Provenance. * Get the authorizable for querying Provenance.
@ -139,18 +140,7 @@ public interface AuthorizableLookup {
* @param id controller service id * @param id controller service id
* @return authorizable * @return authorizable
*/ */
ConfigurableComponentAuthorizable getControllerService(String id); ComponentAuthorizable getControllerService(String id);
/**
* Get the authorizable for this Controller Service. This will create a dummy instance of the
* controller service. The intent of this method is to provide access to the PropertyDescriptors
* prior to the component being created.
*
* @param type controller service type
* @param bundle bundle
* @return authorizable
*/
ConfigurableComponentAuthorizable getControllerServiceByType(String type, BundleDTO bundle);
/** /**
* Get the authorizable referencing component. * Get the authorizable referencing component.
@ -167,18 +157,7 @@ public interface AuthorizableLookup {
* @param id reporting task id * @param id reporting task id
* @return authorizable * @return authorizable
*/ */
ConfigurableComponentAuthorizable getReportingTask(String id); ComponentAuthorizable getReportingTask(String id);
/**
* Get the authorizable for this Reporting Task. This will create a dummy instance of the
* reporting task. The intent of this method is to provide access to the PropertyDescriptors
* prior to the component being created.
*
* @param type reporting task type
* @param bundle bundle
* @return authorizable
*/
ConfigurableComponentAuthorizable getReportingTaskByType(String type, BundleDTO bundle);
/** /**
* Get the authorizable Template. * Get the authorizable Template.

View File

@ -37,7 +37,7 @@ public final class AuthorizeControllerServiceReference {
* @param authorizer authorizer * @param authorizer authorizer
* @param lookup lookup * @param lookup lookup
*/ */
public static void authorizeControllerServiceReferences(final ConfigurableComponentAuthorizable authorizable, final Authorizer authorizer, public static void authorizeControllerServiceReferences(final ComponentAuthorizable authorizable, final Authorizer authorizer,
final AuthorizableLookup lookup, final boolean authorizeTransitiveServices) { final AuthorizableLookup lookup, final boolean authorizeTransitiveServices) {
// consider each property when looking for service references // consider each property when looking for service references
@ -50,7 +50,7 @@ public final class AuthorizeControllerServiceReference {
// authorize the service if configured // authorize the service if configured
if (serviceId != null) { if (serviceId != null) {
try { try {
final ConfigurableComponentAuthorizable currentServiceAuthorizable = lookup.getControllerService(serviceId); final ComponentAuthorizable currentServiceAuthorizable = lookup.getControllerService(serviceId);
currentServiceAuthorizable.getAuthorizable().authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()); currentServiceAuthorizable.getAuthorizable().authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
if (authorizeTransitiveServices) { if (authorizeTransitiveServices) {
@ -72,7 +72,7 @@ public final class AuthorizeControllerServiceReference {
* @param authorizer authorizer * @param authorizer authorizer
* @param lookup lookup * @param lookup lookup
*/ */
public static void authorizeControllerServiceReferences(final Map<String, String> proposedProperties, final ConfigurableComponentAuthorizable authorizable, public static void authorizeControllerServiceReferences(final Map<String, String> proposedProperties, final ComponentAuthorizable authorizable,
final Authorizer authorizer, final AuthorizableLookup lookup) { final Authorizer authorizer, final AuthorizableLookup lookup) {
// only attempt to authorize if properties are changing // only attempt to authorize if properties are changing

View File

@ -24,7 +24,7 @@ import java.util.List;
/** /**
* Authorizable for a component that references a ControllerService. * Authorizable for a component that references a ControllerService.
*/ */
public interface ConfigurableComponentAuthorizable { public interface ComponentAuthorizable {
/** /**
* Returns the base authorizable for this ControllerServiceReference. Non null * Returns the base authorizable for this ControllerServiceReference. Non null
* *

View File

@ -36,7 +36,7 @@ public interface ProcessGroupAuthorizable {
* *
* @return all encapsulated processors * @return all encapsulated processors
*/ */
Set<ConfigurableComponentAuthorizable> getEncapsulatedProcessors(); Set<ComponentAuthorizable> getEncapsulatedProcessors();
/** /**
* The authorizables for all encapsulated connections. Non null * The authorizables for all encapsulated connections. Non null
@ -99,6 +99,6 @@ public interface ProcessGroupAuthorizable {
* *
* @return all encapsulated input ports * @return all encapsulated input ports
*/ */
Set<ConfigurableComponentAuthorizable> getEncapsulatedControllerServices(); Set<ComponentAuthorizable> getEncapsulatedControllerServices();
} }

View File

@ -36,7 +36,7 @@ public interface SnippetAuthorizable {
* *
* @return processors * @return processors
*/ */
Set<ConfigurableComponentAuthorizable> getSelectedProcessors(); Set<ComponentAuthorizable> getSelectedProcessors();
/** /**
* The authorizables for selected connections. Non null * The authorizables for selected connections. Non null

View File

@ -17,6 +17,7 @@
package org.apache.nifi.authorization; package org.apache.nifi.authorization;
import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.authorization.resource.AccessPolicyAuthorizable; import org.apache.nifi.authorization.resource.AccessPolicyAuthorizable;
import org.apache.nifi.authorization.resource.Authorizable; import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.authorization.resource.DataAuthorizable; import org.apache.nifi.authorization.resource.DataAuthorizable;
@ -27,6 +28,7 @@ import org.apache.nifi.authorization.resource.RestrictedComponentsAuthorizable;
import org.apache.nifi.authorization.resource.TenantAuthorizable; import org.apache.nifi.authorization.resource.TenantAuthorizable;
import org.apache.nifi.authorization.user.NiFiUser; import org.apache.nifi.authorization.user.NiFiUser;
import org.apache.nifi.bundle.BundleCoordinate; import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.Connection; import org.apache.nifi.connectable.Connection;
@ -142,19 +144,19 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
} }
@Override @Override
public ConfigurableComponentAuthorizable getProcessor(final String id) { public ComponentAuthorizable getConfigurableComponent(final String type, final BundleDTO bundle) {
final ProcessorNode processorNode = processorDAO.getProcessor(id); try {
return new ProcessorConfigurableComponentAuthorizable(processorNode); final ConfigurableComponent configurableComponent = controllerFacade.getTemporaryComponent(type, bundle);
return new ConfigurableComponentAuthorizable(configurableComponent);
} catch (final Exception e) {
throw new AccessDeniedException("Unable to create component to verify if it references any Controller Services.");
}
} }
@Override @Override
public ConfigurableComponentAuthorizable getProcessorByType(String type, BundleDTO bundle) { public ComponentAuthorizable getProcessor(final String id) {
try { final ProcessorNode processorNode = processorDAO.getProcessor(id);
final ProcessorNode processorNode = controllerFacade.createTemporaryProcessor(type, bundle); return new ProcessorComponentAuthorizable(processorNode);
return new ProcessorConfigurableComponentAuthorizable(processorNode);
} catch (final Exception e) {
throw new AccessDeniedException("Unable to create processor to verify if it references any Controller Services.");
}
} }
@Override @Override
@ -251,19 +253,9 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
} }
@Override @Override
public ConfigurableComponentAuthorizable getControllerService(final String id) { public ComponentAuthorizable getControllerService(final String id) {
final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(id); final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(id);
return new ControllerServiceConfigurableComponentAuthorizable(controllerService); return new ControllerServiceComponentAuthorizable(controllerService);
}
@Override
public ConfigurableComponentAuthorizable getControllerServiceByType(String type, BundleDTO bundle) {
try {
final ControllerServiceNode controllerService = controllerFacade.createTemporaryControllerService(type, bundle);
return new ControllerServiceConfigurableComponentAuthorizable(controllerService);
} catch (final Exception e) {
throw new AccessDeniedException("Unable to create controller service to verify if it references any Controller Services.");
}
} }
@Override @Override
@ -310,19 +302,9 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
} }
@Override @Override
public ConfigurableComponentAuthorizable getReportingTask(final String id) { public ComponentAuthorizable getReportingTask(final String id) {
final ReportingTaskNode reportingTaskNode = reportingTaskDAO.getReportingTask(id); final ReportingTaskNode reportingTaskNode = reportingTaskDAO.getReportingTask(id);
return new ReportingTaskConfigurableComponentAuthorizable(reportingTaskNode); return new ReportingTaskComponentAuthorizable(reportingTaskNode);
}
@Override
public ConfigurableComponentAuthorizable getReportingTaskByType(String type, BundleDTO bundle) {
try {
final ReportingTaskNode reportingTask = controllerFacade.createTemporaryReportingTask(type, bundle);
return new ReportingTaskConfigurableComponentAuthorizable(reportingTask);
} catch (final Exception e) {
throw new AccessDeniedException("Unable to create reporting to verify if it references any Controller Services.");
}
} }
@Override @Override
@ -337,7 +319,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
} }
@Override @Override
public Set<ConfigurableComponentAuthorizable> getSelectedProcessors() { public Set<ComponentAuthorizable> getSelectedProcessors() {
return processGroup.getProcessors().stream() return processGroup.getProcessors().stream()
.filter(processor -> snippet.getProcessors().containsKey(processor.getIdentifier())) .filter(processor -> snippet.getProcessors().containsKey(processor.getIdentifier()))
.map(processor -> getProcessor(processor.getIdentifier())) .map(processor -> getProcessor(processor.getIdentifier()))
@ -616,8 +598,8 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
* @param controllerServices controller services * @param controllerServices controller services
*/ */
private void createTemporaryProcessorsAndControllerServices(final FlowSnippetDTO snippet, private void createTemporaryProcessorsAndControllerServices(final FlowSnippetDTO snippet,
final Set<ConfigurableComponentAuthorizable> processors, final Set<ComponentAuthorizable> processors,
final Set<ConfigurableComponentAuthorizable> controllerServices) { final Set<ComponentAuthorizable> controllerServices) {
if (snippet == null) { if (snippet == null) {
return; return;
@ -627,7 +609,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
snippet.getProcessors().forEach(processor -> { snippet.getProcessors().forEach(processor -> {
try { try {
final BundleCoordinate bundle = BundleUtils.getCompatibleBundle(processor.getType(), processor.getBundle()); final BundleCoordinate bundle = BundleUtils.getCompatibleBundle(processor.getType(), processor.getBundle());
processors.add(getProcessorByType(processor.getType(), new BundleDTO(bundle.getGroup(), bundle.getId(), bundle.getVersion()))); processors.add(getConfigurableComponent(processor.getType(), new BundleDTO(bundle.getGroup(), bundle.getId(), bundle.getVersion())));
} catch (final IllegalStateException e) { } catch (final IllegalStateException e) {
// no compatible bundles... no additional auth checks necessary... if created, will be ghosted // no compatible bundles... no additional auth checks necessary... if created, will be ghosted
} }
@ -638,7 +620,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
snippet.getControllerServices().forEach(controllerService -> { snippet.getControllerServices().forEach(controllerService -> {
try { try {
final BundleCoordinate bundle = BundleUtils.getCompatibleBundle(controllerService.getType(), controllerService.getBundle()); final BundleCoordinate bundle = BundleUtils.getCompatibleBundle(controllerService.getType(), controllerService.getBundle());
controllerServices.add(getControllerServiceByType(controllerService.getType(), new BundleDTO(bundle.getGroup(), bundle.getId(), bundle.getVersion()))); controllerServices.add(getConfigurableComponent(controllerService.getType(), new BundleDTO(bundle.getGroup(), bundle.getId(), bundle.getVersion())));
} catch (final IllegalStateException e) { } catch (final IllegalStateException e) {
// no compatible bundles... no additional auth checks necessary... if created, will be ghosted // no compatible bundles... no additional auth checks necessary... if created, will be ghosted
} }
@ -658,20 +640,20 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
@Override @Override
public TemplateContentsAuthorizable getTemplateContents(final FlowSnippetDTO snippet) { public TemplateContentsAuthorizable getTemplateContents(final FlowSnippetDTO snippet) {
// templates are immutable so we can pre-compute all encapsulated processors and controller services // templates are immutable so we can pre-compute all encapsulated processors and controller services
final Set<ConfigurableComponentAuthorizable> processors = new HashSet<>(); final Set<ComponentAuthorizable> processors = new HashSet<>();
final Set<ConfigurableComponentAuthorizable> controllerServices = new HashSet<>(); final Set<ComponentAuthorizable> controllerServices = new HashSet<>();
// find all processors and controller services // find all processors and controller services
createTemporaryProcessorsAndControllerServices(snippet, processors, controllerServices); createTemporaryProcessorsAndControllerServices(snippet, processors, controllerServices);
return new TemplateContentsAuthorizable() { return new TemplateContentsAuthorizable() {
@Override @Override
public Set<ConfigurableComponentAuthorizable> getEncapsulatedProcessors() { public Set<ComponentAuthorizable> getEncapsulatedProcessors() {
return processors; return processors;
} }
@Override @Override
public Set<ConfigurableComponentAuthorizable> getEncapsulatedControllerServices() { public Set<ComponentAuthorizable> getEncapsulatedControllerServices() {
return controllerServices; return controllerServices;
} }
}; };
@ -700,12 +682,54 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
} }
/** /**
* ConfigurableComponentAuthorizable for a ProcessorNode. * ComponentAuthorizable for a ConfigurableComponent. This authorizable is intended only to be used when
* creating new components.
*/ */
private static class ProcessorConfigurableComponentAuthorizable implements ConfigurableComponentAuthorizable { private static class ConfigurableComponentAuthorizable implements ComponentAuthorizable {
private final ConfigurableComponent configurableComponent;
public ConfigurableComponentAuthorizable(final ConfigurableComponent configurableComponent) {
this.configurableComponent = configurableComponent;
}
@Override
public Authorizable getAuthorizable() {
throw new UnsupportedOperationException();
}
@Override
public boolean isRestricted() {
return configurableComponent.getClass().isAnnotationPresent(Restricted.class);
}
@Override
public String getValue(PropertyDescriptor propertyDescriptor) {
return null;
}
@Override
public PropertyDescriptor getPropertyDescriptor(String propertyName) {
return configurableComponent.getPropertyDescriptor(propertyName);
}
@Override
public List<PropertyDescriptor> getPropertyDescriptors() {
return configurableComponent.getPropertyDescriptors();
}
@Override
public void cleanUpResources() {
ExtensionManager.removeInstanceClassLoader(configurableComponent.getIdentifier());
}
}
/**
* ComponentAuthorizable for a ProcessorNode.
*/
private static class ProcessorComponentAuthorizable implements ComponentAuthorizable {
private final ProcessorNode processorNode; private final ProcessorNode processorNode;
public ProcessorConfigurableComponentAuthorizable(ProcessorNode processorNode) { public ProcessorComponentAuthorizable(ProcessorNode processorNode) {
this.processorNode = processorNode; this.processorNode = processorNode;
} }
@ -736,17 +760,17 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
@Override @Override
public void cleanUpResources() { public void cleanUpResources() {
ExtensionManager.removeInstanceClassLoaderIfExists(processorNode.getIdentifier()); ExtensionManager.removeInstanceClassLoader(processorNode.getIdentifier());
} }
} }
/** /**
* ConfigurableComponentAuthorizable for a ControllerServiceNode. * ComponentAuthorizable for a ControllerServiceNode.
*/ */
private static class ControllerServiceConfigurableComponentAuthorizable implements ConfigurableComponentAuthorizable { private static class ControllerServiceComponentAuthorizable implements ComponentAuthorizable {
private final ControllerServiceNode controllerServiceNode; private final ControllerServiceNode controllerServiceNode;
public ControllerServiceConfigurableComponentAuthorizable(ControllerServiceNode controllerServiceNode) { public ControllerServiceComponentAuthorizable(ControllerServiceNode controllerServiceNode) {
this.controllerServiceNode = controllerServiceNode; this.controllerServiceNode = controllerServiceNode;
} }
@ -777,17 +801,17 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
@Override @Override
public void cleanUpResources() { public void cleanUpResources() {
ExtensionManager.removeInstanceClassLoaderIfExists(controllerServiceNode.getIdentifier()); ExtensionManager.removeInstanceClassLoader(controllerServiceNode.getIdentifier());
} }
} }
/** /**
* ConfigurableComponentAuthorizable for a ProcessorNode. * ComponentAuthorizable for a ProcessorNode.
*/ */
private static class ReportingTaskConfigurableComponentAuthorizable implements ConfigurableComponentAuthorizable { private static class ReportingTaskComponentAuthorizable implements ComponentAuthorizable {
private final ReportingTaskNode reportingTaskNode; private final ReportingTaskNode reportingTaskNode;
public ReportingTaskConfigurableComponentAuthorizable(ReportingTaskNode reportingTaskNode) { public ReportingTaskComponentAuthorizable(ReportingTaskNode reportingTaskNode) {
this.reportingTaskNode = reportingTaskNode; this.reportingTaskNode = reportingTaskNode;
} }
@ -818,7 +842,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
@Override @Override
public void cleanUpResources() { public void cleanUpResources() {
ExtensionManager.removeInstanceClassLoaderIfExists(reportingTaskNode.getIdentifier()); ExtensionManager.removeInstanceClassLoader(reportingTaskNode.getIdentifier());
} }
} }
@ -835,9 +859,9 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
} }
@Override @Override
public Set<ConfigurableComponentAuthorizable> getEncapsulatedProcessors() { public Set<ComponentAuthorizable> getEncapsulatedProcessors() {
return processGroup.findAllProcessors().stream().map( return processGroup.findAllProcessors().stream().map(
processorNode -> new ProcessorConfigurableComponentAuthorizable(processorNode)).collect(Collectors.toSet()); processorNode -> new ProcessorComponentAuthorizable(processorNode)).collect(Collectors.toSet());
} }
@Override @Override
@ -883,9 +907,9 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
} }
@Override @Override
public Set<ConfigurableComponentAuthorizable> getEncapsulatedControllerServices() { public Set<ComponentAuthorizable> getEncapsulatedControllerServices() {
return processGroup.findAllControllerServices().stream().map( return processGroup.findAllControllerServices().stream().map(
controllerServiceNode -> new ControllerServiceConfigurableComponentAuthorizable(controllerServiceNode)).collect(Collectors.toSet()); controllerServiceNode -> new ControllerServiceComponentAuthorizable(controllerServiceNode)).collect(Collectors.toSet());
} }
} }

View File

@ -27,13 +27,13 @@ public interface TemplateContentsAuthorizable {
* *
* @return temporary instances of all encapsulated processors * @return temporary instances of all encapsulated processors
*/ */
Set<ConfigurableComponentAuthorizable> getEncapsulatedProcessors(); Set<ComponentAuthorizable> getEncapsulatedProcessors();
/** /**
* Returns temporary instances of all encapsulated controller services. Non null * Returns temporary instances of all encapsulated controller services. Non null
* *
* @return temporary instances of all encapsulated controller services * @return temporary instances of all encapsulated controller services
*/ */
Set<ConfigurableComponentAuthorizable> getEncapsulatedControllerServices(); Set<ComponentAuthorizable> getEncapsulatedControllerServices();
} }

View File

@ -52,6 +52,7 @@ import org.apache.nifi.cluster.event.NodeEvent;
import org.apache.nifi.cluster.manager.exception.IllegalNodeDeletionException; import org.apache.nifi.cluster.manager.exception.IllegalNodeDeletionException;
import org.apache.nifi.cluster.manager.exception.UnknownNodeException; import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator; import org.apache.nifi.components.Validator;
@ -83,6 +84,7 @@ import org.apache.nifi.groups.RemoteProcessGroup;
import org.apache.nifi.history.History; import org.apache.nifi.history.History;
import org.apache.nifi.history.HistoryQuery; import org.apache.nifi.history.HistoryQuery;
import org.apache.nifi.history.PreviousValue; import org.apache.nifi.history.PreviousValue;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.remote.RootGroupPort; import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.reporting.Bulletin; import org.apache.nifi.reporting.Bulletin;
import org.apache.nifi.reporting.BulletinQuery; import org.apache.nifi.reporting.BulletinQuery;
@ -1678,8 +1680,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
} }
try { try {
final ControllerService controllerService = controllerFacade.createTemporaryControllerService(dto.getType(), dto.getBundle()).getControllerServiceImplementation(); final ConfigurableComponent configurableComponent = controllerFacade.getTemporaryComponent(dto.getType(), dto.getBundle());
controllerService.getPropertyDescriptors().forEach(descriptor -> { configurableComponent.getPropertyDescriptors().forEach(descriptor -> {
if (dto.getProperties().get(descriptor.getName()) == null) { if (dto.getProperties().get(descriptor.getName()) == null) {
dto.getProperties().put(descriptor.getName(), descriptor.getDefaultValue()); dto.getProperties().put(descriptor.getName(), descriptor.getDefaultValue());
} }
@ -1702,8 +1704,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
} }
try { try {
final ProcessorNode processorNode = controllerFacade.createTemporaryProcessor(dto.getType(), dto.getBundle()); final ConfigurableComponent configurableComponent = controllerFacade.getTemporaryComponent(dto.getType(), dto.getBundle());
processorNode.getPropertyDescriptors().forEach(descriptor -> { configurableComponent.getPropertyDescriptors().forEach(descriptor -> {
if (config.getProperties().get(descriptor.getName()) == null) { if (config.getProperties().get(descriptor.getName()) == null) {
config.getProperties().put(descriptor.getName(), descriptor.getDefaultValue()); config.getProperties().put(descriptor.getName(), descriptor.getDefaultValue());
} }

View File

@ -45,7 +45,7 @@ import org.apache.nifi.authorization.AuthorizationResult;
import org.apache.nifi.authorization.AuthorizationResult.Result; import org.apache.nifi.authorization.AuthorizationResult.Result;
import org.apache.nifi.authorization.AuthorizeControllerServiceReference; import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
import org.apache.nifi.authorization.Authorizer; import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.ConfigurableComponentAuthorizable; import org.apache.nifi.authorization.ComponentAuthorizable;
import org.apache.nifi.authorization.RequestAction; import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.UserContextKeys; import org.apache.nifi.authorization.UserContextKeys;
import org.apache.nifi.authorization.resource.Authorizable; import org.apache.nifi.authorization.resource.Authorizable;
@ -396,7 +396,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
// authorize access // authorize access
serviceFacade.authorizeAccess(lookup -> { serviceFacade.authorizeAccess(lookup -> {
// authorize the processor // authorize the processor
final ConfigurableComponentAuthorizable authorizable = lookup.getProcessor(id); final ComponentAuthorizable authorizable = lookup.getProcessor(id);
authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
// authorize any referenced service // authorize any referenced service
@ -587,7 +587,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
// authorize access // authorize access
serviceFacade.authorizeAccess(lookup -> { serviceFacade.authorizeAccess(lookup -> {
// authorize the controller service // authorize the controller service
final ConfigurableComponentAuthorizable authorizable = lookup.getControllerService(id); final ComponentAuthorizable authorizable = lookup.getControllerService(id);
authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
// authorize any referenced service // authorize any referenced service
@ -752,7 +752,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
// authorize access // authorize access
serviceFacade.authorizeAccess(lookup -> { serviceFacade.authorizeAccess(lookup -> {
// authorize the reporting task // authorize the reporting task
final ConfigurableComponentAuthorizable authorizable = lookup.getReportingTask(id); final ComponentAuthorizable authorizable = lookup.getReportingTask(id);
authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
// authorize any referenced service // authorize any referenced service

View File

@ -30,7 +30,7 @@ import org.apache.nifi.authorization.AuthorizationResult;
import org.apache.nifi.authorization.AuthorizationResult.Result; import org.apache.nifi.authorization.AuthorizationResult.Result;
import org.apache.nifi.authorization.AuthorizeControllerServiceReference; import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
import org.apache.nifi.authorization.Authorizer; import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.ConfigurableComponentAuthorizable; import org.apache.nifi.authorization.ComponentAuthorizable;
import org.apache.nifi.authorization.RequestAction; import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.UserContextKeys; import org.apache.nifi.authorization.UserContextKeys;
import org.apache.nifi.authorization.resource.ResourceFactory; import org.apache.nifi.authorization.resource.ResourceFactory;
@ -295,9 +295,9 @@ public class ControllerResource extends ApplicationResource {
lookup -> { lookup -> {
authorizeController(RequestAction.WRITE); authorizeController(RequestAction.WRITE);
ConfigurableComponentAuthorizable authorizable = null; ComponentAuthorizable authorizable = null;
try { try {
authorizable = lookup.getReportingTaskByType(requestReportingTask.getType(), requestReportingTask.getBundle()); authorizable = lookup.getConfigurableComponent(requestReportingTask.getType(), requestReportingTask.getBundle());
if (authorizable.isRestricted()) { if (authorizable.isRestricted()) {
lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
@ -401,9 +401,9 @@ public class ControllerResource extends ApplicationResource {
lookup -> { lookup -> {
authorizeController(RequestAction.WRITE); authorizeController(RequestAction.WRITE);
ConfigurableComponentAuthorizable authorizable = null; ComponentAuthorizable authorizable = null;
try { try {
authorizable = lookup.getControllerServiceByType(requestControllerService.getType(), requestControllerService.getBundle()); authorizable = lookup.getConfigurableComponent(requestControllerService.getType(), requestControllerService.getBundle());
if (authorizable.isRestricted()) { if (authorizable.isRestricted()) {
lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());

View File

@ -25,7 +25,7 @@ import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.AuthorizeControllerServiceReference; import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
import org.apache.nifi.authorization.Authorizer; import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.ConfigurableComponentAuthorizable; import org.apache.nifi.authorization.ComponentAuthorizable;
import org.apache.nifi.authorization.RequestAction; import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable; import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.authorization.user.NiFiUserUtils; import org.apache.nifi.authorization.user.NiFiUserUtils;
@ -624,7 +624,7 @@ public class ControllerServiceResource extends ApplicationResource {
requestRevision, requestRevision,
lookup -> { lookup -> {
// authorize the service // authorize the service
final ConfigurableComponentAuthorizable authorizable = lookup.getControllerService(id); final ComponentAuthorizable authorizable = lookup.getControllerService(id);
authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
// authorize any referenced services // authorize any referenced services
@ -710,7 +710,7 @@ public class ControllerServiceResource extends ApplicationResource {
requestControllerServiceEntity, requestControllerServiceEntity,
requestRevision, requestRevision,
lookup -> { lookup -> {
final ConfigurableComponentAuthorizable controllerService = lookup.getControllerService(id); final ComponentAuthorizable controllerService = lookup.getControllerService(id);
// ensure write permission to the controller service // ensure write permission to the controller service
controllerService.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); controllerService.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());

View File

@ -28,7 +28,7 @@ import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.AuthorizableLookup; import org.apache.nifi.authorization.AuthorizableLookup;
import org.apache.nifi.authorization.AuthorizeControllerServiceReference; import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
import org.apache.nifi.authorization.Authorizer; import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.ConfigurableComponentAuthorizable; import org.apache.nifi.authorization.ComponentAuthorizable;
import org.apache.nifi.authorization.ProcessGroupAuthorizable; import org.apache.nifi.authorization.ProcessGroupAuthorizable;
import org.apache.nifi.authorization.RequestAction; import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.SnippetAuthorizable; import org.apache.nifi.authorization.SnippetAuthorizable;
@ -657,9 +657,9 @@ public class ProcessGroupResource extends ApplicationResource {
final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable(); final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable();
processGroup.authorize(authorizer, RequestAction.WRITE, user); processGroup.authorize(authorizer, RequestAction.WRITE, user);
ConfigurableComponentAuthorizable authorizable = null; ComponentAuthorizable authorizable = null;
try { try {
authorizable = lookup.getProcessorByType(requestProcessor.getType(), requestProcessor.getBundle()); authorizable = lookup.getConfigurableComponent(requestProcessor.getType(), requestProcessor.getBundle());
if (authorizable.isRestricted()) { if (authorizable.isRestricted()) {
lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, user); lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, user);
@ -1806,7 +1806,7 @@ public class ProcessGroupResource extends ApplicationResource {
// flag to only perform the restricted check once, atomic reference so we can mark final and use in lambda // flag to only perform the restricted check once, atomic reference so we can mark final and use in lambda
final AtomicBoolean restrictedCheckPerformed = new AtomicBoolean(false); final AtomicBoolean restrictedCheckPerformed = new AtomicBoolean(false);
final Consumer<ConfigurableComponentAuthorizable> authorizeRestricted = authorizable -> { final Consumer<ComponentAuthorizable> authorizeRestricted = authorizable -> {
if (authorizable.isRestricted() && restrictedCheckPerformed.compareAndSet(false, true)) { if (authorizable.isRestricted() && restrictedCheckPerformed.compareAndSet(false, true)) {
lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, user); lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, user);
} }
@ -1982,7 +1982,7 @@ public class ProcessGroupResource extends ApplicationResource {
// flag to only perform the restricted check once, atomic reference so we can mark final and use in lambda // flag to only perform the restricted check once, atomic reference so we can mark final and use in lambda
final AtomicBoolean restrictedCheckPerformed = new AtomicBoolean(false); final AtomicBoolean restrictedCheckPerformed = new AtomicBoolean(false);
final Consumer<ConfigurableComponentAuthorizable> authorizeRestricted = authorizable -> { final Consumer<ComponentAuthorizable> authorizeRestricted = authorizable -> {
if (authorizable.isRestricted() && restrictedCheckPerformed.compareAndSet(false, true)) { if (authorizable.isRestricted() && restrictedCheckPerformed.compareAndSet(false, true)) {
lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, user); lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, user);
} }
@ -2347,9 +2347,9 @@ public class ProcessGroupResource extends ApplicationResource {
final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable(); final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable();
processGroup.authorize(authorizer, RequestAction.WRITE, user); processGroup.authorize(authorizer, RequestAction.WRITE, user);
ConfigurableComponentAuthorizable authorizable = null; ComponentAuthorizable authorizable = null;
try { try {
authorizable = lookup.getControllerServiceByType(requestControllerService.getType(), requestControllerService.getBundle()); authorizable = lookup.getConfigurableComponent(requestControllerService.getType(), requestControllerService.getBundle());
if (authorizable.isRestricted()) { if (authorizable.isRestricted()) {
lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, user); lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, user);

View File

@ -25,7 +25,7 @@ import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.AuthorizeControllerServiceReference; import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
import org.apache.nifi.authorization.Authorizer; import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.ConfigurableComponentAuthorizable; import org.apache.nifi.authorization.ComponentAuthorizable;
import org.apache.nifi.authorization.RequestAction; import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable; import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.authorization.user.NiFiUser; import org.apache.nifi.authorization.user.NiFiUser;
@ -459,7 +459,7 @@ public class ProcessorResource extends ApplicationResource {
lookup -> { lookup -> {
final NiFiUser user = NiFiUserUtils.getNiFiUser(); final NiFiUser user = NiFiUserUtils.getNiFiUser();
final ConfigurableComponentAuthorizable authorizable = lookup.getProcessor(id); final ComponentAuthorizable authorizable = lookup.getProcessor(id);
authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, user); authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, user);
final ProcessorConfigDTO config = requestProcessorDTO.getConfig(); final ProcessorConfigDTO config = requestProcessorDTO.getConfig();
@ -543,7 +543,7 @@ public class ProcessorResource extends ApplicationResource {
requestProcessorEntity, requestProcessorEntity,
requestRevision, requestRevision,
lookup -> { lookup -> {
final ConfigurableComponentAuthorizable processor = lookup.getProcessor(id); final ComponentAuthorizable processor = lookup.getProcessor(id);
// ensure write permission to the processor // ensure write permission to the processor
processor.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); processor.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());

View File

@ -25,7 +25,7 @@ import com.wordnik.swagger.annotations.Authorization;
import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.AuthorizeControllerServiceReference; import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
import org.apache.nifi.authorization.Authorizer; import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.ConfigurableComponentAuthorizable; import org.apache.nifi.authorization.ComponentAuthorizable;
import org.apache.nifi.authorization.RequestAction; import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.resource.Authorizable; import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.authorization.user.NiFiUserUtils; import org.apache.nifi.authorization.user.NiFiUserUtils;
@ -428,7 +428,7 @@ public class ReportingTaskResource extends ApplicationResource {
requestRevision, requestRevision,
lookup -> { lookup -> {
// authorize reporting task // authorize reporting task
final ConfigurableComponentAuthorizable authorizable = lookup.getReportingTask(id); final ComponentAuthorizable authorizable = lookup.getReportingTask(id);
authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
// authorize any referenced services // authorize any referenced services
@ -513,7 +513,7 @@ public class ReportingTaskResource extends ApplicationResource {
requestReportingTaskEntity, requestReportingTaskEntity,
requestRevision, requestRevision,
lookup -> { lookup -> {
final ConfigurableComponentAuthorizable reportingTask = lookup.getReportingTask(id); final ComponentAuthorizable reportingTask = lookup.getReportingTask(id);
// ensure write permission to the reporting task // ensure write permission to the reporting task
reportingTask.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); reportingTask.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());

View File

@ -32,6 +32,7 @@ import org.apache.nifi.authorization.user.NiFiUserUtils;
import org.apache.nifi.bundle.Bundle; import org.apache.nifi.bundle.Bundle;
import org.apache.nifi.bundle.BundleCoordinate; import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.Connection; import org.apache.nifi.connectable.Connection;
@ -45,11 +46,9 @@ import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.Template; import org.apache.nifi.controller.Template;
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
import org.apache.nifi.controller.label.Label; import org.apache.nifi.controller.label.Label;
import org.apache.nifi.controller.queue.FlowFileQueue; import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.queue.QueueSize; import org.apache.nifi.controller.queue.QueueSize;
import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
import org.apache.nifi.controller.repository.ContentNotFoundException; import org.apache.nifi.controller.repository.ContentNotFoundException;
import org.apache.nifi.controller.repository.claim.ContentDirection; import org.apache.nifi.controller.repository.claim.ContentDirection;
import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceNode;
@ -134,7 +133,6 @@ import java.util.Set;
import java.util.SortedSet; import java.util.SortedSet;
import java.util.TimeZone; import java.util.TimeZone;
import java.util.TreeSet; import java.util.TreeSet;
import java.util.UUID;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.function.Consumer; import java.util.function.Consumer;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -200,35 +198,21 @@ public class ControllerFacade implements Authorizable {
} }
/** /**
* Create a temporary Processor used for extracting PropertyDescriptor's for ControllerService reference authorization. * Gets the cached temporary instance of the component for the given type and bundle.
* *
* @param type type of processor * @param type type of the component
* @return processor * @param bundle the bundle of the component
* @throws ProcessorInstantiationException when unable to instantiate the processor * @return the temporary component
* @throws IllegalStateException if no temporary component exists for the given type and bundle
*/ */
public ProcessorNode createTemporaryProcessor(String type, BundleDTO bundle) throws ProcessorInstantiationException { public ConfigurableComponent getTemporaryComponent(final String type, final BundleDTO bundle) {
return flowController.createProcessor(type, UUID.randomUUID().toString(), BundleUtils.getBundle(type, bundle), false); final ConfigurableComponent configurableComponent = ExtensionManager.getTempComponent(type, BundleUtils.getBundle(type, bundle));
}
/** if (configurableComponent == null) {
* Create a temporary ReportingTask used for extracting PropertyDescriptor's for ControllerService reference authorization. throw new IllegalStateException("Unable to obtain temporary component for " + type);
* }
* @param type type of reporting task
* @return reporting task
* @throws ReportingTaskInstantiationException when unable to instantiate the reporting task
*/
public ReportingTaskNode createTemporaryReportingTask(String type, BundleDTO bundle) throws ReportingTaskInstantiationException {
return flowController.createReportingTask(type, UUID.randomUUID().toString(), BundleUtils.getBundle(type, bundle), false, false);
}
/** return configurableComponent;
* Create a temporary ControllerService used for extracting PropertyDescriptor's for ControllerService reference authorization.
*
* @param type type of controller service
* @return controller service
*/
public ControllerServiceNode createTemporaryControllerService(String type, BundleDTO bundle) {
return flowController.createControllerService(type, UUID.randomUUID().toString(), BundleUtils.getBundle(type, bundle), false);
} }
/** /**

View File

@ -77,7 +77,8 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
try { try {
// create the controller service // create the controller service
final ControllerServiceNode controllerService = serviceProvider.createControllerService( final ControllerServiceNode controllerService = serviceProvider.createControllerService(
controllerServiceDTO.getType(), controllerServiceDTO.getId(), BundleUtils.getBundle(controllerServiceDTO.getType(), controllerServiceDTO.getBundle()), true); controllerServiceDTO.getType(), controllerServiceDTO.getId(), BundleUtils.getBundle(controllerServiceDTO.getType(),
controllerServiceDTO.getBundle()), Collections.emptySet(), true);
// ensure we can perform the update // ensure we can perform the update
verifyUpdate(controllerService, controllerServiceDTO); verifyUpdate(controllerService, controllerServiceDTO);
@ -170,7 +171,7 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
if (bundleDTO != null) { if (bundleDTO != null) {
final BundleCoordinate incomingCoordinate = BundleUtils.getBundle(controllerService.getCanonicalClassName(), bundleDTO); final BundleCoordinate incomingCoordinate = BundleUtils.getBundle(controllerService.getCanonicalClassName(), bundleDTO);
try { try {
flowController.changeControllerServiceType(controllerService, controllerService.getCanonicalClassName(), incomingCoordinate); flowController.reload(controllerService, controllerService.getCanonicalClassName(), incomingCoordinate, Collections.emptySet());
} catch (ControllerServiceInstantiationException e) { } catch (ControllerServiceInstantiationException e) {
throw new NiFiCoreException(String.format("Unable to update controller service %s from %s to %s due to: %s", throw new NiFiCoreException(String.format("Unable to update controller service %s from %s to %s due to: %s",
controllerServiceDTO.getId(), controllerService.getBundleCoordinate().getCoordinate(), incomingCoordinate.getCoordinate(), e.getMessage()), e); controllerServiceDTO.getId(), controllerService.getBundleCoordinate().getCoordinate(), incomingCoordinate.getCoordinate(), e.getMessage()), e);

View File

@ -48,6 +48,7 @@ import org.slf4j.LoggerFactory;
import java.text.ParseException; import java.text.ParseException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -453,7 +454,7 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO {
if (bundleDTO != null) { if (bundleDTO != null) {
BundleCoordinate incomingCoordinate = BundleUtils.getBundle(processor.getCanonicalClassName(), bundleDTO); BundleCoordinate incomingCoordinate = BundleUtils.getBundle(processor.getCanonicalClassName(), bundleDTO);
try { try {
flowController.changeProcessorType(processor, processor.getCanonicalClassName(), incomingCoordinate); flowController.reload(processor, processor.getCanonicalClassName(), incomingCoordinate, Collections.emptySet());
} catch (ProcessorInstantiationException e) { } catch (ProcessorInstantiationException e) {
throw new NiFiCoreException(String.format("Unable to update processor %s from %s to %s due to: %s", throw new NiFiCoreException(String.format("Unable to update processor %s from %s to %s due to: %s",
processorDTO.getId(), processor.getBundleCoordinate().getCoordinate(), incomingCoordinate.getCoordinate(), e.getMessage()), e); processorDTO.getId(), processor.getBundleCoordinate().getCoordinate(), incomingCoordinate.getCoordinate(), e.getMessage()), e);

View File

@ -20,6 +20,7 @@ import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.bundle.BundleCoordinate; import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.components.state.Scope; import org.apache.nifi.components.state.Scope;
import org.apache.nifi.components.state.StateMap; import org.apache.nifi.components.state.StateMap;
import org.apache.nifi.controller.ReloadComponent;
import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.exception.ComponentLifeCycleException; import org.apache.nifi.controller.exception.ComponentLifeCycleException;
@ -39,6 +40,7 @@ import org.quartz.CronExpression;
import java.text.ParseException; import java.text.ParseException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
@ -49,6 +51,7 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
private ReportingTaskProvider reportingTaskProvider; private ReportingTaskProvider reportingTaskProvider;
private ComponentStateDAO componentStateDAO; private ComponentStateDAO componentStateDAO;
private ReloadComponent reloadComponent;
private ReportingTaskNode locateReportingTask(final String reportingTaskId) { private ReportingTaskNode locateReportingTask(final String reportingTaskId) {
// get the reporting task // get the reporting task
@ -167,7 +170,7 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
if (bundleDTO != null) { if (bundleDTO != null) {
final BundleCoordinate incomingCoordinate = BundleUtils.getBundle(reportingTask.getCanonicalClassName(), bundleDTO); final BundleCoordinate incomingCoordinate = BundleUtils.getBundle(reportingTask.getCanonicalClassName(), bundleDTO);
try { try {
reportingTaskProvider.changeReportingTaskType(reportingTask, reportingTask.getCanonicalClassName(), incomingCoordinate); reloadComponent.reload(reportingTask, reportingTask.getCanonicalClassName(), incomingCoordinate, Collections.emptySet());
} catch (ReportingTaskInstantiationException e) { } catch (ReportingTaskInstantiationException e) {
throw new NiFiCoreException(String.format("Unable to update reporting task %s from %s to %s due to: %s", throw new NiFiCoreException(String.format("Unable to update reporting task %s from %s to %s due to: %s",
reportingTaskDTO.getId(), reportingTask.getBundleCoordinate().getCoordinate(), incomingCoordinate.getCoordinate(), e.getMessage()), e); reportingTaskDTO.getId(), reportingTask.getBundleCoordinate().getCoordinate(), incomingCoordinate.getCoordinate(), e.getMessage()), e);
@ -356,4 +359,8 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
public void setComponentStateDAO(ComponentStateDAO componentStateDAO) { public void setComponentStateDAO(ComponentStateDAO componentStateDAO) {
this.componentStateDAO = componentStateDAO; this.componentStateDAO = componentStateDAO;
} }
public void setReloadComponent(ReloadComponent reloadComponent) {
this.reloadComponent = reloadComponent;
}
} }

View File

@ -97,6 +97,7 @@
<bean id="reportingTaskDAO" class="org.apache.nifi.web.dao.impl.StandardReportingTaskDAO"> <bean id="reportingTaskDAO" class="org.apache.nifi.web.dao.impl.StandardReportingTaskDAO">
<property name="reportingTaskProvider" ref="reportingTaskProvider"/> <property name="reportingTaskProvider" ref="reportingTaskProvider"/>
<property name="componentStateDAO" ref="componentStateDAO"/> <property name="componentStateDAO" ref="componentStateDAO"/>
<property name="reloadComponent" ref="flowController" />
</bean> </bean>
<bean id="componentStateDAO" class="org.apache.nifi.web.dao.impl.StandardComponentStateDAO"> <bean id="componentStateDAO" class="org.apache.nifi.web.dao.impl.StandardComponentStateDAO">
<property name="stateManagerProvider" ref="stateManagerProvider"/> <property name="stateManagerProvider" ref="stateManagerProvider"/>

View File

@ -24,6 +24,7 @@ import org.apache.nifi.integration.util.NiFiTestServer;
import org.apache.nifi.integration.util.NiFiTestUser; import org.apache.nifi.integration.util.NiFiTestUser;
import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.NarClassLoaders; import org.apache.nifi.nar.NarClassLoaders;
import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.NiFiProperties;
import java.io.File; import java.io.File;
@ -64,7 +65,7 @@ public class AccessControlHelper {
flowXmlPath = props.getProperty(NiFiProperties.FLOW_CONFIGURATION_FILE); flowXmlPath = props.getProperty(NiFiProperties.FLOW_CONFIGURATION_FILE);
// load extensions // load extensions
final Bundle systemBundle = ExtensionManager.createSystemBundle(props); final Bundle systemBundle = SystemBundle.create(props);
NarClassLoaders.getInstance().init(props.getFrameworkWorkingDirectory(), props.getExtensionsWorkingDirectory()); NarClassLoaders.getInstance().init(props.getFrameworkWorkingDirectory(), props.getExtensionsWorkingDirectory());
ExtensionManager.discoverExtensions(systemBundle, NarClassLoaders.getInstance().getBundles()); ExtensionManager.discoverExtensions(systemBundle, NarClassLoaders.getInstance().getBundles());

View File

@ -25,6 +25,7 @@ import org.apache.nifi.integration.util.NiFiTestUser;
import org.apache.nifi.integration.util.SourceTestProcessor; import org.apache.nifi.integration.util.SourceTestProcessor;
import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.NarClassLoaders; import org.apache.nifi.nar.NarClassLoaders;
import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.security.util.SslContextFactory; import org.apache.nifi.security.util.SslContextFactory;
import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.api.dto.AccessConfigurationDTO; import org.apache.nifi.web.api.dto.AccessConfigurationDTO;
@ -71,7 +72,7 @@ public class ITAccessTokenEndpoint {
FileUtils.deleteDirectory(props.getDatabaseRepositoryPath().toFile()); FileUtils.deleteDirectory(props.getDatabaseRepositoryPath().toFile());
// load extensions // load extensions
final Bundle systemBundle = ExtensionManager.createSystemBundle(props); final Bundle systemBundle = SystemBundle.create(props);
NarClassLoaders.getInstance().init(props.getFrameworkWorkingDirectory(), props.getExtensionsWorkingDirectory()); NarClassLoaders.getInstance().init(props.getFrameworkWorkingDirectory(), props.getExtensionsWorkingDirectory());
ExtensionManager.discoverExtensions(systemBundle, NarClassLoaders.getInstance().getBundles()); ExtensionManager.discoverExtensions(systemBundle, NarClassLoaders.getInstance().getBundles());

View File

@ -25,7 +25,7 @@ import org.apache.nifi.authorization.AuthorizableLookup;
import org.apache.nifi.authorization.AuthorizationRequest; import org.apache.nifi.authorization.AuthorizationRequest;
import org.apache.nifi.authorization.AuthorizationResult; import org.apache.nifi.authorization.AuthorizationResult;
import org.apache.nifi.authorization.Authorizer; import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.authorization.ConfigurableComponentAuthorizable; import org.apache.nifi.authorization.ComponentAuthorizable;
import org.apache.nifi.authorization.Resource; import org.apache.nifi.authorization.Resource;
import org.apache.nifi.authorization.resource.Authorizable; import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.authorization.resource.ResourceFactory; import org.apache.nifi.authorization.resource.ResourceFactory;
@ -114,7 +114,7 @@ public class StandardNiFiServiceFacadeTest {
} }
// component authorizable // component authorizable
final ConfigurableComponentAuthorizable componentAuthorizable = mock(ConfigurableComponentAuthorizable.class); final ComponentAuthorizable componentAuthorizable = mock(ComponentAuthorizable.class);
when(componentAuthorizable.getAuthorizable()).then(getAuthorizableInvocation -> { when(componentAuthorizable.getAuthorizable()).then(getAuthorizableInvocation -> {
// authorizable // authorizable

View File

@ -32,6 +32,7 @@
<module>nifi-framework-core</module> <module>nifi-framework-core</module>
<module>nifi-framework-cluster-protocol</module> <module>nifi-framework-cluster-protocol</module>
<module>nifi-framework-cluster</module> <module>nifi-framework-cluster</module>
<module>nifi-framework-nar-utils</module>
<module>nifi-user-actions</module> <module>nifi-user-actions</module>
<module>nifi-framework-authorization</module> <module>nifi-framework-authorization</module>
<module>nifi-file-authorizer</module> <module>nifi-file-authorizer</module>

View File

@ -63,6 +63,11 @@
<artifactId>nifi-framework-core-api</artifactId> <artifactId>nifi-framework-core-api</artifactId>
<version>1.2.0-SNAPSHOT</version> <version>1.2.0-SNAPSHOT</version>
</dependency> </dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-framework-nar-utils</artifactId>
<version>1.2.0-SNAPSHOT</version>
</dependency>
<dependency> <dependency>
<groupId>org.apache.nifi</groupId> <groupId>org.apache.nifi</groupId>
<artifactId>nifi-site-to-site</artifactId> <artifactId>nifi-site-to-site</artifactId>

View File

@ -28,7 +28,7 @@
<dependencies> <dependencies>
<dependency> <dependency>
<groupId>org.apache.nifi</groupId> <groupId>org.apache.nifi</groupId>
<artifactId>nifi-standard-services-api-nar</artifactId> <artifactId>nifi-hadoop-libraries-nar</artifactId>
<type>nar</type> <type>nar</type>
</dependency> </dependency>
<dependency> <dependency>

View File

@ -44,14 +44,20 @@
<groupId>org.apache.nifi</groupId> <groupId>org.apache.nifi</groupId>
<artifactId>nifi-flowfile-packager</artifactId> <artifactId>nifi-flowfile-packager</artifactId>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.apache.hadoop</groupId> <groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId> <artifactId>hadoop-common</artifactId>
<scope>provided</scope>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.apache.nifi</groupId> <groupId>org.apache.nifi</groupId>
<artifactId>nifi-distributed-cache-client-service-api</artifactId> <artifactId>nifi-distributed-cache-client-service-api</artifactId>
</dependency> </dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId>
<scope>provided</scope>
</dependency>
<dependency> <dependency>
<groupId>org.apache.nifi</groupId> <groupId>org.apache.nifi</groupId>
<artifactId>nifi-mock</artifactId> <artifactId>nifi-mock</artifactId>

View File

@ -37,6 +37,7 @@ import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator; import org.apache.nifi.components.Validator;
import org.apache.nifi.hadoop.KerberosProperties; import org.apache.nifi.hadoop.KerberosProperties;
import org.apache.nifi.hadoop.SecurityUtil; import org.apache.nifi.hadoop.SecurityUtil;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.ProcessorInitializationContext;
@ -45,7 +46,6 @@ import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.util.StringUtils; import org.apache.nifi.util.StringUtils;
import javax.net.SocketFactory; import javax.net.SocketFactory;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.lang.ref.WeakReference; import java.lang.ref.WeakReference;
@ -65,7 +65,7 @@ import java.util.concurrent.atomic.AtomicReference;
/** /**
* This is a base class that is helpful when building processors interacting with HDFS. * This is a base class that is helpful when building processors interacting with HDFS.
*/ */
@RequiresInstanceClassLoading @RequiresInstanceClassLoading(cloneAncestorResources = true)
public abstract class AbstractHadoopProcessor extends AbstractProcessor { public abstract class AbstractHadoopProcessor extends AbstractProcessor {
/** /**
* Compression Type Enum * Compression Type Enum
@ -191,7 +191,9 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
// then load the Configuration and set the new resources in the holder // then load the Configuration and set the new resources in the holder
if (resources == null || !configResources.equals(resources.getConfigResources())) { if (resources == null || !configResources.equals(resources.getConfigResources())) {
getLogger().debug("Reloading validation resources"); getLogger().debug("Reloading validation resources");
resources = new ValidationResources(configResources, getConfigurationFromResources(configResources)); final Configuration config = new ExtendedConfiguration(getLogger());
config.setClassLoader(Thread.currentThread().getContextClassLoader());
resources = new ValidationResources(configResources, getConfigurationFromResources(config, configResources));
validationResourceHolder.set(resources); validationResourceHolder.set(resources);
} }
@ -240,9 +242,8 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
hdfsResources.set(new HdfsResources(null, null, null)); hdfsResources.set(new HdfsResources(null, null, null));
} }
private static Configuration getConfigurationFromResources(String configResources) throws IOException { private static Configuration getConfigurationFromResources(final Configuration config, String configResources) throws IOException {
boolean foundResources = false; boolean foundResources = false;
final Configuration config = new ExtendedConfiguration();
if (null != configResources) { if (null != configResources) {
String[] resources = configResources.split(","); String[] resources = configResources.split(",");
for (String resource : resources) { for (String resource : resources) {
@ -272,8 +273,10 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
* Reset Hadoop Configuration and FileSystem based on the supplied configuration resources. * Reset Hadoop Configuration and FileSystem based on the supplied configuration resources.
*/ */
HdfsResources resetHDFSResources(String configResources, ProcessContext context) throws IOException { HdfsResources resetHDFSResources(String configResources, ProcessContext context) throws IOException {
Configuration config = getConfigurationFromResources(configResources); Configuration config = new ExtendedConfiguration(getLogger());
config.setClassLoader(Thread.currentThread().getContextClassLoader()); // set the InstanceClassLoader config.setClassLoader(Thread.currentThread().getContextClassLoader());
getConfigurationFromResources(config, configResources);
// first check for timeout on HDFS connection, because FileSystem has a hard coded 15 minute timeout // first check for timeout on HDFS connection, because FileSystem has a hard coded 15 minute timeout
checkHdfsUriForTimeout(config); checkHdfsUriForTimeout(config);
@ -531,16 +534,22 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
*/ */
static class ExtendedConfiguration extends Configuration { static class ExtendedConfiguration extends Configuration {
private final ComponentLog logger;
private final Map<ClassLoader, Map<String, WeakReference<Class<?>>>> CACHE_CLASSES = new WeakHashMap<>(); private final Map<ClassLoader, Map<String, WeakReference<Class<?>>>> CACHE_CLASSES = new WeakHashMap<>();
public Class<?> getClassByNameOrNull(String name) { public ExtendedConfiguration(final ComponentLog logger) {
Map<String, WeakReference<Class<?>>> map; this.logger = logger;
}
public Class<?> getClassByNameOrNull(String name) {
final ClassLoader classLoader = getClassLoader();
Map<String, WeakReference<Class<?>>> map;
synchronized (CACHE_CLASSES) { synchronized (CACHE_CLASSES) {
map = CACHE_CLASSES.get(getClassLoader()); map = CACHE_CLASSES.get(classLoader);
if (map == null) { if (map == null) {
map = Collections.synchronizedMap(new WeakHashMap<>()); map = Collections.synchronizedMap(new WeakHashMap<>());
CACHE_CLASSES.put(getClassLoader(), map); CACHE_CLASSES.put(classLoader, map);
} }
} }
@ -552,9 +561,9 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
if (clazz == null) { if (clazz == null) {
try { try {
clazz = Class.forName(name, true, getClassLoader()); clazz = Class.forName(name, true, classLoader);
} catch (ClassNotFoundException e) { } catch (ClassNotFoundException e) {
e.printStackTrace(); logger.error(e.getMessage(), e);
return null; return null;
} }
// two putters can race here, but they'll put the same class // two putters can race here, but they'll put the same class

View File

@ -24,6 +24,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.CompressionCodecFactory; import org.apache.hadoop.io.compress.CompressionCodecFactory;
import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.nifi.annotation.behavior.InputRequirement; import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.Restricted; import org.apache.nifi.annotation.behavior.Restricted;
@ -46,6 +47,7 @@ import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.net.URI; import java.net.URI;
import java.security.PrivilegedAction;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
@ -112,9 +114,10 @@ public class FetchHDFS extends AbstractHadoopProcessor {
} }
final FileSystem hdfs = getFileSystem(); final FileSystem hdfs = getFileSystem();
final UserGroupInformation ugi = getUserGroupInformation();
final String filenameValue = context.getProperty(FILENAME).evaluateAttributeExpressions(flowFile).getValue(); final String filenameValue = context.getProperty(FILENAME).evaluateAttributeExpressions(flowFile).getValue();
Path path = null; final Path path;
try { try {
path = new Path(filenameValue); path = new Path(filenameValue);
} catch (IllegalArgumentException e) { } catch (IllegalArgumentException e) {
@ -125,54 +128,64 @@ public class FetchHDFS extends AbstractHadoopProcessor {
return; return;
} }
InputStream stream = null;
CompressionCodec codec = null;
Configuration conf = getConfiguration();
final CompressionCodecFactory compressionCodecFactory = new CompressionCodecFactory(conf);
final CompressionType compressionType = CompressionType.valueOf(context.getProperty(COMPRESSION_CODEC).toString());
final boolean inferCompressionCodec = compressionType == CompressionType.AUTOMATIC;
if(inferCompressionCodec) {
codec = compressionCodecFactory.getCodec(path);
} else if (compressionType != CompressionType.NONE) {
codec = getCompressionCodec(context, getConfiguration());
}
final URI uri = path.toUri(); final URI uri = path.toUri();
final StopWatch stopWatch = new StopWatch(true); final StopWatch stopWatch = new StopWatch(true);
try { final FlowFile finalFlowFile = flowFile;
final String outputFilename; ugi.doAs(new PrivilegedAction<Object>() {
final String originalFilename = path.getName(); @Override
stream = hdfs.open(path, 16384); public Object run() {
InputStream stream = null;
CompressionCodec codec = null;
Configuration conf = getConfiguration();
final CompressionCodecFactory compressionCodecFactory = new CompressionCodecFactory(conf);
final CompressionType compressionType = CompressionType.valueOf(context.getProperty(COMPRESSION_CODEC).toString());
final boolean inferCompressionCodec = compressionType == CompressionType.AUTOMATIC;
// Check if compression codec is defined (inferred or otherwise) if(inferCompressionCodec) {
if (codec != null) { codec = compressionCodecFactory.getCodec(path);
stream = codec.createInputStream(stream); } else if (compressionType != CompressionType.NONE) {
outputFilename = StringUtils.removeEnd(originalFilename, codec.getDefaultExtension()); codec = getCompressionCodec(context, getConfiguration());
} else { }
outputFilename = originalFilename;
FlowFile flowFile = finalFlowFile;
try {
final String outputFilename;
final String originalFilename = path.getName();
stream = hdfs.open(path, 16384);
// Check if compression codec is defined (inferred or otherwise)
if (codec != null) {
stream = codec.createInputStream(stream);
outputFilename = StringUtils.removeEnd(originalFilename, codec.getDefaultExtension());
} else {
outputFilename = originalFilename;
}
flowFile = session.importFrom(stream, finalFlowFile);
flowFile = session.putAttribute(flowFile, CoreAttributes.FILENAME.key(), outputFilename);
stopWatch.stop();
getLogger().info("Successfully received content from {} for {} in {}", new Object[] {uri, flowFile, stopWatch.getDuration()});
session.getProvenanceReporter().fetch(flowFile, uri.toString(), stopWatch.getDuration(TimeUnit.MILLISECONDS));
session.transfer(flowFile, REL_SUCCESS);
} catch (final FileNotFoundException | AccessControlException e) {
getLogger().error("Failed to retrieve content from {} for {} due to {}; routing to failure", new Object[] {uri, flowFile, e});
flowFile = session.putAttribute(flowFile, "hdfs.failure.reason", e.getMessage());
flowFile = session.penalize(flowFile);
session.transfer(flowFile, REL_FAILURE);
} catch (final IOException e) {
getLogger().error("Failed to retrieve content from {} for {} due to {}; routing to comms.failure", new Object[] {uri, flowFile, e});
flowFile = session.penalize(flowFile);
session.transfer(flowFile, REL_COMMS_FAILURE);
} finally {
IOUtils.closeQuietly(stream);
}
return null;
} }
});
flowFile = session.importFrom(stream, flowFile);
flowFile = session.putAttribute(flowFile, CoreAttributes.FILENAME.key(), outputFilename);
stopWatch.stop();
getLogger().info("Successfully received content from {} for {} in {}", new Object[] {uri, flowFile, stopWatch.getDuration()});
session.getProvenanceReporter().fetch(flowFile, uri.toString(), stopWatch.getDuration(TimeUnit.MILLISECONDS));
session.transfer(flowFile, REL_SUCCESS);
} catch (final FileNotFoundException | AccessControlException e) {
getLogger().error("Failed to retrieve content from {} for {} due to {}; routing to failure", new Object[] {uri, flowFile, e});
flowFile = session.putAttribute(flowFile, "hdfs.failure.reason", e.getMessage());
flowFile = session.penalize(flowFile);
session.transfer(flowFile, REL_FAILURE);
} catch (final IOException e) {
getLogger().error("Failed to retrieve content from {} for {} due to {}; routing to comms.failure", new Object[] {uri, flowFile, e});
flowFile = session.penalize(flowFile);
session.transfer(flowFile, REL_COMMS_FAILURE);
} finally {
IOUtils.closeQuietly(stream);
}
} }
} }

View File

@ -22,6 +22,7 @@ import org.apache.nifi.authorization.Authorizer;
import org.apache.nifi.bundle.Bundle; import org.apache.nifi.bundle.Bundle;
import org.apache.nifi.controller.repository.FlowFileEventRepository; import org.apache.nifi.controller.repository.FlowFileEventRepository;
import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.provenance.MockProvenanceRepository; import org.apache.nifi.provenance.MockProvenanceRepository;
import org.apache.nifi.util.CapturingLogger; import org.apache.nifi.util.CapturingLogger;
import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.NiFiProperties;
@ -151,7 +152,7 @@ public class MonitorMemoryTest {
final NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(null, addProps); final NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(null, addProps);
// build the system bundle // build the system bundle
final Bundle bundle = ExtensionManager.createSystemBundle(nifiProperties); final Bundle bundle = SystemBundle.create(nifiProperties);
ExtensionManager.discoverExtensions(bundle, Collections.emptySet()); ExtensionManager.discoverExtensions(bundle, Collections.emptySet());
return new Tuple<>(FlowController.createStandaloneInstance( return new Tuple<>(FlowController.createStandaloneInstance(