You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2015/01/20 19:03:40 UTC

[1/5] incubator-nifi git commit: NIFI-268: Added controller service lifecycle support to mock framework

Repository: incubator-nifi
Updated Branches:
  refs/heads/annotations 850396cc9 -> 1a402d468


NIFI-268: Added controller service lifecycle support to mock framework


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/53328a4a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/53328a4a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/53328a4a

Branch: refs/heads/annotations
Commit: 53328a4a029058c07ff0b429fcac3e29af7f8267
Parents: 850396c
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Jan 20 12:59:13 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Jan 20 12:59:13 2015 -0500

----------------------------------------------------------------------
 .../util/ControllerServiceConfiguration.java    |   2 +-
 .../nifi/util/MockControllerServiceLookup.java  |  10 ++
 .../apache/nifi/util/MockProcessContext.java    |   1 +
 .../nifi/util/StandardProcessorTestRunner.java  | 168 ++++++++++++++++++-
 .../java/org/apache/nifi/util/TestRunner.java   | 155 +++++++++++++++++
 5 files changed, 332 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/53328a4a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/ControllerServiceConfiguration.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/ControllerServiceConfiguration.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/ControllerServiceConfiguration.java
index 28e679b..bd623ca 100644
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/ControllerServiceConfiguration.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/ControllerServiceConfiguration.java
@@ -27,7 +27,7 @@ import org.apache.nifi.controller.ControllerService;
 public class ControllerServiceConfiguration {
 
     private final ControllerService service;
-    private final AtomicBoolean enabled = new AtomicBoolean(true);
+    private final AtomicBoolean enabled = new AtomicBoolean(false);
     private String annotationData;
     private Map<PropertyDescriptor, String> properties = new HashMap<>();
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/53328a4a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java
index ed0d650..f5a716c 100644
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java
@@ -38,6 +38,16 @@ public abstract class MockControllerServiceLookup implements ControllerServiceLo
         return addControllerService(service, service.getIdentifier());
     }
 
+    public void removeControllerService(ControllerService service) {
+        final ControllerService canonical = getControllerService(service.getIdentifier());
+        if ( canonical == null || canonical != service ) {
+            throw new IllegalArgumentException("Controller Service " + service + " is not known");
+        }
+        
+        controllerServiceMap.remove(service.getIdentifier());
+    }
+
+    
     protected ControllerServiceConfiguration getConfiguration(final String identifier) {
         return controllerServiceMap.get(identifier);
     }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/53328a4a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
index 15591d7..f83301c 100644
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
@@ -152,6 +152,7 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
         config.setProperties(properties);
         config.setAnnotationData(annotationData);
     }
