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 2017/08/03 18:41:00 UTC

nifi-minifi git commit: MINIFI-347: Adding tests for C2 file system cache.

Repository: nifi-minifi
Updated Branches:
  refs/heads/master 101763a87 -> 40cb28e62


MINIFI-347: Adding tests for C2 file system cache.

This closes #88.

Signed-off-by: Aldrin Piri <al...@apache.org>


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

Branch: refs/heads/master
Commit: 40cb28e625055913e84826431043cf89d8ee3cb9
Parents: 101763a
Author: jzonthemtn <jz...@apache.org>
Authored: Tue Aug 1 10:26:51 2017 -0400
Committer: Aldrin Piri <al...@apache.org>
Committed: Thu Aug 3 14:40:19 2017 -0400

----------------------------------------------------------------------
 .../minifi/c2/api/cache/ConfigurationCache.java |  24 ++--
 .../minifi-c2-cache-filesystem/pom.xml          |   6 +
 .../FileSystemConfigurationCache.java           |  34 +++--
 .../FileSystemConfigurationCacheTest.java       | 125 +++++++++++++++++++
 .../test/resources/files/config.text.yaml.v1    |  63 ++++++++++
 pom.xml                                         |   1 +
 6 files changed, 235 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/40cb28e6/minifi-c2/minifi-c2-api/src/main/java/org/apache/nifi/minifi/c2/api/cache/ConfigurationCache.java
----------------------------------------------------------------------
diff --git a/minifi-c2/minifi-c2-api/src/main/java/org/apache/nifi/minifi/c2/api/cache/ConfigurationCache.java b/minifi-c2/minifi-c2-api/src/main/java/org/apache/nifi/minifi/c2/api/cache/ConfigurationCache.java
index 43d4b93..fdfd868 100644
--- a/minifi-c2/minifi-c2-api/src/main/java/org/apache/nifi/minifi/c2/api/cache/ConfigurationCache.java
+++ b/minifi-c2/minifi-c2-api/src/main/java/org/apache/nifi/minifi/c2/api/cache/ConfigurationCache.java
@@ -17,21 +17,23 @@
 
 package org.apache.nifi.minifi.c2.api.cache;
 
-import org.apache.nifi.minifi.c2.api.InvalidParameterException;
-
 import java.util.List;
 import java.util.Map;
 
+import org.apache.nifi.minifi.c2.api.InvalidParameterException;
+
 /**
- * Cache for storing configurations so they don't have to be pulled from the provider more often than necessary
+ * Cache for storing configurations so they don't have to be pulled from
+ * the provider more often than necessary.
  */
 public interface ConfigurationCache {
-    /**
-     * Returns the information on a given cache entry
-     *
-     * @param parameters the parameters that identify the entry
-     * @return information on the entry
-     * @throws InvalidParameterException if there are illegal/invalid parameters
-     */
-    ConfigurationCacheFileInfo getCacheFileInfo(String contentType, Map<String, List<String>> parameters) throws InvalidParameterException;
+  /**
+  * Returns the information on a given cache entry.
+  *
+  * @param parameters The parameters that identify the entry.
+  * @return {@link ConfigurationCacheFileInfo information} on the entry.
+  * @throws InvalidParameterException Thrown when there are illegal/invalid parameters.
+  */
+  ConfigurationCacheFileInfo getCacheFileInfo(String contentType,
+      Map<String, List<String>> parameters) throws InvalidParameterException;
 }

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/40cb28e6/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/pom.xml b/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/pom.xml
index 18ffa54..62026bc 100644
--- a/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/pom.xml
+++ b/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/pom.xml
@@ -31,5 +31,11 @@ limitations under the License.
             <artifactId>minifi-c2-api</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>com.github.stefanbirkner</groupId>
+            <artifactId>system-rules</artifactId>
+            <version>${system.rules.version}</version>
+            <scope>test</scope>
+       </dependency>
     </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/40cb28e6/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/src/main/java/org/apache/nifi/minifi/c2/cache/filesystem/FileSystemConfigurationCache.java
----------------------------------------------------------------------
diff --git a/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/src/main/java/org/apache/nifi/minifi/c2/cache/filesystem/FileSystemConfigurationCache.java b/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/src/main/java/org/apache/nifi/minifi/c2/cache/filesystem/FileSystemConfigurationCache.java
index 1e26009..b6b9cb7 100644
--- a/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/src/main/java/org/apache/nifi/minifi/c2/cache/filesystem/FileSystemConfigurationCache.java
+++ b/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/src/main/java/org/apache/nifi/minifi/c2/cache/filesystem/FileSystemConfigurationCache.java
@@ -17,13 +17,6 @@
 
 package org.apache.nifi.minifi.c2.cache.filesystem;
 
