You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by al...@apache.org on 2016/04/12 22:51:32 UTC

nifi-minifi git commit: MINIFI-2 MINIFI-4 Establishing a base implementation of the configuration change listener and notification services as well as a File implementation.

Repository: nifi-minifi
Updated Branches:
  refs/heads/master 5751e23d4 -> 59f2d4418


MINIFI-2 MINIFI-4 Establishing a base implementation of the configuration change listener and notification services as well as a File implementation.


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

Branch: refs/heads/master
Commit: 59f2d441867eb349104795cbe98d490385eb415a
Parents: 5751e23
Author: Aldrin Piri <al...@apache.org>
Authored: Fri Apr 8 14:59:58 2016 -0400
Committer: Aldrin Piri <al...@apache.org>
Committed: Tue Apr 12 16:49:37 2016 -0400

----------------------------------------------------------------------
 .../ConfigurationChangeListener.java            |  34 +++
 .../ConfigurationChangeNotifier.java            |  57 +++++
 .../minifi-framework/minifi-runtime/pom.xml     |  28 ++-
 .../configuration/FileChangeNotifier.java       | 173 ++++++++++++++++
 .../configuration/TestFileChangeNotifier.java   | 206 +++++++++++++++++++
 .../src/test/resources/config.yml               |   0
 pom.xml                                         |  41 ++++
 7 files changed, 533 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/59f2d441/minifi-api/src/main/java/org/apache/nifi/minifi/configuration/ConfigurationChangeListener.java