+    
 
     @Override
     public int getMaxConcurrentTasks() {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/53328a4a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
index d0ecd52..63d1c9f 100644
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
@@ -27,6 +27,7 @@ import java.lang.reflect.Method;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -43,15 +44,19 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.nifi.annotation.behavior.TriggerSerially;
 import org.apache.nifi.annotation.lifecycle.OnAdded;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.annotation.lifecycle.OnRemoved;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnShutdown;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ControllerService;
-import org.apache.nifi.controller.annotation.OnConfigured;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processor.ProcessSessionFactory;
@@ -504,6 +509,12 @@ public class StandardProcessorTestRunner implements TestRunner {
 
     @Override
     public void addControllerService(final String identifier, final ControllerService service, final Map<String, String> properties) throws InitializationException {
+//        for ( final Method method : service.getClass().getMethods() ) {
+//            if ( method.isAnnotationPresent(org.apache.nifi.controller.annotation.OnConfigured.class) ) {
+//                Assert.fail("Controller Service " + service + " is using deprecated Annotation " + org.apache.nifi.controller.annotation.OnConfigured.class + " for method " + method);
+//            }
+//        }
+        
         final MockControllerServiceInitializationContext initContext = new MockControllerServiceInitializationContext(requireNonNull(service), requireNonNull(identifier));
         service.initialize(initContext);
 
@@ -512,9 +523,8 @@ public class StandardProcessorTestRunner implements TestRunner {
             resolvedProps.put(service.getPropertyDescriptor(entry.getKey()), entry.getValue());
         }
 
-        final MockConfigurationContext configurationContext = new MockConfigurationContext(resolvedProps, context);
         try {
-            ReflectionUtils.invokeMethodsWithAnnotation(OnConfigured.class, service, configurationContext);
+            ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, service);
         } catch (final InvocationTargetException | IllegalAccessException | IllegalArgumentException e) {
             throw new InitializationException(e);
         }
@@ -522,6 +532,158 @@ public class StandardProcessorTestRunner implements TestRunner {
         context.addControllerService(identifier, service, resolvedProps, null);
     }
 
+    
+    @Override
+    public void assertNotValid(final ControllerService service) {
+        final ValidationContext validationContext = new MockValidationContext(context).getControllerServiceValidationContext(service);
+        final Collection<ValidationResult> results = context.getControllerService(service.getIdentifier()).validate(validationContext);
+        
+        for ( final ValidationResult result : results ) {
+            if ( !result.isValid() ) {
+                return;
+            }
+        }
+        
+        Assert.fail("Expected Controller Service " + service + " to be invalid but it is valid");
+    }
+    
+    @Override
+    public void assertValid(final ControllerService service) {
+        final ValidationContext validationContext = new MockValidationContext(context).getControllerServiceValidationContext(service);
+        final Collection<ValidationResult> results = context.getControllerService(service.getIdentifier()).validate(validationContext);
+        
+        for ( final ValidationResult result : results ) {
+            if ( !result.isValid() ) {
+                Assert.fail("Expected Controller Service to be valid but it is invalid due to: " + result.toString());
+            }
+        }
+    }
+    
+    
+    @Override
+    public void disableControllerService(final ControllerService service) {
+        final ControllerServiceConfiguration configuration = context.getConfiguration(service.getIdentifier());
+        if ( configuration == null ) {
+            throw new IllegalArgumentException("Controller Service " + service + " is not known");
+        }
+        
+        if ( !configuration.isEnabled() ) {
+            throw new IllegalStateException("Controller service " + service + " cannot be disabled because it is not enabled");
+        }
+        
+        try {
+            ReflectionUtils.invokeMethodsWithAnnotation(OnDisabled.class, service);
+        } catch (final Exception e) {
+            e.printStackTrace();
+            Assert.fail("Failed to disable Controller Service " + service + " due to " + e);
+        }
+        
+        configuration.setEnabled(false);
+    }
+    
+    @Override
+    public void enableControllerService(final ControllerService service) {
+        final ControllerServiceConfiguration configuration = context.getConfiguration(service.getIdentifier());
+        if ( configuration == null ) {
+            throw new IllegalArgumentException("Controller Service " + service + " is not known");
+        }
+        
+        if ( configuration.isEnabled() ) {
+            throw new IllegalStateException("Cannot enable Controller Service " + service + " because it is not disabled");
+        }
+        
+        try {
+            final ConfigurationContext configContext = new MockConfigurationContext(configuration.getProperties(), context);
+            ReflectionUtils.invokeMethodsWithAnnotation(OnEnabled.class, service, configContext);
+        } catch (final InvocationTargetException ite) {
+            ite.getCause().printStackTrace();
+            Assert.fail("Failed to enable Controller Service " + service + " due to " + ite.getCause());
+        } catch (final Exception e) {
+            e.printStackTrace();
+            Assert.fail("Failed to enable Controller Service " + service + " due to " + e);
+        }
+
+        configuration.setEnabled(true);        
+    }
+    
+    @Override
+    public boolean isControllerServiceEnabled(final ControllerService service) {
+        final ControllerServiceConfiguration configuration = context.getConfiguration(service.getIdentifier());
+        if ( configuration == null ) {
+            throw new IllegalArgumentException("Controller Service " + service + " is not known");
+        }
+
+        return configuration.isEnabled();
+    }
+    
+    @Override
+    public void removeControllerService(final ControllerService service) {
+        disableControllerService(service);
+        
+        try {
+            ReflectionUtils.invokeMethodsWithAnnotation(OnRemoved.class, service);
+        } catch (final Exception e) {
+            e.printStackTrace();
+            Assert.fail("Failed to remove Controller Service " + service + " due to " + e);
+        }
+        
+        context.removeControllerService(service);
+    }
+    
+    @Override
+    public void setAnnotationData(final ControllerService service, final String annotationData) {
+        final ControllerServiceConfiguration configuration = getConfigToUpdate(service);
+        configuration.setAnnotationData(annotationData);
+    }
+    
+    private ControllerServiceConfiguration getConfigToUpdate(final ControllerService service) {
+        final ControllerServiceConfiguration configuration = context.getConfiguration(service.getIdentifier());
+        if ( configuration == null ) {
+            throw new IllegalArgumentException("Controller Service " + service + " is not known");
+        }
+        
+        if ( configuration.isEnabled() ) {
+            throw new IllegalStateException("Controller service " + service + " cannot be modified because it is not disabled");
+        }
+        
+        return configuration;
+    }
+    
+    @Override
+    public ValidationResult setProperty(final ControllerService service, final PropertyDescriptor property, final AllowableValue value) {
+        return setProperty(service, property, value.getValue());
+    }
+    
+    @Override
+    public ValidationResult setProperty(final ControllerService service, final PropertyDescriptor property, final String value) {
+        final ControllerServiceConfiguration configuration = getConfigToUpdate(service);
+        final Map<PropertyDescriptor, String> curProps = configuration.getProperties();
+        final Map<PropertyDescriptor, String> updatedProps = new HashMap<>(curProps);
+        
+        final ValidationContext validationContext = new MockValidationContext(context).getControllerServiceValidationContext(service);
+        final ValidationResult validationResult = property.validate(value, validationContext);
+        
+        updatedProps.put(property, value);
+        configuration.setProperties(updatedProps);
+        
+        return validationResult;
+    }
+    
+    @Override
+    public ValidationResult setProperty(final ControllerService service, final String propertyName, final String value) {
+        final PropertyDescriptor descriptor = service.getPropertyDescriptor(propertyName);
+        if ( descriptor == null ) {
+            return new ValidationResult.Builder()
+                .input(propertyName)
+                .explanation(propertyName + " is not a known Property for Controller Service " + service)
+                .subject("Invalid property")
+                .valid(false)
+                .build();
+        }
+        return setProperty(service, descriptor, value);
+    }
+    
+    
     @Override
     public ControllerService getControllerService(final String identifier) {
         return context.getControllerService(identifier);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/53328a4a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java
index a539cab..177c889 100644
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java
@@ -464,6 +464,9 @@ public interface TestRunner {
      * {@link nifi.controller.ControllerServiceInitializationContext ControllerServiceInitializationContext}
      * and initialize the ControllerService with no specified properties.
      *
+     * This will call any method on the given Controller Service that is annotated with
+     * the {@link org.apache.nifi.annotation.lifecycle.OnAdded @OnAdded} annotation.
+     *
      * @param identifier
      * @param service
      * @throws InitializationException
@@ -478,6 +481,9 @@ public interface TestRunner {
      * {@link nifi.controller.ControllerServiceInitializationContext ControllerServiceInitializationContext}
      * and initialize the ControllerService with the given properties.
      *
+     * This will call any method on the given Controller Service that is annotated with
+     * the {@link org.apache.nifi.annotation.lifecycle.OnAdded @OnAdded} annotation.
+     * 
      * @param identifier
      * @param service
      * @param properties
@@ -486,6 +492,132 @@ public interface TestRunner {
     void addControllerService(String identifier, ControllerService service, Map<String, String> properties) throws InitializationException;
 
     /**
+     * <p>
+     * Marks the Controller Service as enabled so that it can be used by other components.
+     * </p>
+     * 
+     * <p>
+     * This method will result in calling any method in the Controller Service that is
+     * annotated with the {@link org.apache.nifi.annotation.lifecycle.OnEnabled @OnEnabled} annotation.
+     * </p>
+     * 
+     * @param service
+     */
+    void enableControllerService(ControllerService service);
+    
+    /**
+     * <p>
+     * Marks the Controller Service as disabled so that it cannot be used by other components.
+     * </p>
+     * 
+     * <p>
+     * This method will result in calling any method in the Controller Service that is
+     * annotated with the {@link org.apache.nifi.annotation.lifecycle.OnDisabled @OnDisabled} annotation.
+     * </p>
+     * 
+     * @param service
+     */
+    void disableControllerService(ControllerService service);
+    
+    /**
+     * <p>
+     * Returns {@code true} if the given Controller Service is enabled, {@code false} if it is disabled.
+     * </p>
+     * 
+     * @param service
+     * @return
+     * 
+     * @throws IllegalArgumentException if the given ControllerService is not known by this TestRunner
+     * (i.e., it has not been added via the {@link #addControllerService(String, ControllerService)} or
+     * {@link #addControllerService(String, ControllerService, Map)} method or if the Controller Service
+     * has been removed via the {@link #removeControllerService(ControllerService)} method.
+     */
+    boolean isControllerServiceEnabled(ControllerService service);
+    
+    /**
+     * <p>
+     * Removes the Controller Service from the TestRunner. This will call any method on the ControllerService
+     * that is annotated with the {@link org.apache.nifi.annotation.lifecycle.OnRemoved @OnRemoved} annotation.
+     * </p>
+     * 
+     * @param service
+     * 
+     * @throws IllegalStateException if the ControllerService is not disabled
+     * @throws IllegalArgumentException if the given ControllerService is not known by this TestRunner
+     *      (i.e., it has not been added via the {@link #addControllerService(String, ControllerService)} or
+     *      {@link #addControllerService(String, ControllerService, Map)} method or if the Controller Service
+     *      has been removed via the {@link #removeControllerService(ControllerService)} method.
+     * 
+     */
+    void removeControllerService(ControllerService service);
+    
+    /**
+     * Sets the given property on the given ControllerService
+     * 
+     * @param service
+     * @param property
+     * @param value
+     * 
+     * @throws IllegalStateException if the ControllerService is not disabled
+     * @throws IllegalArgumentException if the given ControllerService is not known by this TestRunner
+     *      (i.e., it has not been added via the {@link #addControllerService(String, ControllerService)} or
+     *      {@link #addControllerService(String, ControllerService, Map)} method or if the Controller Service
+     *      has been removed via the {@link #removeControllerService(ControllerService)} method.
+     * 
+     */
+    ValidationResult setProperty(ControllerService service, PropertyDescriptor property, String value);
+    
+    
+    /**
+     * Sets the given property on the given ControllerService
+     * 
+     * @param service
+     * @param property
+     * @param value
+     * 
+     * @throws IllegalStateException if the ControllerService is not disabled
+     * @throws IllegalArgumentException if the given ControllerService is not known by this TestRunner
+     *      (i.e., it has not been added via the {@link #addControllerService(String, ControllerService)} or
+     *      {@link #addControllerService(String, ControllerService, Map)} method or if the Controller Service
+     *      has been removed via the {@link #removeControllerService(ControllerService)} method.
+     * 
+     */
+    ValidationResult setProperty(ControllerService service, PropertyDescriptor property, AllowableValue value);
+    
+    
+    /**
+     * Sets the property with the given name on the given ControllerService
+     * 
+     * @param service
+     * @param property
+     * @param value
+     * 
+     * @throws IllegalStateException if the ControllerService is not disabled
+     * @throws IllegalArgumentException if the given ControllerService is not known by this TestRunner
+     *      (i.e., it has not been added via the {@link #addControllerService(String, ControllerService)} or
+     *      {@link #addControllerService(String, ControllerService, Map)} method or if the Controller Service
+     *      has been removed via the {@link #removeControllerService(ControllerService)} method.
+     * 
+     */
+    ValidationResult setProperty(ControllerService service, String propertyName, String value);
+    
+    /**
+     * Sets the annontation data of the given service to the provided annotation data.
+     *
+     * @param service
+     * @param annotationData
+     * 
+     * @throws IllegalStateException if the Controller Service is not disabled
+     *
+     * @throws IllegalArgumentException if the given ControllerService is not known by this TestRunner
+     *      (i.e., it has not been added via the {@link #addControllerService(String, ControllerService)} or
+     *      {@link #addControllerService(String, ControllerService, Map)} method or if the Controller Service
+     *      has been removed via the {@link #removeControllerService(ControllerService)} method.
+     */
+    void setAnnotationData(ControllerService service, String annotationData);
+
+    
+    /**
      * Returns the {@link ControllerService} that is registered with the given
      * identifier, or <code>null</code> if no Controller Service exists with the
      * given identifier
@@ -496,6 +628,29 @@ public interface TestRunner {
     ControllerService getControllerService(String identifier);
 
     /**
+     * Assert that the currently configured set of properties/annotation data
+     * are valid for the given Controller Service.
+     * 
+     * @throws IllegalArgumentException if the given ControllerService is not known by this TestRunner
+     *      (i.e., it has not been added via the {@link #addControllerService(String, ControllerService)} or
+     *      {@link #addControllerService(String, ControllerService, Map)} method or if the Controller Service
+     *      has been removed via the {@link #removeControllerService(ControllerService)} method.
+     */
+    void assertValid(ControllerService service);
+
+    /**
+     * Assert that the currently configured set of properties/annotation data
+     * are NOT valid for the given Controller Service.
+     * 
+     * @throws IllegalArgumentException if the given ControllerService is not known by this TestRunner
+     *      (i.e., it has not been added via the {@link #addControllerService(String, ControllerService)} or
+     *      {@link #addControllerService(String, ControllerService, Map)} method or if the Controller Service
+     *      has been removed via the {@link #removeControllerService(ControllerService)} method.
+     * 
+     */
+    void assertNotValid(ControllerService service);
+
+    /**
      * Returns the {@link ControllerService} that is registered with the given
      * identifier, cast as the provided service type, or <code>null</code> if no
      * Controller Service exists with the given identifier


[5/5] incubator-nifi git commit: NIFI-4: Use @OnEnabled instead of @OnConfigured in ssl context service

Posted by ma...@apache.org.
NIFI-4: Use @OnEnabled instead of @OnConfigured in ssl context service


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/1a402d46
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/1a402d46
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/1a402d46

Branch: refs/heads/annotations
Commit: 1a402d468a42286fef4276d61de999f0862dc6a3
Parents: 1b1f45f
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Jan 20 13:03:21 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Jan 20 13:03:21 2015 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/nifi/ssl/StandardSSLContextService.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1a402d46/nifi/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/main/java/org/apache/nifi/ssl/StandardSSLContextService.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/main/java/org/apache/nifi/ssl/StandardSSLContextService.java b/nifi/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/main/java/org/apache/nifi/ssl/StandardSSLContextService.java
index d7aae16..68f83d4 100644
--- a/nifi/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/main/java/org/apache/nifi/ssl/StandardSSLContextService.java
+++ b/nifi/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/main/java/org/apache/nifi/ssl/StandardSSLContextService.java
@@ -26,13 +26,13 @@ import java.util.Map;
 
 import javax.net.ssl.SSLContext;
 
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
-import org.apache.nifi.controller.annotation.OnConfigured;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.reporting.InitializationException;
@@ -104,7 +104,7 @@ public class StandardSSLContextService extends AbstractControllerService impleme
     }
     private ConfigurationContext configContext;
 
-    @OnConfigured
+    @OnEnabled
     public void onConfigured(final ConfigurationContext context) throws InitializationException {
         configContext = context;
 


[3/5] incubator-nifi git commit: NIFI-4: Updated docs for OnEnabled

Posted by ma...@apache.org.
NIFI-4: Updated docs for OnEnabled


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/bd290018
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/bd290018
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/bd290018

Branch: refs/heads/annotations
Commit: bd290018adf08bf027d19f9891e257517e74a7a4
Parents: b595670
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Jan 20 13:00:44 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Jan 20 13:00:44 2015 -0500

----------------------------------------------------------------------
 .../nifi/annotation/lifecycle/OnEnabled.java     | 19 ++++++++++++++++---
 1 file changed, 16 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/bd290018/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnEnabled.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnEnabled.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnEnabled.java
index a0d7a14..8d7d6b3 100644
--- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnEnabled.java
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnEnabled.java
@@ -30,9 +30,22 @@ import java.lang.annotation.Target;
  * can use to indicate a method should be called whenever the component is enabled. 
  *
  * <p>
- * Methods using this annotation must take no arguments. If a method with this annotation
- * throws a Throwable, a log message and bulletin will be issued for the component, but
- * the component will still be enabled.
+ * Methods using this annotation must take either 0 arguments or a single argument.
+ * </p>
+ * 
+ * <p>
+ * If using 1 argument and the component using the annotation is a Processor, that argument must
+ * be of type {@link org.apache.nifi.processor.ProcessContext ProcessContext}.
+ * </p>
+ * 
+ * <p>
+ * If using 1 argument and the component using the annotation is a Reporting Task or Controller Service, 
+ * that argument must be of type {@link org.apache.nifi.controller.ConfigurationContext ConfigurationContext}.
+ * </p>
+ * 
+ * <p>
+ * If a method with this annotation throws a Throwable, a log message and bulletin will be issued 
+ * for the component, but the component will still be enabled.
  * </p>
  *
  * @author none


[2/5] incubator-nifi git commit: NIFI-4: Added better support for reporting task and controller service lifecycle via annotations

Posted by ma...@apache.org.
NIFI-4: Added better support for reporting task and controller service lifecycle via annotations


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/b5956709
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/b5956709
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/b5956709

Branch: refs/heads/annotations
Commit: b5956709b724a2a3520c6b6f5c877653b599ac5e
Parents: 53328a4
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Jan 20 12:59:59 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Jan 20 12:59:59 2015 -0500

----------------------------------------------------------------------
 .../cluster/manager/impl/WebClusterManager.java | 23 ++++-----
 .../nifi/controller/ReportingTaskNode.java      |  1 +
 .../service/ControllerServiceNode.java          |  1 +
 .../reporting/AbstractReportingTaskNode.java    | 53 ++++++++++++++++++--
 .../service/ControllerServiceLoader.java        | 10 ++--
 .../service/StandardControllerServiceNode.java  | 25 ++++++++-
 .../StandardControllerServiceProvider.java      |  1 +
 .../nifi/web/controller/ControllerFacade.java   |  8 ++-
 8 files changed, 97 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b5956709/nifi/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java b/nifi/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
index 511bb7d..cec9b74 100644
--- a/nifi/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
+++ b/nifi/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
@@ -191,7 +191,6 @@ import org.apache.nifi.web.api.entity.ProvenanceEventEntity;
 import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
 import org.apache.nifi.web.api.entity.RemoteProcessGroupsEntity;
 import org.apache.nifi.web.util.WebUtils;
-
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -1290,18 +1289,6 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
         }
     }
 
-    /**
-     * Adds an instance of a specified controller service.
-     *
-     * @param type
-     * @param id
-     * @param properties
-     * @return
-     */
-    @Override
-    public ControllerServiceNode createControllerService(String type, String id, Map<String, String> properties) {
-        return controllerServiceProvider.createControllerService(type, id, properties);
-    }
 
     @Override
     public ControllerService getControllerService(String serviceIdentifier) {
@@ -1323,6 +1310,16 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
         return controllerServiceProvider.isControllerServiceEnabled(serviceIdentifier);
     }
 
+    @Override
+    public ControllerServiceNode createControllerService(final String type, final String id, final boolean firstTimeAdded) {
+        return controllerServiceProvider.createControllerService(type, id, firstTimeAdded);
+    }
+    
+    @Override
+    public void removeControllerService(final ControllerServiceNode serviceNode) {
+        controllerServiceProvider.removeControllerService(serviceNode);
+    }
+    
     /**
      * Handle a bulletins message.
      *

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b5956709/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java
index 0db49bd..fa48cb3 100644
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java
@@ -73,4 +73,5 @@ public interface ReportingTaskNode extends ConfiguredComponent {
     void verifyCanDisable();
     void verifyCanEnable();
     void verifyCanDelete();
+    void verifyCanUpdate();
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b5956709/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java
index 357d4de..66bad39 100644
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java
@@ -43,4 +43,5 @@ public interface ControllerServiceNode extends ConfiguredComponent {
     void verifyCanEnable();
     void verifyCanDisable();
     void verifyCanDelete();
+    void verifyCanUpdate();
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b5956709/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
index f299781..7c3734a 100644
--- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
@@ -19,7 +19,6 @@ package org.apache.nifi.controller.reporting;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
-import org.apache.nifi.annotation.lifecycle.OnAdded;
 import org.apache.nifi.controller.AbstractConfiguredComponent;
 import org.apache.nifi.controller.Availability;
 import org.apache.nifi.controller.ConfigurationContext;
@@ -27,7 +26,6 @@ import org.apache.nifi.controller.ControllerServiceLookup;
 import org.apache.nifi.controller.ProcessScheduler;
 import org.apache.nifi.controller.ReportingTaskNode;
 import org.apache.nifi.controller.ScheduledState;
-import org.apache.nifi.controller.StandardProcessorNode;
 import org.apache.nifi.controller.ValidationContextFactory;
 import org.apache.nifi.controller.annotation.OnConfigured;
 import org.apache.nifi.controller.exception.ProcessorLifeCycleException;
@@ -145,6 +143,8 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon
     }
     
     private void onConfigured() {
+        // We need to invoke any method annotation with the OnConfigured annotation in order to
+        // maintain backward compatibility. This will be removed when we remove the old, deprecated annotations.
         try (final NarCloseable x = NarCloseable.withNarLoader()) {
             final ConfigurationContext configContext = new StandardConfigurationContext(this, serviceLookup);
             ReflectionUtils.invokeMethodsWithAnnotation(OnConfigured.class, reportingTask, configContext);
@@ -153,11 +153,58 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon
         }
     }
     
+    public boolean isDisabled() {
+        return scheduledState == ScheduledState.DISABLED;
+    }
+    
     @Override
     public void verifyCanDelete() {
         if (isRunning()) {
-            throw new IllegalStateException(this + " is running");
+            throw new IllegalStateException("Cannot delete " + reportingTask + " because it is currently running");
+        }
+    }
+    
+    @Override
+    public void verifyCanDisable() {
+        if ( isRunning() ) {
+            throw new IllegalStateException("Cannot disable " + reportingTask + " because it is currently running");
+        }
+        
+        if ( isDisabled() ) {
+            throw new IllegalStateException("Cannot disable " + reportingTask + " because it is already disabled");
+        }
+    }
+    
+    
+    @Override
+    public void verifyCanEnable() {
+        if ( !isDisabled() ) {
+            throw new IllegalStateException("Cannot enable " + reportingTask + " because it is not disabled");
         }
     }
     
+    @Override
+    public void verifyCanStart() {
+        if ( isDisabled() ) {
+            throw new IllegalStateException("Cannot start " + reportingTask + " because it is currently disabled");
+        }
+        
+        if ( isRunning() ) {
+            throw new IllegalStateException("Cannot start " + reportingTask + " because it is already running");
+        }
+    }
+    
+    @Override
+    public void verifyCanStop() {
+        if ( !isRunning() ) {
+            throw new IllegalStateException("Cannot stop " + reportingTask + " because it is not running");
+        }
+    }
+    
+    @Override
+    public void verifyCanUpdate() {
+        if ( isRunning() ) {
+            throw new IllegalStateException("Cannot update " + reportingTask + " because it is currently running");
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b5956709/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java
index 42bd55f..9fec307 100644
--- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java
@@ -123,23 +123,21 @@ public class ControllerServiceLoader {
 
                 final List<Element> serviceNodes = DomUtils.getChildElementsByTagName(servicesElement, "service");
                 for (final Element serviceElement : serviceNodes) {
-                    //add global properties common to all tasks
-                    Map<String, String> properties = new HashMap<>();
-
                     //get properties for the specific controller task - id, name, class,
                     //and schedulingPeriod must be set
                     final String serviceId = DomUtils.getChild(serviceElement, "identifier").getTextContent().trim();
                     final String serviceClass = DomUtils.getChild(serviceElement, "class").getTextContent().trim();
 
+                    //set the class to be used for the configured controller task
+                    final ControllerServiceNode serviceNode = provider.createControllerService(serviceClass, serviceId, false);
+
                     //optional task-specific properties
                     for (final Element optionalProperty : DomUtils.getChildElementsByTagName(serviceElement, "property")) {
                         final String name = optionalProperty.getAttribute("name").trim();
                         final String value = optionalProperty.getTextContent().trim();
-                        properties.put(name, value);
+                        serviceNode.setProperty(name, value);
                     }
 
-                    //set the class to be used for the configured controller task
-                    final ControllerServiceNode serviceNode = provider.createControllerService(serviceClass, serviceId, properties);
                     services.add(serviceNode);
                     serviceNode.setDisabled(false);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b5956709/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
index 61a3aa8..4681293 100644
--- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
@@ -166,7 +166,30 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
     @Override
     public void verifyCanDelete() {
         if ( !isDisabled() ) {
-            throw new IllegalStateException(this + " cannot be deleted because it has not been disabled");
+            throw new IllegalStateException(implementation + " cannot be deleted because it is not disabled");
+        }
+    }
+    
+    @Override
+    public void verifyCanDisable() {
+        final ControllerServiceReference references = getReferences();
+        final int numRunning = references.getRunningReferences().size();
+        if ( numRunning > 0 ) {
+            throw new IllegalStateException(implementation + " cannot be disabled because it is referenced by " + numRunning + " components that are currently running");
+        }
+    }
+    
+    @Override
+    public void verifyCanEnable() {
+        if ( !isDisabled() ) {
+            throw new IllegalStateException(implementation + " cannot be enabled because it is not disabled");
+        }
+    }
+    
+    @Override
+    public void verifyCanUpdate() {
+        if ( !isDisabled() ) {
+            throw new IllegalStateException(implementation + " cannot be updated because it is not disabled");
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b5956709/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
index 1deed59..cc7a18a 100644
--- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
@@ -117,6 +117,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
                 public Object invoke(final Object proxy, final Method method, final Object[] args) throws Throwable {
                     final ControllerServiceNode node = serviceNodeHolder.get();
                     if (node.isDisabled() && !validDisabledMethods.contains(method)) {
+                        // Use nar class loader here because we are implicitly calling toString() on the original implementation.
                         try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
                             throw new IllegalStateException("Cannot invoke method " + method + " on Controller Service " + originalService + " because the Controller Service is disabled");
                         } catch (final Throwable e) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b5956709/nifi/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
index dbc4b3c..c97d38c 100644
--- a/nifi/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
+++ b/nifi/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
@@ -384,8 +384,12 @@ public class ControllerFacade implements ControllerServiceProvider {
     }
 
     @Override
-    public ControllerServiceNode createControllerService(String type, String id, Map<String, String> properties) {
-        return flowController.createControllerService(type, id, properties);
+    public ControllerServiceNode createControllerService(String type, String id, boolean firstTimeAdded) {
+        return flowController.createControllerService(type, id, firstTimeAdded);
+    }
+    
+    public void removeControllerService(ControllerServiceNode serviceNode) {
+        flowController.removeControllerService(serviceNode);
     }
 
     @Override


[4/5] incubator-nifi git commit: NIFI-268: Updated unit tests

Posted by ma...@apache.org.
NIFI-268: Updated unit tests


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/1b1f45fa
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/1b1f45fa
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/1b1f45fa

Branch: refs/heads/annotations
Commit: 1b1f45faac1dc611a6df38abd6cdfb51ba382023
Parents: bd29001
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Jan 20 13:02:59 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Jan 20 13:02:59 2015 -0500

----------------------------------------------------------------------
 .../standard/TestDetectDuplicate.java           |   6 +-
 .../nifi/processors/standard/TestGetHTTP.java   |  15 +-
 .../processors/standard/TestInvokeHTTP.java     |   4 +-
 .../apache/nifi/ssl/SSLContextServiceTest.java  | 182 ++++++++-----------
 4 files changed, 83 insertions(+), 124 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1b1f45fa/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java
index 6296f7d..4a74416 100644
--- a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java
+++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDetectDuplicate.java
@@ -55,7 +55,6 @@ public class TestDetectDuplicate {
 
     @Test
     public void testDuplicate() throws InitializationException {
-
         TestRunner runner = TestRunners.newTestRunner(DetectDuplicate.class);
         final DistributedMapCacheClientImpl client = createClient();
         final Map<String, String> clientProperties = new HashMap<>();
@@ -67,6 +66,8 @@ public class TestDetectDuplicate {
         Map<String, String> props = new HashMap<>();
         props.put("hash.value", "1000");
         runner.enqueue(new byte[]{}, props);
+        runner.enableControllerService(client);
+        
         runner.run();
         runner.assertAllFlowFilesTransferred(DetectDuplicate.REL_NON_DUPLICATE, 1);
         runner.clearTransferState();
@@ -89,9 +90,12 @@ public class TestDetectDuplicate {
         runner.setProperty(DetectDuplicate.DISTRIBUTED_CACHE_SERVICE, "client");
         runner.setProperty(DetectDuplicate.FLOWFILE_DESCRIPTION, "The original flow file");
         runner.setProperty(DetectDuplicate.AGE_OFF_DURATION, "2 secs");
+        runner.enableControllerService(client);
+        
         Map<String, String> props = new HashMap<>();
         props.put("hash.value", "1000");
         runner.enqueue(new byte[]{}, props);
+        
         runner.run();
         runner.assertAllFlowFilesTransferred(DetectDuplicate.REL_NON_DUPLICATE, 1);
         runner.clearTransferState();

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1b1f45fa/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java
index cf6530a..f52b212 100644
--- a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java
+++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java
@@ -40,8 +40,6 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * @author unattributed
@@ -49,7 +47,6 @@ import org.slf4j.LoggerFactory;
  */
 public class TestGetHTTP {
 
-    private static Logger LOGGER;
     private TestRunner controller;
     
     @BeforeClass
@@ -58,7 +55,6 @@ public class TestGetHTTP {
         System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
         System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.GetHTTP", "debug");
         System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.TestGetHTTP", "debug");
-        LOGGER = LoggerFactory.getLogger(TestGetHTTP.class);
         File confDir = new File("conf");
         if (!confDir.exists()) {
             confDir.mkdir();
@@ -77,16 +73,6 @@ public class TestGetHTTP {
         assertTrue(confDir.delete());
     }
 
-    private static Map<String, String> createSslProperties() {
-        Map<String, String> map = new HashMap<String, String>();
-        map.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
-        map.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
-        map.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "JKS");
-        map.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
-        map.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "localtest");
-        map.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
-        return map;
-    }
 
     @Test
     public final void testContentModified() throws Exception {
@@ -308,6 +294,7 @@ public class TestGetHTTP {
         final SSLContextService service = new StandardSSLContextService();
         try {
             controller.addControllerService("ssl-service", service, getSslProperties());
+            controller.enableControllerService(service);
         } catch (InitializationException ex) {
             ex.printStackTrace();
             Assert.fail("Could not create SSL Context Service");

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1b1f45fa/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
index b98ba13..e5950cd 100644
--- a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
+++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
@@ -83,7 +83,9 @@ public class TestInvokeHTTP {
     @Before
     public void before() throws Exception {
         runner = TestRunners.newTestRunner(InvokeHTTP.class);
-        runner.addControllerService("ssl-context", new StandardSSLContextService(), sslProperties);
+        final StandardSSLContextService sslService = new StandardSSLContextService();
+        runner.addControllerService("ssl-context", sslService, sslProperties);
+        runner.enableControllerService(sslService);
         runner.setProperty(Config.PROP_SSL_CONTEXT_SERVICE, "ssl-context");
 
         server.clearHandlers();

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1b1f45fa/nifi/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/java/org/apache/nifi/ssl/SSLContextServiceTest.java
----------------------------------------------------------------------
diff --git a/nifi/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/java/org/apache/nifi/ssl/SSLContextServiceTest.java b/nifi/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/java/org/apache/nifi/ssl/SSLContextServiceTest.java
index 5d583ca..7d191fb 100644
--- a/nifi/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/java/org/apache/nifi/ssl/SSLContextServiceTest.java
+++ b/nifi/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/java/org/apache/nifi/ssl/SSLContextServiceTest.java
@@ -16,141 +16,103 @@
  */
 package org.apache.nifi.ssl;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 import java.util.HashMap;
+import java.util.Map;
 
-import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.ssl.SSLContextService.ClientAuth;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
-
 import org.junit.Assert;
 import org.junit.Test;
 
 public class SSLContextServiceTest {
 
     @Test
-    public void testBad1() {
-        try {
-            TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
-            SSLContextService service = new StandardSSLContextService();
-            HashMap<String, String> properties = new HashMap<String, String>();
-            runner.addControllerService("test-bad1", service, properties);
-            Assert.fail("Should have thrown an Exception");
-        } catch (InitializationException e) {
-            assertEquals(
-                    "org.apache.nifi.reporting.InitializationException: SSLContextService[id=test-bad1] does not have the KeyStore or the TrustStore populated",
-                    e.getCause().getCause().toString());
-        }
+    public void testBad1() throws InitializationException {
+        final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+        final SSLContextService service = new StandardSSLContextService();
+        final Map<String, String> properties = new HashMap<String, String>();
+        runner.addControllerService("test-bad1", service, properties);
+        runner.assertNotValid(service);
     }
 
     @Test
-    public void testBad2() {
-        try {
-            TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
-            SSLContextService service = new StandardSSLContextService();
-            HashMap<String, String> properties = new HashMap<String, String>();
-            properties.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
-            properties.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
-            runner.addControllerService("test-bad2", service, properties);
-            Assert.fail("Should have thrown an Exception");
-        } catch (InitializationException e) {
-            assertEquals(
-                    "org.apache.nifi.reporting.InitializationException: SSLContextService[id=test-bad2] is not valid due to:\n'Keystore Properties' is invalid because Must set either 0 or 3 properties for Keystore",
-                    e.getCause().getCause().toString());
-        }
+    public void testBad2() throws InitializationException {
+        final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+        final SSLContextService service = new StandardSSLContextService();
+        final Map<String, String> properties = new HashMap<String, String>();
+        properties.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
+        properties.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
+        runner.addControllerService("test-bad2", service, properties);
+        runner.assertNotValid(service);
     }
 
     @Test
-    public void testBad3() {
-        try {
-            TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
-            SSLContextService service = new StandardSSLContextService();
-            HashMap<String, String> properties = new HashMap<String, String>();
-            properties.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
-            properties.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
-            properties.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "JKS");
-            properties.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
-            runner.addControllerService("test-bad3", service, properties);
-            Assert.fail("Should have thrown an Exception");
-        } catch (InitializationException e) {
-            assertEquals(
-                    "org.apache.nifi.reporting.InitializationException: SSLContextService[id=test-bad3] is not valid due to:\n'Truststore Properties' is invalid because Must set either 0 or 3 properties for Truststore",
-                    e.getCause().getCause().toString());
-        }
+    public void testBad3() throws InitializationException {
+        final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+        final SSLContextService service = new StandardSSLContextService();
+        final Map<String, String> properties = new HashMap<String, String>();
+        properties.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
+        properties.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
+        properties.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "JKS");
+        properties.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
+        runner.addControllerService("test-bad3", service, properties);
+        runner.assertNotValid(service);
     }
 
     @Test
-    public void testBad4() {
-        try {
-            TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
-            SSLContextService service = new StandardSSLContextService();
-            HashMap<String, String> properties = new HashMap<String, String>();
-            properties.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
-            properties.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "wrongpassword");
-            properties.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "PKCS12");
-            properties.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
-            properties.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "wrongpassword");
-            properties.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
-            runner.addControllerService("test-bad4", service, properties);
-            Assert.fail("Should have thrown an Exception");
-        } catch (InitializationException e) {
-            assertEquals(
-                    "org.apache.nifi.reporting.InitializationException: SSLContextService[id=test-bad4] is not valid due to:\n"
-                    + "'Keystore Properties' is invalid because Invalid KeyStore Password or Type specified for file src/test/resources/localhost-ks.jks\n"
-                    + "'Truststore Properties' is invalid because Invalid KeyStore Password or Type specified for file src/test/resources/localhost-ts.jks",
-                    e.getCause().getCause().toString());
-        }
+    public void testBad4() throws InitializationException {
+        final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+        final SSLContextService service = new StandardSSLContextService();
+        final Map<String, String> properties = new HashMap<String, String>();
+        properties.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
+        properties.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "wrongpassword");
+        properties.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "PKCS12");
+        properties.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
+        properties.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "wrongpassword");
+        properties.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
+        runner.addControllerService("test-bad4", service, properties);
+        
+        runner.assertNotValid(service);
     }
 
     @Test
-    public void testBad5() {
-        try {
-            TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
-            SSLContextService service = new StandardSSLContextService();
-            HashMap<String, String> properties = new HashMap<String, String>();
-            properties.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/DOES-NOT-EXIST.jks");
-            properties.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
-            properties.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "PKCS12");
-            properties.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
-            properties.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "localtest");
-            properties.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
-            runner.addControllerService("test-bad5", service, properties);
-            Assert.fail("Should have thrown an Exception");
-        } catch (InitializationException e) {
-            assertTrue(e.getCause().getCause().toString().startsWith("org.apache.nifi.reporting.InitializationException: "
-                    + "SSLContextService[id=test-bad5] is not valid due to:\n'Keystore Properties' is invalid "
-                    + "because Cannot access file"));
-        }
+    public void testBad5() throws InitializationException {
+        final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+        final SSLContextService service = new StandardSSLContextService();
+        final Map<String, String> properties = new HashMap<String, String>();
+        properties.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/DOES-NOT-EXIST.jks");
+        properties.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
+        properties.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "PKCS12");
+        properties.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
+        properties.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "localtest");
+        properties.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
+        runner.addControllerService("test-bad5", service, properties);
+        runner.assertNotValid(service);
     }
 
     @Test
-    public void testGood() {
-        try {
-            TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
-            ControllerService service = new StandardSSLContextService();
-            HashMap<String, String> properties = new HashMap<String, String>();
-            properties.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
-            properties.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
-            properties.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "PKCS12");
-            properties.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
-            properties.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "localtest");
-            properties.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
-            runner.addControllerService("test-good1", service, properties);
-            runner.setProperty("SSL Context Svc ID", "test-good1");
-            runner.assertValid();
-            service = runner.getProcessContext().getControllerServiceLookup().getControllerService("test-good1");
-            Assert.assertNotNull(service);
-            Assert.assertTrue(service instanceof StandardSSLContextService);
-            SSLContextService sslService = (SSLContextService) service;
-            sslService.createSSLContext(ClientAuth.REQUIRED);
-            sslService.createSSLContext(ClientAuth.WANT);
-            sslService.createSSLContext(ClientAuth.NONE);
-        } catch (InitializationException e) {
-        }
+    public void testGood() throws InitializationException {
+        final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+        SSLContextService service = new StandardSSLContextService();
+        runner.addControllerService("test-good1", service);
+        runner.setProperty(service, StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
+        runner.setProperty(service, StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
+        runner.setProperty(service, StandardSSLContextService.KEYSTORE_TYPE.getName(), "JKS");
+        runner.setProperty(service, StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
+        runner.setProperty(service, StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "localtest");
+        runner.setProperty(service, StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
+        runner.enableControllerService(service);
+
+        runner.setProperty("SSL Context Svc ID", "test-good1");
+        runner.assertValid(service);
+        service = (SSLContextService) runner.getProcessContext().getControllerServiceLookup().getControllerService("test-good1");
+        Assert.assertNotNull(service);
+        SSLContextService sslService = (SSLContextService) service;
+        sslService.createSSLContext(ClientAuth.REQUIRED);
+        sslService.createSSLContext(ClientAuth.WANT);
+        sslService.createSSLContext(ClientAuth.NONE);
     }
 
     @Test
@@ -163,6 +125,8 @@ public class SSLContextServiceTest {
             properties.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "localtest");
             properties.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
             runner.addControllerService("test-good2", service, properties);
+            runner.enableControllerService(service);
+            
             runner.setProperty("SSL Context Svc ID", "test-good2");
             runner.assertValid();
             Assert.assertNotNull(service);
@@ -182,6 +146,8 @@ public class SSLContextServiceTest {
             properties.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
             properties.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "JKS");
             runner.addControllerService("test-good3", service, properties);
+            runner.enableControllerService(service);
+
             runner.setProperty("SSL Context Svc ID", "test-good3");
             runner.assertValid();
             Assert.assertNotNull(service);