-import org.apache.nifi.minifi.c2.api.InvalidParameterException;
-import org.apache.nifi.minifi.c2.api.cache.ConfigurationCache;
-import org.apache.nifi.minifi.c2.api.cache.ConfigurationCacheFileInfo;
-import org.apache.nifi.minifi.c2.api.util.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
@@ -33,18 +26,45 @@ import java.util.Map;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 
+import org.apache.nifi.minifi.c2.api.InvalidParameterException;
+import org.apache.nifi.minifi.c2.api.cache.ConfigurationCache;
+import org.apache.nifi.minifi.c2.api.cache.ConfigurationCacheFileInfo;
+import org.apache.nifi.minifi.c2.api.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation of {@link ConfigurationCache} that uses the local file
+ * system for caching configurations.
+ *
+ */
 public class FileSystemConfigurationCache implements ConfigurationCache {
     private static final Logger logger = LoggerFactory.getLogger(FileSystemConfigurationCache.class);
 
     private final Path pathRoot;
     private final String pathPattern;
 
+    /**
+     * Creates a new cache.
+     * @param pathRoot The root path for configurations. This path will be appended
+     * to the environment variable <code>C2_SERVER_HOME</code>.
+     * @param pathPattern The pattern to determine the path.
+     * @throws IOException Thrown if the path cannot be created.
+     */
     public FileSystemConfigurationCache(String pathRoot, String pathPattern) throws IOException {
         this.pathRoot = Paths.get(System.getenv("C2_SERVER_HOME")).resolve(pathRoot).toAbsolutePath();
         Files.createDirectories(this.pathRoot);
         this.pathPattern = pathPattern;
     }
 
+    /**
+     * Resolves a parent {@link Path path} and child directory.
+     * @param parent The parent {@link Path path}.
+     * @param s The child directory.
+     * @return The resolved {@link Path}.
+     * @throws InvalidParameterException Thrown if the child directory is
+     * not actually a child directory of the parent.
+     */
     protected Path resolveChildAndVerifyParent(Path parent, String s) throws InvalidParameterException {
         Path child = parent.resolve(s).toAbsolutePath();
         if (child.toAbsolutePath().getParent().equals(parent)) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/40cb28e6/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/src/test/java/org/apache/nifi/minfi/c2/cache/filesystem/FileSystemConfigurationCacheTest.java
----------------------------------------------------------------------
diff --git a/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/src/test/java/org/apache/nifi/minfi/c2/cache/filesystem/FileSystemConfigurationCacheTest.java b/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/src/test/java/org/apache/nifi/minfi/c2/cache/filesystem/FileSystemConfigurationCacheTest.java
new file mode 100644
index 0000000..82948b5
--- /dev/null
+++ b/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/src/test/java/org/apache/nifi/minfi/c2/cache/filesystem/FileSystemConfigurationCacheTest.java
@@ -0,0 +1,125 @@
+/*
+ * 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.minfi.c2.cache.filesystem;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+
+import org.apache.nifi.minifi.c2.api.ConfigurationProviderException;
+import org.apache.nifi.minifi.c2.api.InvalidParameterException;
+import org.apache.nifi.minifi.c2.api.cache.ConfigurationCacheFileInfo;
+import org.apache.nifi.minifi.c2.api.cache.WriteableConfiguration;
+import org.apache.nifi.minifi.c2.cache.filesystem.FileSystemConfigurationCache;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.EnvironmentVariables;
+
+public class FileSystemConfigurationCacheTest {
+
+  @Rule
+  public final EnvironmentVariables environmentVariables = new EnvironmentVariables();
+
+  @Before
+  public void before() {
+    File resourcesDirectory = new File("src/test/resources/");
+    environmentVariables.set("C2_SERVER_HOME", resourcesDirectory.getAbsolutePath());
+  }
+
+  @Test
+  public void getConfigurationTest() throws IOException, ConfigurationProviderException {
+
+    final String pathRoot = "files";
+    final String pathPattern = "config";
+
+    FileSystemConfigurationCache cache = new FileSystemConfigurationCache(pathRoot, pathPattern);
+
+    Map<String, List<String>> parameters = new HashMap<>();
+
+    ConfigurationCacheFileInfo info = cache.getCacheFileInfo("text/yaml", parameters);
+
+    WriteableConfiguration configuration = info.getConfiguration(1);
+
+    assertEquals("config.text.yaml.v1", configuration.getName());
+    assertEquals("1", configuration.getVersion());
+    assertTrue(configuration.exists());
+
+  }
+
+  @Test
+  public void getNonexistantConfigurationTest() throws IOException, ConfigurationProviderException {
+
+    final String pathRoot = "files";
+    final String pathPattern = "config";
+
+    FileSystemConfigurationCache cache = new FileSystemConfigurationCache(pathRoot, pathPattern);
+
+    Map<String, List<String>> parameters = new HashMap<>();
+
+    ConfigurationCacheFileInfo info = cache.getCacheFileInfo("test/contenttype", parameters);
+
+    WriteableConfiguration configuration = info.getConfiguration(1);
+
+    assertEquals("config.test.contenttype.v1", configuration.getName());
+    assertEquals("1", configuration.getVersion());
+    assertFalse(configuration.exists());
+
+  }
+
+  @Test
+  public void getCachedConfigurationsTest() throws IOException, ConfigurationProviderException {
+
+    final String pathRoot = "files";
+    final String pathPattern = "config";
+
+    FileSystemConfigurationCache cache = new FileSystemConfigurationCache(pathRoot, pathPattern);
+
+    Map<String, List<String>> parameters = new HashMap<>();
+
+    ConfigurationCacheFileInfo info = cache.getCacheFileInfo("text/yaml", parameters);
+
+    Stream<WriteableConfiguration> configs = info.getCachedConfigurations();
+
+    assertEquals(1, configs.count());
+
+  }
+
+  @Test(expected = InvalidParameterException.class)
+  public void getConfigurationInvalidParametersTest() throws IOException,
+      InvalidParameterException {
+
+    final String pathRoot = "files";
+    final String pathPattern = "${test}/config";
+
+    FileSystemConfigurationCache cache = new FileSystemConfigurationCache(pathRoot, pathPattern);
+
+    Map<String, List<String>> parameters = new HashMap<>();
+
+    cache.getCacheFileInfo("test/contenttype", parameters);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/40cb28e6/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/src/test/resources/files/config.text.yaml.v1
----------------------------------------------------------------------
diff --git a/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/src/test/resources/files/config.text.yaml.v1 b/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/src/test/resources/files/config.text.yaml.v1
new file mode 100644
index 0000000..5237bc1
--- /dev/null
+++ b/minifi-c2/minifi-c2-cache/minifi-c2-cache-filesystem/src/test/resources/files/config.text.yaml.v1
@@ -0,0 +1,63 @@
+# 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.
+
+MiNiFi Config Version: 3
+Flow Controller:
+  name: MiNiFi Flow
+  comment: ''
+Core Properties:
+  flow controller graceful shutdown period: 10 sec
+  flow service write delay interval: 500 ms
+  administrative yield duration: 30 sec
+  bored yield duration: 10 millis
+  max concurrent threads: 1
+FlowFile Repository:
+  partitions: 256
+  checkpoint interval: 2 mins
+  always sync: false
+  Swap:
+    threshold: 20000
+    in period: 5 sec
+    in threads: 1
+    out period: 5 sec
+    out threads: 4
+Content Repository:
+  content claim max appendable size: 10 MB
+  content claim max flow files: 100
+  always sync: false
+Provenance Repository:
+  provenance rollover time: 1 min
+Component Status Repository:
+  buffer size: 1440
+  snapshot frequency: 1 min
+Security Properties:
+  keystore: ''
+  keystore type: ''
+  keystore password: ''
+  key password: ''
+  truststore: ''
+  truststore type: ''
+  truststore password: ''
+  ssl protocol: ''
+  Sensitive Props:
+    key: ''
+    algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+    provider: BC
+Processors: []
+Process Groups: []
+Funnels: []
+Connections: []
+Remote Process Groups: []
+NiFi Properties Overrides: {}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/40cb28e6/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e882e13..a07f995 100644
--- a/pom.xml
+++ b/pom.xml
@@ -105,6 +105,7 @@ limitations under the License.
         <yammer.metrics.version>2.2.0</yammer.metrics.version>
         <spring.version>4.2.4.RELEASE</spring.version>
         <spring.security.version>4.0.3.RELEASE</spring.security.version>
+        <system.rules.version>1.16.1</system.rules.version>
     </properties>
 
     <dependencies>