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/07/01 17:43:54 UTC

[1/2] incubator-nifi git commit: NIFI-719: Expose scheduling period to the ConfigurationContext

Repository: incubator-nifi
Updated Branches:
  refs/heads/develop 50215707f -> a09180799


NIFI-719: Expose scheduling period to the ConfigurationContext


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

Branch: refs/heads/develop
Commit: ba3155980efd846b45bd47eefa14f503a13908f3
Parents: e767f5c
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Jun 23 20:47:35 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Jun 24 08:13:06 2015 -0400

----------------------------------------------------------------------
 .../nifi/controller/ConfigurationContext.java   | 23 +++++++++++++++--
 .../nifi/util/MockConfigurationContext.java     | 11 +++++++++
 .../apache/nifi/controller/FlowController.java  | 18 +++++++-------
 .../reporting/AbstractReportingTaskNode.java    |  4 +--
 .../scheduling/StandardProcessScheduler.java    | 24 +++++++++---------
 .../service/StandardConfigurationContext.java   | 26 +++++++++++++++++++-
 .../service/StandardControllerServiceNode.java  |  2 +-
 .../StandardControllerServiceProvider.java      | 10 ++++----
 .../apache/nifi/controller/MonitorMemory.java   | 14 +++--------
 9 files changed, 89 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ba315598/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ConfigurationContext.java b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ConfigurationContext.java
