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/05/19 15:36:49 UTC

nifi-minifi-cpp git commit: MINIFI-290 Improve Yaml Configuration [Forced Update!]

Repository: nifi-minifi-cpp
Updated Branches:
  refs/heads/master a57082a42 -> bc0d65e1f (forced update)


MINIFI-290 Improve Yaml Configuration

- Correct doxygen function comment in YamlConfiguration
- Correct a log message in YamlConfiguration
- Tweak ubuntu Dockerfile to fix docker build target
- Add --output-on-failure flag to Travis config

This closes #95.

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


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

Branch: refs/heads/master
Commit: bc0d65e1f802cf2c33b053eb9160ece58109f2da
Parents: 082c85a
Author: Kevin Doran <kd...@gmail.com>
Authored: Mon May 8 18:25:17 2017 -0400
Committer: Aldrin Piri <al...@apache.org>
Committed: Fri May 19 11:36:25 2017 -0400

----------------------------------------------------------------------
 .travis.yml                                     |   2 +-
 CMakeLists.txt                                  |  12 +-
 cmake/BuildTests.cmake                          |  21 +-
 docker/Dockerfile                               |   1 +
 libminifi/cmake/.DS_Store                       | Bin 6148 -> 0 bytes
 libminifi/include/core/yaml/YamlConfiguration.h |  15 +-
 libminifi/src/core/yaml/YamlConfiguration.cpp   |  15 +-
 libminifi/test/resource/TestHTTPGet.yml         |  73 -------
 libminifi/test/resource/TestHTTPPost.yml        |  87 ---------
 libminifi/test/unit/YamlConfigurationTests.cpp  | 193 +++++++++++++++++++
 libminifi/test/unit/YamlCongifurationTests.cpp  | 185 ------------------
 11 files changed, 226 insertions(+), 378 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/bc0d65e1/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index b93301c..61cbbee 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -63,4 +63,4 @@ matrix:
         - package='graphviz'; [[ $(brew ls --versions ${package}) ]] && { brew outdated ${package} || brew upgrade ${package}; } || brew install ${package}
 
 script:
-  - mkdir ./build && cd ./build && cmake .. && make VERBOSE=1 && make test ARGS="-V" && make linter && make docs
+  - mkdir ./build && cd ./build && cmake .. && make VERBOSE=1 && make test ARGS="--output-on-failure" && make linter && make docs

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/bc0d65e1/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 73222dd..d392512 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -30,7 +30,7 @@ option(test "Build all tests." ON)
 # Enable usage of the VERSION specifier
 # https://cmake.org/cmake/help/v3.0/policy/CMP0048.html#policy:CMP0048
 IF(POLICY CMP0048)
-  CMAKE_POLICY(SET CMP0048 OLD)
+    CMAKE_POLICY(SET CMP0048 OLD)
 ENDIF(POLICY CMP0048)
 
 include(CheckCXXCompilerFlag)
@@ -41,7 +41,7 @@ if(COMPILER_SUPPORTS_CXX11)
 elseif(COMPILER_SUPPORTS_CXX0X)
     set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++0x")
 else()
-        message(STATUS "The compiler ${CMAKE_CXX_COMPILER} has no C++11 support. Please use a different C++ compiler.")
+    message(STATUS "The compiler ${CMAKE_CXX_COMPILER} has no C++11 support. Please use a different C++ compiler.")
 endif()
 
 set(CMAKE_CXX_STANDARD 11)