----------------------------------------------------------------------
diff --git a/minifi-api/src/main/java/org/apache/nifi/minifi/configuration/ConfigurationChangeListener.java b/minifi-api/src/main/java/org/apache/nifi/minifi/configuration/ConfigurationChangeListener.java
new file mode 100644
index 0000000..0d09f72
--- /dev/null
+++ b/minifi-api/src/main/java/org/apache/nifi/minifi/configuration/ConfigurationChangeListener.java
@@ -0,0 +1,34 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.minifi.configuration;
+
+import java.io.InputStream;
+
+/**
+ * Interface for handling events detected and driven by an associated {@link ConfigurationChangeNotifier} to which the listener
+ * has registered via {@link ConfigurationChangeNotifier#registerListener(ConfigurationChangeListener)}.
+ */
+public interface ConfigurationChangeListener {
+
+    /**
+     * Provides a mechanism for the implementation to interpret the specified configuration change
+     *
+     * @param is stream of the detected content received from the change notifier
+     */
+    void handleChange(InputStream is);
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/59f2d441/minifi-api/src/main/java/org/apache/nifi/minifi/configuration/ConfigurationChangeNotifier.java
----------------------------------------------------------------------
diff --git a/minifi-api/src/main/java/org/apache/nifi/minifi/configuration/ConfigurationChangeNotifier.java b/minifi-api/src/main/java/org/apache/nifi/minifi/configuration/ConfigurationChangeNotifier.java
new file mode 100644
index 0000000..30a98c2
--- /dev/null
+++ b/minifi-api/src/main/java/org/apache/nifi/minifi/configuration/ConfigurationChangeNotifier.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.minifi.configuration;
+
+import java.util.Properties;
+import java.util.Set;
+
+public interface ConfigurationChangeNotifier {
+
+
+    /**
+     * Provides an opportunity for the implementation to perform configuration and initialization based on properties received from the bootstrapping configuration
+     *
+     * @param properties from the bootstrap configuration
+     */
+    void initialize(Properties properties);
+
+    /**
+     * Begins the associated notification service provided by the given implementation.  In most implementations, no action will occur until this method is invoked.
+     */
+    void start();
+
+    /**
+     * Provides an immutable collection of listeners for the notifier instance
+     *
+     * @return a collection of those listeners registered for notifications
+     */
+    Set<ConfigurationChangeListener> getChangeListeners();
+
+    /**
+     * Adds a listener to be notified of configuration changes
+     *
+     * @param listener to be added to the collection
+     * @return true if the listener was added; false if already registered
+     */
+    boolean registerListener(ConfigurationChangeListener listener);
+
+    /**
+     * Provide the mechanism by which listeners are notified
+     */
+    void notifyListeners();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/59f2d441/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/pom.xml b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/pom.xml
index d8ec39a..a656a7a 100644
--- a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/pom.xml
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/pom.xml
@@ -18,17 +18,14 @@ limitations under the License.
 <project xmlns="http://maven.apache.org/POM/4.0.0"
          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
     <parent>
-        <artifactId>minifi-framework</artifactId>
         <groupId>org.apache.nifi.minifi</groupId>
+        <artifactId>minifi-framework</artifactId>
         <version>0.0.1-SNAPSHOT</version>
     </parent>
-    <modelVersion>4.0.0</modelVersion>
 
-    <groupId>org.apache.nifi.minifi</groupId>
     <artifactId>minifi-runtime</artifactId>
-    <packaging>jar</packaging>
-
 
     <dependencies>
         <dependency>
@@ -39,5 +36,24 @@ limitations under the License.
             <groupId>org.slf4j</groupId>
             <artifactId>jul-to-slf4j</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi.minifi</groupId>
+            <artifactId>minifi-api</artifactId>
+        </dependency>
     </dependencies>
-</project>
\ No newline at end of file
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <configuration>
+                    <excludes combine.children="append">
+                        <exclude>src/test/resources/config.yml</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/59f2d441/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/configuration/FileChangeNotifier.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/configuration/FileChangeNotifier.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/configuration/FileChangeNotifier.java
new file mode 100644
index 0000000..36c5968
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/main/java/org/apache/nifi/minifi/configuration/FileChangeNotifier.java
@@ -0,0 +1,173 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.minifi.configuration;
+
+import static java.nio.file.StandardWatchEventKinds.ENTRY_MODIFY;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.file.FileSystems;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.WatchEvent;
+import java.nio.file.WatchKey;
+import java.nio.file.WatchService;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * FileChangeNotifier provides a simple FileSystem monitor for detecting changes for a specified file as generated from its corresponding {@link Path}.  Upon modifications to the associated file,
+ * associated listeners receive notification of a change allowing configuration logic to be reanalyzed.  The backing implementation is associated with a {@link ScheduledExecutorService} that
+ * ensures continuity of monitoring.
+ */
+public class FileChangeNotifier implements Runnable, ConfigurationChangeNotifier, Closeable {
+
+    private Path configFile;
+    private WatchService watchService;
+    private long pollingSeconds;
+
+    private final ScheduledExecutorService executorService = Executors.newSingleThreadScheduledExecutor();
+    private final Set<ConfigurationChangeListener> configurationChangeListeners = new HashSet<>();
+
+    protected static final String CONFIG_FILE_PATH_KEY = "nifi.minifi.notifier.file.config.path";
+    protected static final String POLLING_PERIOD_INTERVAL_KEY = "nifi.minifi.notifier.file.polling.period.seconds";
+
+    protected static final int DEFAULT_POLLING_PERIOD_INTERVAL = 15;
+    protected static final TimeUnit DEFAULT_POLLING_PERIOD_UNIT = TimeUnit.SECONDS;
+
+    @Override
+    public Set<ConfigurationChangeListener> getChangeListeners() {
+        return Collections.unmodifiableSet(configurationChangeListeners);
+    }
+
+    @Override
+    public void notifyListeners() {
+        final File fileToRead = configFile.toFile();
+        for (final ConfigurationChangeListener listener : getChangeListeners()) {
+            try (final FileInputStream fis = new FileInputStream(fileToRead);) {
+                listener.handleChange(fis);
+            } catch (IOException ex) {
+                throw new IllegalStateException("Unable to read the changed file " + configFile, ex);
+            }
+        }
+    }
+
+    @Override
+    public boolean registerListener(ConfigurationChangeListener listener) {
+        return this.configurationChangeListeners.add(listener);
+    }
+
+    protected boolean targetChanged() {
+        boolean targetChanged = false;
+
+        final WatchKey watchKey = this.watchService.poll();
+
+        if (watchKey == null) {
+            return targetChanged;
+        }
+
+        for (WatchEvent<?> watchEvt : watchKey.pollEvents()) {
+            final WatchEvent.Kind<?> evtKind = watchEvt.kind();
+
+            final WatchEvent<Path> pathEvent = (WatchEvent<Path>) watchEvt;
+            final Path changedFile = pathEvent.context();
+
+            // determine target change by verifying if the changed file corresponds to the config file monitored for this path
+            targetChanged = (evtKind == ENTRY_MODIFY && changedFile.equals(configFile.getName(configFile.getNameCount() - 1)));
+        }
+
+        // After completing inspection, reset for detection of subsequent change events
+        boolean valid = watchKey.reset();
+        if (!valid) {
+            throw new IllegalStateException("Unable to reinitialize file system watcher.");
+        }
+
+        return targetChanged;
+    }
+
+    protected static WatchService initializeWatcher(Path filePath) {
+        try {
+            final WatchService fsWatcher = FileSystems.getDefault().newWatchService();
+            final Path watchDirectory = filePath.getParent();
+            watchDirectory.register(fsWatcher, ENTRY_MODIFY);
+
+            return fsWatcher;
+        } catch (IOException ioe) {
+            throw new IllegalStateException("Unable to initialize a file system watcher for the path " + filePath, ioe);
+        }
+    }
+
+    @Override
+    public void run() {
+        if (targetChanged()) {
+            notifyListeners();
+        }
+    }
+
+    @Override
+    public void initialize(Properties properties) {
+        final String rawPath = properties.getProperty(CONFIG_FILE_PATH_KEY);
+        final String rawPollingDuration = properties.getProperty(POLLING_PERIOD_INTERVAL_KEY, Long.toString(DEFAULT_POLLING_PERIOD_INTERVAL));
+
+        try {
+            setConfigFile(Paths.get(rawPath));
+            setPollingPeriod(Long.parseLong(rawPollingDuration), DEFAULT_POLLING_PERIOD_UNIT);
+            setWatchService(initializeWatcher(configFile));
+        } catch (Exception e) {
+            throw new IllegalArgumentException("Could not successfully initialize file change notifier.", e);
+        }
+    }
+
+    protected void setConfigFile(Path configFile) {
+        final File file = configFile.toFile();
+        if (!file.exists() || !file.canRead() || !file.isFile()) {
+            throw new IllegalArgumentException(String.format("The specified path %s must be a readable file.", configFile));
+        }
+        this.configFile = configFile;
+    }
+
+    protected void setWatchService(WatchService watchService) {
+        this.watchService = watchService;
+    }
+
+    protected void setPollingPeriod(long duration, TimeUnit unit) {
+        if (duration < 0) {
+            throw new IllegalArgumentException("Cannot specify a polling period with duration <=0");
+        }
+        this.pollingSeconds = TimeUnit.SECONDS.convert(duration, unit);
+    }
+
+    @Override
+    public void start() {
+        this.executorService.scheduleWithFixedDelay(this, 0, pollingSeconds, DEFAULT_POLLING_PERIOD_UNIT);
+    }
+
+    @Override
+    public void close() {
+        if (!this.executorService.isShutdown() || !this.executorService.isTerminated()) {
+            this.executorService.shutdownNow();
+        }
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/59f2d441/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/test/java/org/apache/nifi/minifi/configuration/TestFileChangeNotifier.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/test/java/org/apache/nifi/minifi/configuration/TestFileChangeNotifier.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/test/java/org/apache/nifi/minifi/configuration/TestFileChangeNotifier.java
new file mode 100644
index 0000000..d75a436
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/test/java/org/apache/nifi/minifi/configuration/TestFileChangeNotifier.java
@@ -0,0 +1,206 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.minifi.configuration;
+
+import static java.nio.file.StandardWatchEventKinds.ENTRY_MODIFY;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.InputStream;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.WatchEvent;
+import java.nio.file.WatchKey;
+import java.nio.file.WatchService;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestFileChangeNotifier {
+
+    private static final String CONFIG_FILENAME = "config.yml";
+    private static final String TEST_CONFIG_PATH = "src/test/resources/config.yml";
+
+    private FileChangeNotifier notifierSpy;
+    private WatchService mockWatchService;
+    private Properties testProperties;
+
+    @Before
+    public void setUp() throws Exception {
+        mockWatchService = Mockito.mock(WatchService.class);
+        notifierSpy = Mockito.spy(new FileChangeNotifier());
+        notifierSpy.setConfigFile(Paths.get(TEST_CONFIG_PATH));
+        notifierSpy.setWatchService(mockWatchService);
+
+        testProperties = new Properties();
+        testProperties.put(FileChangeNotifier.CONFIG_FILE_PATH_KEY, TEST_CONFIG_PATH);
+        testProperties.put(FileChangeNotifier.POLLING_PERIOD_INTERVAL_KEY, FileChangeNotifier.DEFAULT_POLLING_PERIOD_INTERVAL);
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        notifierSpy.close();
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testInitialize_invalidFile() throws Exception {
+        testProperties.put(FileChangeNotifier.CONFIG_FILE_PATH_KEY, "/land/of/make/believe");
+        notifierSpy.initialize(testProperties);
+    }
+
+    @Test
+    public void testInitialize_validFile() throws Exception {
+        notifierSpy.initialize(testProperties);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testInitialize_invalidPollingPeriod() throws Exception {
+        testProperties.put(FileChangeNotifier.POLLING_PERIOD_INTERVAL_KEY, "abc");
+        notifierSpy.initialize(testProperties);
+    }
+
+    @Test
+    public void testInitialize_useDefaultPolling() throws Exception {
+        testProperties.remove(FileChangeNotifier.POLLING_PERIOD_INTERVAL_KEY);
+        notifierSpy.initialize(testProperties);
+    }
+
+
+    @Test
+    public void testNotifyListeners() throws Exception {
+        final ConfigurationChangeListener testListener = Mockito.mock(ConfigurationChangeListener.class);
+        boolean wasRegistered = notifierSpy.registerListener(testListener);
+
+        Assert.assertTrue("Registration did not correspond to newly added listener", wasRegistered);
+        Assert.assertEquals("Did not receive the correct number of registered listeners", notifierSpy.getChangeListeners().size(), 1);
+
+        notifierSpy.notifyListeners();
+
+        verify(testListener, Mockito.atMost(1)).handleChange(Mockito.any(InputStream.class));
+    }
+
+    @Test
+    public void testRegisterListener() throws Exception {
+        final ConfigurationChangeListener firstListener = Mockito.mock(ConfigurationChangeListener.class);
+        boolean wasRegistered = notifierSpy.registerListener(firstListener);
+
+        Assert.assertTrue("Registration did not correspond to newly added listener", wasRegistered);
+        Assert.assertEquals("Did not receive the correct number of registered listeners", notifierSpy.getChangeListeners().size(), 1);
+
+        final ConfigurationChangeListener secondListener = Mockito.mock(ConfigurationChangeListener.class);
+        wasRegistered = notifierSpy.registerListener(secondListener);
+        Assert.assertEquals("Did not receive the correct number of registered listeners", notifierSpy.getChangeListeners().size(), 2);
+
+    }
+
+    @Test
+    public void testRegisterDuplicateListener() throws Exception {
+        final ConfigurationChangeListener firstListener = Mockito.mock(ConfigurationChangeListener.class);
+        boolean wasRegistered = notifierSpy.registerListener(firstListener);
+
+        Assert.assertTrue("Registration did not correspond to newly added listener", wasRegistered);
+        Assert.assertEquals("Did not receive the correct number of registered listeners", notifierSpy.getChangeListeners().size(), 1);
+
+        wasRegistered = notifierSpy.registerListener(firstListener);
+
+        Assert.assertEquals("Did not receive the correct number of registered listeners", notifierSpy.getChangeListeners().size(), 1);
+        Assert.assertFalse("Registration did not correspond to newly added listener", wasRegistered);
+    }
+
+    /* Verify handleChange events */
+    @Test
+    public void testTargetChangedNoModification() throws Exception {
+        final ConfigurationChangeListener testListener = Mockito.mock(ConfigurationChangeListener.class);
+
+        // In this case the WatchKey is null because there were no events found
+        establishMockEnvironmentForChangeTests(testListener, null);
+
+        verify(testListener, Mockito.never()).handleChange(Mockito.any(InputStream.class));
+    }
+
+    @Test
+    public void testTargetChangedWithModificationEvent_nonConfigFile() throws Exception {
+        final ConfigurationChangeListener testListener = Mockito.mock(ConfigurationChangeListener.class);
+
+        // In this case, we receive a trigger event for the directory monitored, but it was another file not being monitored
+        final WatchKey mockWatchKey = createMockWatchKeyForPath("footage_not_found.yml");
+
+        establishMockEnvironmentForChangeTests(testListener, mockWatchKey);
+
+        notifierSpy.targetChanged();
+
+        verify(testListener, Mockito.never()).handleChange(Mockito.any(InputStream.class));
+    }
+
+    @Test
+    public void testTargetChangedWithModificationEvent() throws Exception {
+        final ConfigurationChangeListener testListener = Mockito.mock(ConfigurationChangeListener.class);
+
+        final WatchKey mockWatchKey = createMockWatchKeyForPath(CONFIG_FILENAME);
+        // Provided as a spy to allow injection of mock objects for some tests when dealing with the finalized FileSystems class
+        establishMockEnvironmentForChangeTests(testListener, mockWatchKey);
+
+        // Invoke the method of interest
+        notifierSpy.run();
+
+        verify(mockWatchService, Mockito.atLeastOnce()).poll();
+        verify(testListener, Mockito.atLeastOnce()).handleChange(Mockito.any(InputStream.class));
+    }
+
+    /* Helper methods to establish mock environment */
+    private WatchKey createMockWatchKeyForPath(String configFilePath) {
+        final WatchKey mockWatchKey = Mockito.mock(WatchKey.class);
+        final List<WatchEvent<?>> mockWatchEvents = (List<WatchEvent<?>>) Mockito.mock(List.class);
+        when(mockWatchKey.pollEvents()).thenReturn(mockWatchEvents);
+        when(mockWatchKey.reset()).thenReturn(true);
+
+        final Iterator mockIterator = Mockito.mock(Iterator.class);
+        when(mockWatchEvents.iterator()).thenReturn(mockIterator);
+
+        final WatchEvent mockWatchEvent = Mockito.mock(WatchEvent.class);
+        when(mockIterator.hasNext()).thenReturn(true, false);
+        when(mockIterator.next()).thenReturn(mockWatchEvent);
+
+        // In this case, we receive a trigger event for the directory monitored, and it was the file monitored
+        when(mockWatchEvent.context()).thenReturn(Paths.get(configFilePath));
+        when(mockWatchEvent.kind()).thenReturn(ENTRY_MODIFY);
+
+        return mockWatchKey;
+    }
+
+    private void establishMockEnvironmentForChangeTests(ConfigurationChangeListener listener, final WatchKey watchKey) throws Exception {
+        final boolean wasRegistered = notifierSpy.registerListener(listener);
+
+        // Establish the file mock and its parent directory
+        final Path mockConfigFilePath = Mockito.mock(Path.class);
+        final Path mockConfigFileParentPath = Mockito.mock(Path.class);
+
+        // When getting the parent of the file, get the directory
+        when(mockConfigFilePath.getParent()).thenReturn(mockConfigFileParentPath);
+
+        Assert.assertTrue("Registration did not correspond to newly added listener", wasRegistered);
+        Assert.assertEquals("Did not receive the correct number of registered listeners", notifierSpy.getChangeListeners().size(), 1);
+
+        when(mockWatchService.poll()).thenReturn(watchKey);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/59f2d441/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/test/resources/config.yml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/test/resources/config.yml b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-runtime/src/test/resources/config.yml
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/59f2d441/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e4967f9..2c8a881 100644
--- a/pom.xml
+++ b/pom.xml
@@ -96,6 +96,24 @@ limitations under the License.
         <org.apache.nifi.version>0.6.0</org.apache.nifi.version>
     </properties>
 
+    <dependencies>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
     <dependencyManagement>
         <dependencies>
             <dependency>
@@ -327,6 +345,29 @@ limitations under the License.
                 <artifactId>nifi-write-ahead-log</artifactId>
                 <version>${org.apache.nifi.version}</version>
             </dependency>
+
+            <!-- Test Dependencies -->
+            <dependency>
+                <groupId>junit</groupId>
+                <artifactId>junit</artifactId>
+                <version>4.12</version>
+            </dependency>
+            <dependency>
+                <groupId>org.mockito</groupId>
+                <artifactId>mockito-core</artifactId>
+                <version>1.10.19</version>
+            </dependency>
+            <dependency>
+                <groupId>org.mockito</groupId>
+                <artifactId>mockito-all</artifactId>
+                <version>1.10.19</version>
+                <scope>test</scope>
+            </dependency>
+            <dependency>
+                <groupId>org.slf4j</groupId>
+                <artifactId>slf4j-simple</artifactId>
+                <version>${org.slf4j.version}</version>
+            </dependency>
         </dependencies>
     </dependencyManagement>