index 1fff6b9..03965d4 100644
--- a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ConfigurationContext.java
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ConfigurationContext.java
@@ -17,13 +17,14 @@
 package org.apache.nifi.controller;
 
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 
 /**
- * This context is passed to ControllerServices after the service has been
- * initialized.
+ * This context is passed to ControllerServices and Reporting Tasks in order
+ * to expose their configuration to them.
  */
 public interface ConfigurationContext {
 
@@ -39,4 +40,22 @@ public interface ConfigurationContext {
      */
     Map<PropertyDescriptor, String> getProperties();
 
+    /**
+     * @return a String representation of the scheduling period, or <code>null</code> if
+     *         the component does not have a scheduling period (e.g., for ControllerServices)
+     */
+    String getSchedulingPeriod();
+
+    /**
+     * Returns the amount of time, in the given {@link TimeUnit} that will
+     * elapsed between the return of one execution of the
+     * component's <code>onTrigger</code> method and
+     * the time at which the method is invoked again. This method will return
+     * null if the component does not have a scheduling period (e.g., for ControllerServices)
+     *
+     * @param timeUnit unit of time for scheduling
+     * @return period of time or <code>null</code> if component does not have a scheduling
+     *         period
+     */
+    Long getSchedulingPeriod(TimeUnit timeUnit);
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ba315598/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java
index 61af49d..e9afb32 100644
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java
@@ -18,6 +18,7 @@ package org.apache.nifi.util;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
@@ -47,4 +48,14 @@ public class MockConfigurationContext implements ConfigurationContext {
     public Map<PropertyDescriptor, String> getProperties() {
         return new HashMap<>(this.properties);
     }
+
+    @Override
+    public String getSchedulingPeriod() {
+        return "0 secs";
+    }
+
+    @Override
+    public Long getSchedulingPeriod(final TimeUnit timeUnit) {
+        return 0L;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ba315598/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 2ffdd4e..a56ae04 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -593,7 +593,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         writeLock.lock();
         try {
             if (startDelayedComponents) {
-                LOG.info("Starting {} processors/ports/funnels", (startConnectablesAfterInitialization.size() + startRemoteGroupPortsAfterInitialization.size()));
+                LOG.info("Starting {} processors/ports/funnels", startConnectablesAfterInitialization.size() + startRemoteGroupPortsAfterInitialization.size());
                 for (final Connectable connectable : startConnectablesAfterInitialization) {
                     if (connectable.getScheduledState() == ScheduledState.DISABLED) {
                         continue;
@@ -1012,7 +1012,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     public boolean isTerminated() {
         this.readLock.lock();
         try {
-            return (null == this.timerDrivenEngineRef.get() || this.timerDrivenEngineRef.get().isTerminated());
+            return null == this.timerDrivenEngineRef.get() || this.timerDrivenEngineRef.get().isTerminated();
         } finally {
             this.readLock.unlock();
         }
@@ -1054,7 +1054,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             // invoke any methods annotated with @OnShutdown on Controller Services
             for (final ControllerServiceNode serviceNode : getAllControllerServices()) {
                 try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    final ConfigurationContext configContext = new StandardConfigurationContext(serviceNode, controllerServiceProvider);
+                    final ConfigurationContext configContext = new StandardConfigurationContext(serviceNode, controllerServiceProvider, null);
                     ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, serviceNode.getControllerServiceImplementation(), configContext);
                 }
             }
@@ -1828,9 +1828,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         } else if (id1.equals(id2)) {
             return true;
         } else {
-            final String comparable1 = (id1.equals(ROOT_GROUP_ID_ALIAS) ? getRootGroupId() : id1);
-            final String comparable2 = (id2.equals(ROOT_GROUP_ID_ALIAS) ? getRootGroupId() : id2);
-            return (comparable1.equals(comparable2));
+            final String comparable1 = id1.equals(ROOT_GROUP_ID_ALIAS) ? getRootGroupId() : id1;
+            final String comparable2 = id2.equals(ROOT_GROUP_ID_ALIAS) ? getRootGroupId() : id2;
+            return comparable1.equals(comparable2);
         }
     }
 
@@ -1964,7 +1964,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         }
 
         final String searchId = id.equals(ROOT_GROUP_ID_ALIAS) ? getRootGroupId() : id;
-        return (root == null) ? null : root.findProcessGroup(searchId);
+        return root == null ? null : root.findProcessGroup(searchId);
     }
 
     @Override
@@ -2079,8 +2079,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
                 connStatus.setOutputBytes(connectionStatusReport.getContentSizeOut());
                 connStatus.setOutputCount(connectionStatusReport.getFlowFilesOut());
 
-                flowFilesTransferred += (connectionStatusReport.getFlowFilesIn() + connectionStatusReport.getFlowFilesOut());
-                bytesTransferred += (connectionStatusReport.getContentSizeIn() + connectionStatusReport.getContentSizeOut());
+                flowFilesTransferred += connectionStatusReport.getFlowFilesIn() + connectionStatusReport.getFlowFilesOut();
+                bytesTransferred += connectionStatusReport.getContentSizeIn() + connectionStatusReport.getContentSizeOut();
             }
 
             if (StringUtils.isNotBlank(conn.getName())) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ba315598/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
index 7adab68..b537c30 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
@@ -104,7 +104,7 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon
 
     @Override
     public ConfigurationContext getConfigurationContext() {
-        return new StandardConfigurationContext(this, serviceLookup);
+        return new StandardConfigurationContext(this, serviceLookup, getSchedulingPeriod());
     }
 
     @Override
@@ -146,7 +146,7 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon
         // 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);
+            final ConfigurationContext configContext = new StandardConfigurationContext(this, serviceLookup, getSchedulingPeriod());
             ReflectionUtils.invokeMethodsWithAnnotation(OnConfigured.class, reportingTask, configContext);
         } catch (final Exception e) {
             throw new ComponentLifeCycleException("Failed to invoke On-Configured Lifecycle methods of " + reportingTask, e);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ba315598/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
index 7bcecf3..cf644ed 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
@@ -185,7 +185,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
 
                         break;
                     } catch (final Exception e) {
-                        final Throwable cause = (e instanceof InvocationTargetException) ? e.getCause() : e;
+                        final Throwable cause = e instanceof InvocationTargetException ? e.getCause() : e;
                         final ComponentLog componentLog = new SimpleProcessLogger(reportingTask.getIdentifier(), reportingTask);
                         componentLog.error("Failed to invoke @OnEnabled method due to {}", cause);
 
@@ -230,7 +230,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
                         ReflectionUtils.invokeMethodsWithAnnotations(OnUnscheduled.class, org.apache.nifi.processor.annotation.OnUnscheduled.class, reportingTask, configurationContext);
                     }
                 } catch (final Exception e) {
-                    final Throwable cause = (e instanceof InvocationTargetException) ? e.getCause() : e;
+                    final Throwable cause = e instanceof InvocationTargetException ? e.getCause() : e;
                     final ComponentLog componentLog = new SimpleProcessLogger(reportingTask.getIdentifier(), reportingTask);
                     componentLog.error("Failed to invoke @OnUnscheduled method due to {}", cause);
 
@@ -284,7 +284,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
             @SuppressWarnings("deprecation")
             public void run() {
                 try (final NarCloseable x = NarCloseable.withNarLoader()) {
-                    long lastStopTime = scheduleState.getLastStopTime();
+                    final long lastStopTime = scheduleState.getLastStopTime();
                     final StandardProcessContext processContext = new StandardProcessContext(procNode, controllerServiceProvider, encryptor);
 
                     final Set<String> serviceIds = new HashSet<>();
@@ -330,7 +330,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
                                 return;
                             }
                         } catch (final Exception e) {
-                            final Throwable cause = (e instanceof InvocationTargetException) ? e.getCause() : e;
+                            final Throwable cause = e instanceof InvocationTargetException ? e.getCause() : e;
                             final ProcessorLog procLog = new SimpleProcessLogger(procNode.getIdentifier(), procNode.getProcessor());
 
                             procLog.error("{} failed to invoke @OnScheduled method due to {}; processor will not be scheduled to run for {}",
@@ -569,7 +569,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
     @Override
     public boolean isScheduled(final Object scheduled) {
         final ScheduleState scheduleState = scheduleStates.get(scheduled);
-        return (scheduleState == null) ? false : scheduleState.isScheduled();
+        return scheduleState == null ? false : scheduleState.isScheduled();
     }
 
     /**
@@ -582,7 +582,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
         ScheduleState scheduleState = scheduleStates.get(schedulable);
         if (scheduleState == null) {
             scheduleState = new ScheduleState();
-            ScheduleState previous = scheduleStates.putIfAbsent(schedulable, scheduleState);
+            final ScheduleState previous = scheduleStates.putIfAbsent(schedulable, scheduleState);
             if (previous != null) {
                 scheduleState = previous;
             }
@@ -599,8 +599,8 @@ public final class StandardProcessScheduler implements ProcessScheduler {
             @Override
             public void run() {
                 try (final NarCloseable x = NarCloseable.withNarLoader()) {
-                    long lastStopTime = scheduleState.getLastStopTime();
-                    final ConfigurationContext configContext = new StandardConfigurationContext(service, controllerServiceProvider);
+                    final long lastStopTime = scheduleState.getLastStopTime();
+                    final ConfigurationContext configContext = new StandardConfigurationContext(service, controllerServiceProvider, null);
 
                     while (true) {
                         try {
@@ -622,7 +622,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
                                 return;
                             }
                         } catch (final Exception e) {
-                            final Throwable cause = (e instanceof InvocationTargetException) ? e.getCause() : e;
+                            final Throwable cause = e instanceof InvocationTargetException ? e.getCause() : e;
 
                             final ComponentLog componentLog = new SimpleProcessLogger(service.getIdentifier(), service);
                             componentLog.error("Failed to invoke @OnEnabled method due to {}", cause);
@@ -637,7 +637,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
                         }
                     }
                 } catch (final Throwable t) {
-                    final Throwable cause = (t instanceof InvocationTargetException) ? t.getCause() : t;
+                    final Throwable cause = t instanceof InvocationTargetException ? t.getCause() : t;
                     final ComponentLog componentLog = new SimpleProcessLogger(service.getIdentifier(), service);
                     componentLog.error("Failed to invoke @OnEnabled method due to {}", cause);
 
@@ -666,7 +666,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
                 }
 
                 try (final NarCloseable x = NarCloseable.withNarLoader()) {
-                    final ConfigurationContext configContext = new StandardConfigurationContext(service, controllerServiceProvider);
+                    final ConfigurationContext configContext = new StandardConfigurationContext(service, controllerServiceProvider, null);
 
                     while (true) {
                         try {
@@ -675,7 +675,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
                             service.setState(ControllerServiceState.DISABLED);
                             return;
                         } catch (final Exception e) {
-                            final Throwable cause = (e instanceof InvocationTargetException) ? e.getCause() : e;
+                            final Throwable cause = e instanceof InvocationTargetException ? e.getCause() : e;
                             final ComponentLog componentLog = new SimpleProcessLogger(service.getIdentifier(), service);
                             componentLog.error("Failed to invoke @OnDisabled method due to {}", cause);
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ba315598/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardConfigurationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardConfigurationContext.java
index e0020c3..f57dc7c 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardConfigurationContext.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardConfigurationContext.java
@@ -18,6 +18,7 @@ package org.apache.nifi.controller.service;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.nifi.attribute.expression.language.PreparedQuery;
 import org.apache.nifi.attribute.expression.language.Query;
@@ -27,16 +28,29 @@ import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ConfiguredComponent;
 import org.apache.nifi.controller.ControllerServiceLookup;
 import org.apache.nifi.processor.StandardPropertyValue;
+import org.apache.nifi.util.FormatUtils;
 
 public class StandardConfigurationContext implements ConfigurationContext {
 
     private final ConfiguredComponent component;
     private final ControllerServiceLookup serviceLookup;
     private final Map<PropertyDescriptor, PreparedQuery> preparedQueries;
+    private final String schedulingPeriod;
+    private final Long schedulingNanos;
 
-    public StandardConfigurationContext(final ConfiguredComponent component, final ControllerServiceLookup serviceLookup) {
+    public StandardConfigurationContext(final ConfiguredComponent component, final ControllerServiceLookup serviceLookup, final String schedulingPeriod) {
         this.component = component;
         this.serviceLookup = serviceLookup;
+        this.schedulingPeriod = schedulingPeriod;
+        if (schedulingPeriod == null) {
+            schedulingNanos = null;
+        } else {
+            if (FormatUtils.TIME_DURATION_PATTERN.matcher(schedulingPeriod).matches()) {
+                schedulingNanos = FormatUtils.getTimeDuration(schedulingPeriod, TimeUnit.NANOSECONDS);
+            } else {
+                schedulingNanos = null;
+            }
+        }
 
         preparedQueries = new HashMap<>();
         for (final Map.Entry<PropertyDescriptor, String> entry : component.getProperties().entrySet()) {
@@ -61,4 +75,14 @@ public class StandardConfigurationContext implements ConfigurationContext {
     public Map<PropertyDescriptor, String> getProperties() {
         return component.getProperties();
     }
+
+    @Override
+    public String getSchedulingPeriod() {
+        return schedulingPeriod;
+    }
+
+    @Override
+    public Long getSchedulingPeriod(final TimeUnit timeUnit) {
+        return schedulingNanos == null ? null : timeUnit.convert(schedulingNanos, TimeUnit.NANOSECONDS);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ba315598/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
index e577ffe..85f5a32 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
@@ -125,7 +125,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
     @SuppressWarnings("deprecation")
     private void onConfigured() {
         try (final NarCloseable x = NarCloseable.withNarLoader()) {
-            final ConfigurationContext configContext = new StandardConfigurationContext(this, serviceProvider);
+            final ConfigurationContext configContext = new StandardConfigurationContext(this, serviceProvider, null);
             ReflectionUtils.invokeMethodsWithAnnotation(OnConfigured.class, implementation, configContext);
         } catch (final Exception e) {
             throw new ComponentLifeCycleException("Failed to invoke On-Configured Lifecycle methods of " + implementation, e);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ba315598/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
index f0de8d2..086891a 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
@@ -142,7 +142,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
 
                     final ControllerServiceNode node = serviceNodeHolder.get();
                     final ControllerServiceState state = node.getState();
-                    final boolean disabled = (state != ControllerServiceState.ENABLED); // only allow method call if service state is ENABLED.
+                    final boolean disabled = state != ControllerServiceState.ENABLED; // only allow method call if service state is ENABLED.
                     if (disabled && !validDisabledMethods.contains(method)) {
                         // Use nar class loader here because we are implicitly calling toString() on the original implementation.
                         try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
@@ -435,7 +435,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
     @Override
     public ControllerService getControllerService(final String serviceIdentifier) {
         final ControllerServiceNode node = controllerServices.get(serviceIdentifier);
-        return (node == null) ? null : node.getProxiedControllerService();
+        return node == null ? null : node.getProxiedControllerService();
     }
 
     @Override
@@ -446,13 +446,13 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
     @Override
     public boolean isControllerServiceEnabled(final String serviceIdentifier) {
         final ControllerServiceNode node = controllerServices.get(serviceIdentifier);
-        return (node == null) ? false : (ControllerServiceState.ENABLED == node.getState());
+        return node == null ? false : ControllerServiceState.ENABLED == node.getState();
     }
 
     @Override
     public boolean isControllerServiceEnabling(final String serviceIdentifier) {
         final ControllerServiceNode node = controllerServices.get(serviceIdentifier);
-        return (node == null) ? false : (ControllerServiceState.ENABLING == node.getState());
+        return node == null ? false : ControllerServiceState.ENABLING == node.getState();
     }
 
     @Override
@@ -488,7 +488,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
         serviceNode.verifyCanDelete();
 
         try (final NarCloseable x = NarCloseable.withNarLoader()) {
-            final ConfigurationContext configurationContext = new StandardConfigurationContext(serviceNode, this);
+            final ConfigurationContext configurationContext = new StandardConfigurationContext(serviceNode, this, null);
             ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, serviceNode.getControllerServiceImplementation(), configurationContext);
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ba315598/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/java/org/apache/nifi/controller/MonitorMemory.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/java/org/apache/nifi/controller/MonitorMemory.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/java/org/apache/nifi/controller/MonitorMemory.java
index bb8971a..83795b2 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/java/org/apache/nifi/controller/MonitorMemory.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/java/org/apache/nifi/controller/MonitorMemory.java
@@ -38,7 +38,6 @@ import org.apache.nifi.reporting.AbstractReportingTask;
 import org.apache.nifi.reporting.Bulletin;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.reporting.ReportingContext;
-import org.apache.nifi.reporting.ReportingInitializationContext;
 import org.apache.nifi.reporting.Severity;
 import org.apache.nifi.util.FormatUtils;
 import org.slf4j.Logger;
@@ -131,13 +130,6 @@ public class MonitorMemory extends AbstractReportingTask {
     public MonitorMemory() {
     }
 
-    private long schedulingPeriodMillis;
-
-    @Override
-    protected void init(final ReportingInitializationContext config) throws InitializationException {
-        schedulingPeriodMillis = config.getSchedulingPeriod(TimeUnit.MILLISECONDS);
-    }
-
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
         final List<PropertyDescriptor> descriptors = new ArrayList<>(3);
@@ -156,7 +148,7 @@ public class MonitorMemory extends AbstractReportingTask {
         // validate reporting interval
         final Long reportingIntervalValue = config.getProperty(REPORTING_INTERVAL).asTimePeriod(TimeUnit.MILLISECONDS);
         if (reportingIntervalValue == null) {
-            reportingIntervalMillis = schedulingPeriodMillis;
+            reportingIntervalMillis = config.getSchedulingPeriod(TimeUnit.MILLISECONDS);
         } else {
             reportingIntervalMillis = reportingIntervalValue;
         }
@@ -174,7 +166,7 @@ public class MonitorMemory extends AbstractReportingTask {
                 } else {
                     final String percentage = thresholdValue.substring(0, thresholdValue.length() - 1);
                     final double pct = Double.parseDouble(percentage) / 100D;
-                    final long calculatedThreshold = (long) ((double) bean.getUsage().getMax() * pct);
+                    final long calculatedThreshold = (long) (bean.getUsage().getMax() * pct);
                     if (bean.isCollectionUsageThresholdSupported()) {
                         bean.setCollectionUsageThreshold(calculatedThreshold);
                     }
@@ -201,7 +193,7 @@ public class MonitorMemory extends AbstractReportingTask {
         } else {
             final String percentage = threshold.substring(0, threshold.length() - 1);
             final double pct = Double.parseDouble(percentage) / 100D;
-            return (long) ((double) maxBytes * pct);
+            return (long) (maxBytes * pct);
         }
     }
 


[2/2] incubator-nifi git commit: NIFI-719: merged with develop

Posted by ma...@apache.org.
NIFI-719: merged with develop


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

Branch: refs/heads/develop
Commit: a09180799d8a84108c3f5665668bded284768ecf
Parents: 5021570 ba31559
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Jul 1 10:44:42 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Jul 1 10:44:42 2015 -0400

----------------------------------------------------------------------
 .../nifi/controller/ConfigurationContext.java   | 23 +++++++++++++++--
 .../nifi/util/MockConfigurationContext.java     | 11 +++++++++
 .../mock/MockConfigurationContext.java          | 10 ++++++++
 .../apache/nifi/controller/FlowController.java  | 18 +++++++-------
 .../reporting/AbstractReportingTaskNode.java    |  4 +--
 .../scheduling/StandardProcessScheduler.java    | 24 +++++++++---------
 .../service/StandardConfigurationContext.java   | 26 +++++++++++++++++++-
 .../service/StandardControllerServiceNode.java  |  2 +-
 .../StandardControllerServiceProvider.java      | 10 ++++----
 .../apache/nifi/controller/MonitorMemory.java   | 14 +++--------
 10 files changed, 99 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a0918079/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java
----------------------------------------------------------------------
diff --cc nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java
index 742f03b,e9afb32..c90e722
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockConfigurationContext.java
@@@ -55,12 -49,13 +56,22 @@@ public class MockConfigurationContext i
          return new HashMap<>(this.properties);
      }
  
 +    private PropertyDescriptor getActualDescriptor(final PropertyDescriptor property) {
 +        if (service == null) {
 +            return property;
 +        }
 +
 +        final PropertyDescriptor resolved = service.getPropertyDescriptor(property.getName());
 +        return resolved == null ? property : resolved;
 +    }
++
+     @Override
+     public String getSchedulingPeriod() {
+         return "0 secs";
+     }
+ 
+     @Override
+     public Long getSchedulingPeriod(final TimeUnit timeUnit) {
+         return 0L;
+     }
  }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a0918079/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockConfigurationContext.java
----------------------------------------------------------------------
diff --cc nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockConfigurationContext.java
index 1ba5499,0000000..6c9ec9d
mode 100644,000000..100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockConfigurationContext.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockConfigurationContext.java
@@@ -1,38 -1,0 +1,48 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.nifi.documentation.mock;
 +
 +import java.util.Collections;
 +import java.util.Map;
++import java.util.concurrent.TimeUnit;
 +
 +import org.apache.nifi.components.PropertyDescriptor;
 +import org.apache.nifi.components.PropertyValue;
 +import org.apache.nifi.controller.ConfigurationContext;
 +
 +public class MockConfigurationContext implements ConfigurationContext {
 +
 +    @Override
 +    public PropertyValue getProperty(PropertyDescriptor property) {
 +        return null;
 +    }
 +
 +    @Override
 +    public Map<PropertyDescriptor, String> getProperties() {
 +        return Collections.emptyMap();
 +    }
 +
++    @Override
++    public String getSchedulingPeriod() {
++        return "0 secs";
++    }
++
++    @Override
++    public Long getSchedulingPeriod(final TimeUnit timeUnit) {
++        return 0L;
++    }
 +}