@@ -55,15 +55,15 @@ find_package(Threads REQUIRED)
 
 # Set the right openssl root path
 if (${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
-set(OPENSSL_ROOT_DIR "/usr/local/opt/openssl/")
+    set(OPENSSL_ROOT_DIR "/usr/local/opt/openssl/")
 else()
-set(OPENSSL_ROOT_DIR "/usr/lib/x86_64-linux-gnu")
+    set(OPENSSL_ROOT_DIR "/usr/lib/x86_64-linux-gnu")
 endif()
 
 # Include OpenSSL
 find_package (OpenSSL REQUIRED)
 if (OPENSSL_FOUND)
-	include_directories(${OPENSSL_INCLUDE_DIR})
+    include_directories(${OPENSSL_INCLUDE_DIR})
 else ()
     message( FATAL_ERROR "OpenSSL was not found. Please install OpenSSL" )
 endif (OPENSSL_FOUND)
@@ -75,7 +75,7 @@ list(APPEND CMAKE_MODULE_PATH "${CMAKE_CURRENT_SOURCE_DIR}/cmake")
 find_package (Leveldb REQUIRED)
 find_package(CURL REQUIRED)
 if (LEVELDB_FOUND)
-        include_directories(${LEVELDB_INCLUDE_DIRS})
+    include_directories(${LEVELDB_INCLUDE_DIRS})
 else ()
     message( FATAL_ERROR "LevelDB was not found. Please install LevelDB" )
 endif (LEVELDB_FOUND)

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/bc0d65e1/cmake/BuildTests.cmake
----------------------------------------------------------------------
diff --git a/cmake/BuildTests.cmake b/cmake/BuildTests.cmake
index 84cbf58..0c96842 100644
--- a/cmake/BuildTests.cmake
+++ b/cmake/BuildTests.cmake
@@ -56,35 +56,38 @@ endfunction()
 
 enable_testing(test)
 
-GETSOURCEFILES(UNIT_TESTS "${CMAKE_SOURCE_DIR}/libminifi/test/unit/")
-GETSOURCEFILES(INTEGRATION_TESTS "${CMAKE_SOURCE_DIR}/libminifi/test/integration/")
+SET(TEST_DIR ${CMAKE_SOURCE_DIR}/libminifi/test)
+SET(TEST_RESOURCES ${TEST_DIR}/resources)
+
+GETSOURCEFILES(UNIT_TESTS "${TEST_DIR}/unit/")
+GETSOURCEFILES(INTEGRATION_TESTS "${TEST_DIR}/integration/")
 
 SET(UNIT_TEST_COUNT 0)
 FOREACH(testfile ${UNIT_TESTS})
 	get_filename_component(testfilename "${testfile}" NAME_WE)
-	add_executable("${testfilename}" "${CMAKE_SOURCE_DIR}/libminifi/test/unit/${testfile}" ${SPD_SOURCES})
+	add_executable("${testfilename}" "${TEST_DIR}/unit/${testfile}" ${SPD_SOURCES})
 	createTests("${testfilename}")
  	MATH(EXPR UNIT_TEST_COUNT "${UNIT_TEST_COUNT}+1")
-	add_test(NAME "${testfilename}" COMMAND "${testfilename}")
+	add_test(NAME "${testfilename}" COMMAND "${testfilename}" WORKING_DIRECTORY ${TEST_DIR})
 ENDFOREACH()
 message("-- Finished building ${UNIT_TEST_COUNT} unit test file(s)...")
 
 SET(INT_TEST_COUNT 0)
 FOREACH(testfile ${INTEGRATION_TESTS})
 	get_filename_component(testfilename "${testfile}" NAME_WE)
-	add_executable("${testfilename}" "${CMAKE_SOURCE_DIR}/libminifi/test/integration/${testfile}" ${SPD_SOURCES})
+	add_executable("${testfilename}" "${TEST_DIR}/integration/${testfile}" ${SPD_SOURCES})
 	createTests("${testfilename}")
 	#message("Adding ${testfilename} from ${testfile}")
 	MATH(EXPR INT_TEST_COUNT "${INT_TEST_COUNT}+1")
 ENDFOREACH()
 message("-- Finished building ${INT_TEST_COUNT} integration test file(s)...")
 
-add_test(NAME ControllerServiceIntegrationTests COMMAND ControllerServiceIntegrationTests "${CMAKE_SOURCE_DIR}/libminifi/test/resources/TestControllerServices.yml" "${CMAKE_SOURCE_DIR}/libminifi/test/resources/") 
+add_test(NAME ControllerServiceIntegrationTests COMMAND ControllerServiceIntegrationTests "${TEST_RESOURCES}/TestControllerServices.yml" "${TEST_RESOURCES}/")
 
-add_test(NAME HttpGetIntegrationTest COMMAND HttpGetIntegrationTest "${CMAKE_SOURCE_DIR}/libminifi/test/resources/TestHTTPGet.yml"  "${CMAKE_SOURCE_DIR}/libminifi/test/resources/")
+add_test(NAME HttpGetIntegrationTest COMMAND HttpGetIntegrationTest "${TEST_RESOURCES}/TestHTTPGet.yml"  "${TEST_RESOURCES}/")
 
-add_test(NAME HttpGetIntegrationTestSecure COMMAND HttpGetIntegrationTest "${CMAKE_SOURCE_DIR}/libminifi/test/resources/TestHTTPGetSecure.yml"  "${CMAKE_SOURCE_DIR}/libminifi/test/resources/")
+add_test(NAME HttpGetIntegrationTestSecure COMMAND HttpGetIntegrationTest "${TEST_RESOURCES}/TestHTTPGetSecure.yml"  "${TEST_RESOURCES}/")
 
-add_test(NAME HttpPostIntegrationTest COMMAND HttpPostIntegrationTest "${CMAKE_SOURCE_DIR}/libminifi/test/resources/TestHTTPPost.yml" )
+add_test(NAME HttpPostIntegrationTest COMMAND HttpPostIntegrationTest "${TEST_RESOURCES}/TestHTTPPost.yml" )
 
 add_test(NAME TestExecuteProcess COMMAND TestExecuteProcess )

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/bc0d65e1/docker/Dockerfile
----------------------------------------------------------------------
diff --git a/docker/Dockerfile b/docker/Dockerfile
index 68bc773..7721ebc 100644
--- a/docker/Dockerfile
+++ b/docker/Dockerfile
@@ -32,6 +32,7 @@ RUN apt-get update && apt-get install -y build-essential \
 	vim \
 	uuid-dev \
 	libleveldb-dev \
+	libcurl4-openssl-dev \
 	cmake \
 	git \
 	unzip \

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/bc0d65e1/libminifi/cmake/.DS_Store
----------------------------------------------------------------------
diff --git a/libminifi/cmake/.DS_Store b/libminifi/cmake/.DS_Store
deleted file mode 100644
index 5008ddf..0000000
Binary files a/libminifi/cmake/.DS_Store and /dev/null differ

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/bc0d65e1/libminifi/include/core/yaml/YamlConfiguration.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/yaml/YamlConfiguration.h b/libminifi/include/core/yaml/YamlConfiguration.h
index 6226a4e..8d6239c 100644
--- a/libminifi/include/core/yaml/YamlConfiguration.h
+++ b/libminifi/include/core/yaml/YamlConfiguration.h
@@ -63,17 +63,16 @@ class YamlConfiguration : public FlowConfiguration {
 
   /**
    * Returns a shared pointer to a ProcessGroup object containing the
-   * flow configuration. The yamlConfigStr argument must hold a string
-   * for the raw YAML configuration.
+   * flow configuration. The yamlConfigFile argument is the location
+   * of a YAML file containing the flow configuration.
    *
-   * @param yamlConfigStr a string holding the raw YAML to be parsed and
-   *                        loaded into a flow configuration tree
-   * @return              the root ProcessGroup node of the flow
+   * @param yamlConfigFile a string holding the location of the YAML file
+   *                        to be loaded into a flow configuration tree
+   * @return               the root ProcessGroup node of the flow
    *                        configuration tree
    */
-  std::unique_ptr<core::ProcessGroup> getRoot(
-      const std::string &yamlConfigStr) {
-    YAML::Node rootYamlNode = YAML::LoadFile(yamlConfigStr);
+  std::unique_ptr<core::ProcessGroup> getRoot(const std::string &yamlConfigFile) {
+    YAML::Node rootYamlNode = YAML::LoadFile(yamlConfigFile);
     return getRoot(&rootYamlNode);
   }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/bc0d65e1/libminifi/src/core/yaml/YamlConfiguration.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/yaml/YamlConfiguration.cpp b/libminifi/src/core/yaml/YamlConfiguration.cpp
index c2c4950..7bf4f58 100644
--- a/libminifi/src/core/yaml/YamlConfiguration.cpp
+++ b/libminifi/src/core/yaml/YamlConfiguration.cpp
@@ -33,8 +33,7 @@ core::ProcessGroup *YamlConfiguration::parseRootProcessGroupYaml(
     YAML::Node rootFlowNode) {
   uuid_t uuid;
 
-  checkRequiredField(&rootFlowNode, "name",
-  CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
+  checkRequiredField(&rootFlowNode, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
   std::string flowName = rootFlowNode["name"].as<std::string>();
   std::string id = getOrGenerateId(&rootFlowNode);
   uuid_parse(id.c_str(), uuid);
@@ -337,11 +336,9 @@ void YamlConfiguration::parseProvenanceReportingYaml(
 
   YAML::Node node = reportNode->as<YAML::Node>();
 
-  checkRequiredField(&node, "scheduling strategy",
-  CONFIG_YAML_PROVENANCE_REPORT_KEY);
+  checkRequiredField(&node, "scheduling strategy", CONFIG_YAML_PROVENANCE_REPORT_KEY);
   auto schedulingStrategyStr = node["scheduling strategy"].as<std::string>();
-  checkRequiredField(&node, "scheduling period",
-  CONFIG_YAML_PROVENANCE_REPORT_KEY);
+  checkRequiredField(&node, "scheduling period", CONFIG_YAML_PROVENANCE_REPORT_KEY);
   auto schedulingPeriodStr = node["scheduling period"].as<std::string>();
   checkRequiredField(&node, "host", CONFIG_YAML_PROVENANCE_REPORT_KEY);
   auto hostStr = node["host"].as<std::string>();
@@ -527,7 +524,7 @@ void YamlConfiguration::parseConnectionYaml(YAML::Node *connectionsNode,
           std::string connectionDestProcName = connectionNode["destination name"].as<std::string>();
           uuid_t tmpUUID;
           if (!uuid_parse(connectionDestProcName.c_str(), tmpUUID) &&
-          NULL != parent->findProcessor(tmpUUID)) {
+              NULL != parent->findProcessor(tmpUUID)) {
             // the destination name is a remote port id, so use that as the dest id
             uuid_copy(destUUID, tmpUUID);
             logger_->log_debug("Using 'destination name' containing a remote port id to match the destination for "
@@ -573,12 +570,12 @@ void YamlConfiguration::parsePortYaml(YAML::Node *portNode,
   YAML::Node inputPortsObj = portNode->as<YAML::Node>();
 
   // Check for required fields
-  checkRequiredField(&inputPortsObj, "name",
-  CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
+  checkRequiredField(&inputPortsObj, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
   auto nameStr = inputPortsObj["name"].as<std::string>();
   checkRequiredField(
       &inputPortsObj,
       "id",
+      CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY,
       "The field 'id' is required for "
           "the port named '" + nameStr
           + "' in the YAML Config. If this port "

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/bc0d65e1/libminifi/test/resource/TestHTTPGet.yml
----------------------------------------------------------------------
diff --git a/libminifi/test/resource/TestHTTPGet.yml b/libminifi/test/resource/TestHTTPGet.yml
deleted file mode 100644
index 0783b8e..0000000
--- a/libminifi/test/resource/TestHTTPGet.yml
+++ /dev/null
@@ -1,73 +0,0 @@
-#
-# 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.
-#
-Flow Controller:
-    name: MiNiFi Flow
-    id: 2438e3c8-015a-1000-79ca-83af40ec1990
-Processors:
-    - name: invoke
-      id: 2438e3c8-015a-1000-79ca-83af40ec1991
-      class: org.apache.nifi.processors.standard.InvokeHTTP
-      max concurrent tasks: 1
-      scheduling strategy: TIMER_DRIVEN
-      scheduling period: 1 sec
-      penalization period: 30 sec
-      yield period: 1 sec
-      run duration nanos: 0
-      auto-terminated relationships list:
-      Properties:
-          HTTP Method: GET
-          Remote URL: https://curl.haxx.se/libcurl/c/httpput.html
-    - name: OhJeez
-      id: 2438e3c8-015a-1000-79ca-83af40ec1992
-      class: org.apache.nifi.processors.standard.LogAttribute
-      max concurrent tasks: 1
-      scheduling strategy: TIMER_DRIVEN
-      scheduling period: 1 sec
-      penalization period: 30 sec
-      yield period: 1 sec
-      run duration nanos: 0
-      auto-terminated relationships list: response
-      Properties:
-        Log Level: info
-        Log Payload: true
-
-Connections:
-    - name: TransferFilesToRPG
-      id: 2438e3c8-015a-1000-79ca-83af40ec1997
-      source name: invoke
-      source id: 2438e3c8-015a-1000-79ca-83af40ec1991
-      source relationship name: success
-      destination name: OhJeez
-      destination id: 2438e3c8-015a-1000-79ca-83af40ec1992
-      max work queue size: 0
-      max work queue data size: 1 MB
-      flowfile expiration: 60 sec
-    - name: TransferFilesToRPG2
-      id: 2438e3c8-015a-1000-79ca-83af40ec1917
-      source name: OhJeez
-      source id: 2438e3c8-015a-1000-79ca-83af40ec1992
-      destination name: OhJeez
-      destination id: 2438e3c8-015a-1000-79ca-83af40ec1992  
-      source relationship name: success
-      max work queue size: 0
-      max work queue data size: 1 MB
-      flowfile expiration: 60 sec
-
-Remote Processing Groups:
-    
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/bc0d65e1/libminifi/test/resource/TestHTTPPost.yml
----------------------------------------------------------------------
diff --git a/libminifi/test/resource/TestHTTPPost.yml b/libminifi/test/resource/TestHTTPPost.yml
deleted file mode 100644
index 837194d..0000000
--- a/libminifi/test/resource/TestHTTPPost.yml
+++ /dev/null
@@ -1,87 +0,0 @@
-#
-# 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.
-#
-Flow Controller:
-    name: MiNiFi Flow
-    id: 2438e3c8-015a-1000-79ca-83af40ec1990
-Processors:
-    - name: invoke
-      id: 2438e3c8-015a-1000-79ca-83af40ec1991
-      class: org.apache.nifi.processors.standard.GetFile
-      max concurrent tasks: 1
-      scheduling strategy: TIMER_DRIVEN
-      scheduling period: 1 sec
-      penalization period: 30 sec
-      yield period: 1 sec
-      run duration nanos: 0
-      auto-terminated relationships list:
-      Properties:
-          Input Directory: /tmp/aljr39
-          Keep Source File: false
-
-    - name: OhJeez
-      id: 2438e3c8-015a-1000-79ca-83af40ec1992
-      class: org.apache.nifi.processors.standard.InvokeHTTP
-      max concurrent tasks: 1
-      scheduling strategy: TIMER_DRIVEN
-      scheduling period: 1 sec
-      penalization period: 30 sec
-      yield period: 1 sec
-      run duration nanos: 0
-      auto-terminated relationships list: response
-      Properties:
-          HTTP Method: POST
-          Remote URL: http://requestb.in/u8ax9uu8
-          
-    - name: Loggit
-      id: 2438e3c8-015a-1000-79ca-83af40ec1993
-      class: org.apache.nifi.processors.standard.LogAttribute
-      max concurrent tasks: 1
-      scheduling strategy: TIMER_DRIVEN
-      scheduling period: 1 sec
-      penalization period: 30 sec
-      yield period: 1 sec
-      run duration nanos: 0
-      auto-terminated relationships list: response
-      Properties:
-          LogLevel: info
-
-Connections:
-    - name: TransferFilesToRPG
-      id: 2438e3c8-015a-1000-79ca-83af40ec1997
-      source name: invoke
-      source id: 2438e3c8-015a-1000-79ca-83af40ec1991
-      source relationship name: success
-      destination name: OhJeez
-      destination id: 2438e3c8-015a-1000-79ca-83af40ec1992
-      max work queue size: 0
-      max work queue data size: 1 MB
-      flowfile expiration: 60 sec
-    - name: TransferFilesToRPG2
-      id: 2438e3c8-015a-1000-79ca-83af40ec1917
-      source name: OhJeez
-      source id: 2438e3c8-015a-1000-79ca-83af40ec1992
-      destination name: OhJeez
-      destination id: 2438e3c8-015a-1000-79ca-83af40ec1993
-      source relationship name: success
-      max work queue size: 0
-      max work queue data size: 1 MB
-      flowfile expiration: 60 sec
-
-Remote Processing Groups:
-    
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/bc0d65e1/libminifi/test/unit/YamlConfigurationTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/YamlConfigurationTests.cpp b/libminifi/test/unit/YamlConfigurationTests.cpp
new file mode 100644
index 0000000..f958ca1
--- /dev/null
+++ b/libminifi/test/unit/YamlConfigurationTests.cpp
@@ -0,0 +1,193 @@
+/**
+ *
+ * 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.
+ */
+
+#define CATCH_CONFIG_MAIN  // This tells Catch to provide a main() - only do this in one cpp file
+
+#include <memory>
+#include <string>
+#include <core/RepositoryFactory.h>
+#include "core/yaml/YamlConfiguration.h"
+#include "../TestBase.h"
+
+TEST_CASE("Test YAML Config Processing", "[YamlConfiguration]") {
+
+  std::shared_ptr<core::Repository> testProvRepo = core::createRepository("provenancerepository", true);
+  std::shared_ptr<core::Repository> testFlowFileRepo = core::createRepository("flowfilerepository", true);
+  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
+  std::shared_ptr<minifi::io::StreamFactory> streamFactory = std::make_shared<minifi::io::StreamFactory>(configuration);
+  core::YamlConfiguration *yamlConfig = new core::YamlConfiguration(testProvRepo, testFlowFileRepo, streamFactory, configuration);
+
+  SECTION("loading YAML without optional component IDs works") {
+
+    static const std::string CONFIG_YAML_WITHOUT_IDS = ""
+        "MiNiFi Config Version: 1\n"
+        "Flow Controller:\n"
+        "    name: MiNiFi Flow\n"
+        "    comment:\n"
+        "\n"
+        "Core Properties:\n"
+        "    flow controller graceful shutdown period: 10 sec\n"
+        "    flow service write delay interval: 500 ms\n"
+        "    administrative yield duration: 30 sec\n"
+        "    bored yield duration: 10 millis\n"
+        "\n"
+        "FlowFile Repository:\n"
+        "    partitions: 256\n"
+        "    checkpoint interval: 2 mins\n"
+        "    always sync: false\n"
+        "    Swap:\n"
+        "        threshold: 20000\n"
+        "        in period: 5 sec\n"
+        "        in threads: 1\n"
+        "        out period: 5 sec\n"
+        "        out threads: 4\n"
+        "\n"
+        "Provenance Repository:\n"
+        "    provenance rollover time: 1 min\n"
+        "\n"
+        "Content Repository:\n"
+        "    content claim max appendable size: 10 MB\n"
+        "    content claim max flow files: 100\n"
+        "    always sync: false\n"
+        "\n"
+        "Component Status Repository:\n"
+        "    buffer size: 1440\n"
+        "    snapshot frequency: 1 min\n"
+        "\n"
+        "Security Properties:\n"
+        "    keystore: /tmp/ssl/localhost-ks.jks\n"
+        "    keystore type: JKS\n"
+        "    keystore password: localtest\n"
+        "    key password: localtest\n"
+        "    truststore: /tmp/ssl/localhost-ts.jks\n"
+        "    truststore type: JKS\n"
+        "    truststore password: localtest\n"
+        "    ssl protocol: TLS\n"
+        "    Sensitive Props:\n"
+        "        key:\n"
+        "        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL\n"
+        "        provider: BC\n"
+        "\n"
+        "Processors:\n"
+        "    - name: TailFile\n"
+        "      class: org.apache.nifi.processors.standard.TailFile\n"
+        "      max concurrent tasks: 1\n"
+        "      scheduling strategy: TIMER_DRIVEN\n"
+        "      scheduling period: 1 sec\n"
+        "      penalization period: 30 sec\n"
+        "      yield period: 1 sec\n"
+        "      run duration nanos: 0\n"
+        "      auto-terminated relationships list:\n"
+        "      Properties:\n"
+        "          File to Tail: logs/minifi-app.log\n"
+        "          Rolling Filename Pattern: minifi-app*\n"
+        "          Initial Start Position: Beginning of File\n"
+        "\n"
+        "Connections:\n"
+        "    - name: TailToS2S\n"
+        "      source name: TailFile\n"
+        "      source relationship name: success\n"
+        "      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
+        "      max work queue size: 0\n"
+        "      max work queue data size: 1 MB\n"
+        "      flowfile expiration: 60 sec\n"
+        "      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer\n"
+        "\n"
+        "Remote Processing Groups:\n"
+        "    - name: NiFi Flow\n"
+        "      comment:\n"
+        "      url: https://localhost:8090/nifi\n"
+        "      timeout: 30 secs\n"
+        "      yield period: 10 sec\n"
+        "      Input Ports:\n"
+        "          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
+        "            name: tailed log\n"
+        "            comments:\n"
+        "            max concurrent tasks: 1\n"
+        "            use compression: false\n"
+        "\n"
+        "Provenance Reporting:\n"
+        "    comment:\n"
+        "    scheduling strategy: TIMER_DRIVEN\n"
+        "    scheduling period: 30 sec\n"
+        "    host: localhost\n"
+        "    port name: provenance\n"
+        "    port: 8090\n"
+        "    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56\n"
+        "    destination url: https://localhost:8090/\n"
+        "    originating url: http://${hostname(true)}:8081/nifi\n"
+        "    use compression: true\n"
+        "    timeout: 30 secs\n"
+        "    batch size: 1000";
+
+    std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
+    std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig->getRoot(configYamlStream);
+
+    REQUIRE(rootFlowConfig);
+    REQUIRE(rootFlowConfig->findProcessor("TailFile"));
+    REQUIRE(NULL != rootFlowConfig->findProcessor("TailFile")->getUUID());
+    REQUIRE(!rootFlowConfig->findProcessor("TailFile")->getUUIDStr().empty());
+    REQUIRE(1 == rootFlowConfig->findProcessor("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessor("TailFile")->getSchedulingStrategy());
+    REQUIRE(1 == rootFlowConfig->findProcessor("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(1*1000*1000*1000 == rootFlowConfig->findProcessor("TailFile")->getSchedulingPeriodNano());
+    REQUIRE(30*1000 == rootFlowConfig->findProcessor("TailFile")->getPenalizationPeriodMsec());
+    REQUIRE(1*1000 == rootFlowConfig->findProcessor("TailFile")->getYieldPeriodMsec());
+    REQUIRE(0 == rootFlowConfig->findProcessor("TailFile")->getRunDurationNano());
+
+    std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
+    rootFlowConfig->getConnections(connectionMap);
+    REQUIRE(1 == connectionMap.size());
+    // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
+    for(
+        std::map<std::string,std::shared_ptr<minifi::Connection>>::iterator it = connectionMap.begin();
+        it != connectionMap.end();
+        ++it) {
+      REQUIRE(it->second);
+      REQUIRE(!it->second->getUUIDStr().empty());
+      REQUIRE(it->second->getDestination());
+      REQUIRE(it->second->getSource());
+    }
+  }
+
+  SECTION("missing required field in YAML throws exception") {
+
+    static const std::string CONFIG_YAML_NO_RPG_PORT_ID = ""
+        "MiNiFi Config Version: 1\n"
+        "Flow Controller:\n"
+        "  name: MiNiFi Flow\n"
+        "Processors: []\n"
+        "Connections: []\n"
+        "Remote Processing Groups:\n"
+        "    - name: NiFi Flow\n"
+        "      comment:\n"
+        "      url: https://localhost:8090/nifi\n"
+        "      timeout: 30 secs\n"
+        "      yield period: 10 sec\n"
+        "      Input Ports:\n"
+        "          - name: tailed log\n"
+        "            comments:\n"
+        "            max concurrent tasks: 1\n"
+        "            use compression: false\n"
+        "\n";
+
+    std::istringstream configYamlStream(CONFIG_YAML_NO_RPG_PORT_ID);
+    REQUIRE_THROWS_AS(yamlConfig->getRoot(configYamlStream), std::invalid_argument);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/bc0d65e1/libminifi/test/unit/YamlCongifurationTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/YamlCongifurationTests.cpp b/libminifi/test/unit/YamlCongifurationTests.cpp
deleted file mode 100644
index f598f38..0000000
--- a/libminifi/test/unit/YamlCongifurationTests.cpp
+++ /dev/null
@@ -1,185 +0,0 @@
-/**
- *
- * 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.
- */
-#define CATCH_CONFIG_MAIN  // This tells Catch to provide a main() - only do this in one cpp file
-#include <memory>
-#include <string>
-#include <core/RepositoryFactory.h>
-#include "core/yaml/YamlConfiguration.h"
-#include "../TestBase.h"
-
-static const std::shared_ptr<core::Repository> TEST_PROV_REPO = core::createRepository("provenancerepository", true);
-static const std::shared_ptr<core::Repository> TEST_FF_REPO = core::createRepository("flowfilerepository", true);
-
-TEST_CASE("Test YAML Config 1", "[testyamlconfig1]") {
-
-  static const std::string TEST_YAML_WITHOUT_IDS = "MiNiFi Config Version: 1\n"
-      "Flow Controller:\n"
-      "    name: MiNiFi Flow\n"
-      "    comment:\n"
-      "\n"
-      "Core Properties:\n"
-      "    flow controller graceful shutdown period: 10 sec\n"
-      "    flow service write delay interval: 500 ms\n"
-      "    administrative yield duration: 30 sec\n"
-      "    bored yield duration: 10 millis\n"
-      "\n"
-      "FlowFile Repository:\n"
-      "    partitions: 256\n"
-      "    checkpoint interval: 2 mins\n"
-      "    always sync: false\n"
-      "    Swap:\n"
-      "        threshold: 20000\n"
-      "        in period: 5 sec\n"
-      "        in threads: 1\n"
-      "        out period: 5 sec\n"
-      "        out threads: 4\n"
-      "\n"
-      "Provenance Repository:\n"
-      "    provenance rollover time: 1 min\n"
-      "\n"
-      "Content Repository:\n"
-      "    content claim max appendable size: 10 MB\n"
-      "    content claim max flow files: 100\n"
-      "    always sync: false\n"
-      "\n"
-      "Component Status Repository:\n"
-      "    buffer size: 1440\n"
-      "    snapshot frequency: 1 min\n"
-      "\n"
-      "Security Properties:\n"
-      "    keystore: /tmp/ssl/localhost-ks.jks\n"
-      "    keystore type: JKS\n"
-      "    keystore password: localtest\n"
-      "    key password: localtest\n"
-      "    truststore: /tmp/ssl/localhost-ts.jks\n"
-      "    truststore type: JKS\n"
-      "    truststore password: localtest\n"
-      "    ssl protocol: TLS\n"
-      "    Sensitive Props:\n"
-      "        key:\n"
-      "        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL\n"
-      "        provider: BC\n"
-      "\n"
-      "Processors:\n"
-      "    - name: TailFile\n"
-      "      class: org.apache.nifi.processors.standard.TailFile\n"
-      "      max concurrent tasks: 1\n"
-      "      scheduling strategy: TIMER_DRIVEN\n"
-      "      scheduling period: 1 sec\n"
-      "      penalization period: 30 sec\n"
-      "      yield period: 1 sec\n"
-      "      run duration nanos: 0\n"
-      "      auto-terminated relationships list:\n"
-      "      Properties:\n"
-      "          File to Tail: logs/minifi-app.log\n"
-      "          Rolling Filename Pattern: minifi-app*\n"
-      "          Initial Start Position: Beginning of File\n"
-      "\n"
-      "Connections:\n"
-      "    - name: TailToS2S\n"
-      "      source name: TailFile\n"
-      "      source relationship name: success\n"
-      "      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "      max work queue size: 0\n"
-      "      max work queue data size: 1 MB\n"
-      "      flowfile expiration: 60 sec\n"
-      "      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer\n"
-      "\n"
-      "Remote Processing Groups:\n"
-      "    - name: NiFi Flow\n"
-      "      comment:\n"
-      "      url: https://localhost:8090/nifi\n"
-      "      timeout: 30 secs\n"
-      "      yield period: 10 sec\n"
-      "      Input Ports:\n"
-      "          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "            name: tailed log\n"
-      "            comments:\n"
-      "            max concurrent tasks: 1\n"
-      "            use compression: false\n"
-      "\n"
-      "Provenance Reporting:\n"
-      "    comment:\n"
-      "    scheduling strategy: TIMER_DRIVEN\n"
-      "    scheduling period: 30 sec\n"
-      "    host: localhost\n"
-      "    port name: provenance\n"
-      "    port: 8090\n"
-      "    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56\n"
-      "    destination url: https://localhost:8090/\n"
-      "    originating url: http://${hostname(true)}:8081/nifi\n"
-      "    use compression: true\n"
-      "    timeout: 30 secs\n"
-      "    batch size: 1000";
-
-  core::YamlConfiguration *yamlConfig = new core::YamlConfiguration(TEST_PROV_REPO, TEST_FF_REPO, std::make_shared<minifi::io::StreamFactory>(std::make_shared<minifi::Configure>()),std::make_shared<minifi::Configure>());
-  std::istringstream yamlstream(TEST_YAML_WITHOUT_IDS);
-  std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig->getRoot(yamlstream);
-
-  REQUIRE(rootFlowConfig);
-
-  REQUIRE(rootFlowConfig->findProcessor("TailFile"));
-  REQUIRE(NULL != rootFlowConfig->findProcessor("TailFile")->getUUID());
-  REQUIRE(!rootFlowConfig->findProcessor("TailFile")->getUUIDStr().empty());
-  REQUIRE(1 == rootFlowConfig->findProcessor("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessor("TailFile")->getSchedulingStrategy());
-  REQUIRE(1 == rootFlowConfig->findProcessor("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(1*1000*1000*1000 == rootFlowConfig->findProcessor("TailFile")->getSchedulingPeriodNano());
-  REQUIRE(30*1000 == rootFlowConfig->findProcessor("TailFile")->getPenalizationPeriodMsec());
-  REQUIRE(1*1000 == rootFlowConfig->findProcessor("TailFile")->getYieldPeriodMsec());
-  REQUIRE(0 == rootFlowConfig->findProcessor("TailFile")->getRunDurationNano());
-
-  std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
-  rootFlowConfig->getConnections(connectionMap);
-  REQUIRE(1 == connectionMap.size());
-  // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
-  for(
-      std::map<std::string,std::shared_ptr<minifi::Connection>>::iterator it = connectionMap.begin();
-      it != connectionMap.end();
-      ++it) {
-    REQUIRE(it->second);
-    REQUIRE(!it->second->getUUIDStr().empty());
-    REQUIRE(it->second->getDestination());
-    REQUIRE(it->second->getSource());
-  }
-}
-
-TEST_CASE("Test YAML Config Missing Required Fields", "[testyamlconfig2]") {
-
-  static const std::string TEST_YAML_NO_RPG_PORT_ID = "Flow Controller:\n"
-      "  name: MiNiFi Flow\n"
-      "Processors: []\n"
-      "Connections: []\n"
-      "Remote Processing Groups:\n"
-      "    - name: NiFi Flow\n"
-      "      comment:\n"
-      "      url: https://localhost:8090/nifi\n"
-      "      timeout: 30 secs\n"
-      "      yield period: 10 sec\n"
-      "      Input Ports:\n"
-      "          - name: tailed log\n"
-      "            comments:\n"
-      "            max concurrent tasks: 1\n"
-      "            use compression: false\n"
-      "\n";
-
-  core::YamlConfiguration *yamlConfig = new core::YamlConfiguration(TEST_PROV_REPO, TEST_FF_REPO, std::make_shared<minifi::io::StreamFactory>(std::make_shared<minifi::Configure>()),std::make_shared<minifi::Configure>());
-  std::istringstream yamlstream(TEST_YAML_NO_RPG_PORT_ID);
-
-  REQUIRE_THROWS_AS(yamlConfig->getRoot(yamlstream), std::invalid_argument);
-}