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

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

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/NIFI-250
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