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/18 13:07:51 UTC

[1/5] nifi-minifi-cpp git commit: MINIFI-226: Add controller services capabilities along with unit tests Fix test failures Update Travis YML Update readme to link to MiNiFi licensing information

Repository: nifi-minifi-cpp
Updated Branches:
  refs/heads/master 372f2d6f8 -> c9940e945


http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/unit/InvokeHTTPTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/InvokeHTTPTests.cpp b/libminifi/test/unit/InvokeHTTPTests.cpp
index 6b96549..0200710 100644
--- a/libminifi/test/unit/InvokeHTTPTests.cpp
+++ b/libminifi/test/unit/InvokeHTTPTests.cpp
@@ -18,13 +18,13 @@
 #include <uuid/uuid.h>
 #include <fstream>
 #include "FlowController.h"
-#include "ProvenanceTestHelper.h"
 #include "../TestBase.h"
 #include "core/logging/LogAppenders.h"
 #include "core/logging/BaseLogger.h"
 #include "processors/GetFile.h"
 #include "core/Core.h"
 #include "../../include/core/FlowFile.h"
+#include "../unit/ProvenanceTestHelper.h"
 #include "core/Processor.h"
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
@@ -35,7 +35,8 @@ TEST_CASE("HTTPTestsPostNoResourceClaim", "[httptest1]") {
   std::stringstream oss;
   std::unique_ptr<logging::BaseLogger> outputLogger = std::unique_ptr<
       logging::BaseLogger>(
-      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,0));
+      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,
+                                                                         0));
   std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
   logger->updateLogger(std::move(outputLogger));
 
@@ -43,9 +44,7 @@ TEST_CASE("HTTPTestsPostNoResourceClaim", "[httptest1]") {
 
   testController.enableDebug();
 
-
-  std::shared_ptr<TestRepository> repo = std::make_shared<
-      TestRepository>();
+  std::shared_ptr<TestRepository> repo = std::make_shared<TestRepository>();
 
   std::shared_ptr<core::Processor> processor = std::make_shared<
       org::apache::nifi::minifi::processors::ListenHTTP>("listenhttp");
@@ -58,26 +57,23 @@ TEST_CASE("HTTPTestsPostNoResourceClaim", "[httptest1]") {
   uuid_t invokehttp_uuid;
   REQUIRE(true == invokehttp->getUUID(invokehttp_uuid));
 
-
   std::shared_ptr<minifi::Connection> connection = std::make_shared<
-      minifi::Connection>(repo,"getfileCreate2Connection");
+      minifi::Connection>(repo, "getfileCreate2Connection");
   connection->setRelationship(core::Relationship("success", "description"));
 
   std::shared_ptr<minifi::Connection> connection2 = std::make_shared<
-      minifi::Connection>(repo,"listenhttp");
+      minifi::Connection>(repo, "listenhttp");
 
   connection2->setRelationship(core::Relationship("No Retry", "description"));
 
   // link the connections so that we can test results at the end for this
   connection->setSource(processor);
 
-
   // link the connections so that we can test results at the end for this
   connection->setDestination(invokehttp);
 
   connection2->setSource(invokehttp);
 
-
   connection2->setSourceUUID(invokehttp_uuid);
   connection->setSourceUUID(processoruuid);
   connection->setDestinationUUID(invokehttp_uuid);
@@ -86,21 +82,23 @@ TEST_CASE("HTTPTestsPostNoResourceClaim", "[httptest1]") {
   invokehttp->addConnection(connection);
   invokehttp->addConnection(connection2);
 
-
   core::ProcessorNode node(processor);
   core::ProcessorNode node2(invokehttp);
 
-  core::ProcessContext context(node, repo);
-  core::ProcessContext context2(node2, repo);
+  std::shared_ptr<core::controller::ControllerServiceProvider> controller_services_provider =
+      nullptr;
+  core::ProcessContext context(node, controller_services_provider, repo);
+  core::ProcessContext context2(node2, controller_services_provider, repo);
   context.setProperty(org::apache::nifi::minifi::processors::ListenHTTP::Port,
                       "8685");
-  context.setProperty(org::apache::nifi::minifi::processors::ListenHTTP::BasePath,
-                        "/testytesttest");
+  context.setProperty(
+      org::apache::nifi::minifi::processors::ListenHTTP::BasePath,
+      "/testytesttest");
 
-  context2.setProperty(org::apache::nifi::minifi::processors::InvokeHTTP::Method,
-                        "POST");
+  context2.setProperty(
+      org::apache::nifi::minifi::processors::InvokeHTTP::Method, "POST");
   context2.setProperty(org::apache::nifi::minifi::processors::InvokeHTTP::URL,
-                          "http://localhost:8685/testytesttest");
+                       "http://localhost:8685/testytesttest");
   core::ProcessSession session(&context);
   core::ProcessSession session2(&context2);
 
@@ -125,7 +123,6 @@ TEST_CASE("HTTPTestsPostNoResourceClaim", "[httptest1]") {
   REQUIRE(record == nullptr);
   REQUIRE(records.size() == 0);
 
-
   processor->incrementActiveTasks();
   processor->setScheduledState(core::ScheduledState::RUNNING);
   processor->onTrigger(&context, &session);
@@ -142,25 +139,24 @@ TEST_CASE("HTTPTestsPostNoResourceClaim", "[httptest1]") {
   session2.commit();
   records = reporter->getEvents();
 
-
-
   for (provenance::ProvenanceEventRecord *provEventRecord : records) {
     REQUIRE(provEventRecord->getComponentType() == processor->getName());
   }
   std::shared_ptr<core::FlowFile> ffr = session2.get();
   std::string log_attribute_output = oss.str();
-std::cout << log_attribute_output << std::endl;
-  REQUIRE( log_attribute_output.find("exiting because method is POST") != std::string::npos );
+  REQUIRE(
+      log_attribute_output.find("exiting because method is POST")
+          != std::string::npos);
 
 }
 
-
 TEST_CASE("HTTPTestsWithNoResourceClaimPOST", "[httptest1]") {
 
   std::stringstream oss;
   std::unique_ptr<logging::BaseLogger> outputLogger = std::unique_ptr<
       logging::BaseLogger>(
-      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,0));
+      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,
+                                                                         0));
   std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
   logger->updateLogger(std::move(outputLogger));
 
@@ -168,19 +164,16 @@ TEST_CASE("HTTPTestsWithNoResourceClaimPOST", "[httptest1]") {
 
   testController.enableDebug();
 
-
-
-  std::shared_ptr<TestRepository> repo = std::make_shared<
-      TestRepository>();
+  std::shared_ptr<TestRepository> repo = std::make_shared<TestRepository>();
 
   std::shared_ptr<core::Processor> getfileprocessor = std::make_shared<
-        org::apache::nifi::minifi::processors::GetFile>("getfileCreate2");
+      org::apache::nifi::minifi::processors::GetFile>("getfileCreate2");
 
-    std::shared_ptr<core::Processor> logAttribute = std::make_shared<
-        org::apache::nifi::minifi::processors::LogAttribute>("logattribute");
+  std::shared_ptr<core::Processor> logAttribute = std::make_shared<
+      org::apache::nifi::minifi::processors::LogAttribute>("logattribute");
 
-    char format[] = "/tmp/gt.XXXXXX";
-    char *dir = testController.createTempDirectory(format);
+  char format[] = "/tmp/gt.XXXXXX";
+  char *dir = testController.createTempDirectory(format);
 
   std::shared_ptr<core::Processor> listenhttp = std::make_shared<
       org::apache::nifi::minifi::processors::ListenHTTP>("listenhttp");
@@ -193,30 +186,26 @@ TEST_CASE("HTTPTestsWithNoResourceClaimPOST", "[httptest1]") {
   uuid_t invokehttp_uuid;
   REQUIRE(true == invokehttp->getUUID(invokehttp_uuid));
 
-
   std::shared_ptr<minifi::Connection> gcConnection = std::make_shared<
-    minifi::Connection>(repo, "getfileCreate2Connection");
+      minifi::Connection>(repo, "getfileCreate2Connection");
   gcConnection->setRelationship(core::Relationship("success", "description"));
 
-std::shared_ptr<minifi::Connection> laConnection = std::make_shared<
-    minifi::Connection>(repo, "logattribute");
-laConnection->setRelationship(core::Relationship("success", "description"));
-
-
+  std::shared_ptr<minifi::Connection> laConnection = std::make_shared<
+      minifi::Connection>(repo, "logattribute");
+  laConnection->setRelationship(core::Relationship("success", "description"));
 
   std::shared_ptr<minifi::Connection> connection = std::make_shared<
-      minifi::Connection>(repo,"getfileCreate2Connection");
+      minifi::Connection>(repo, "getfileCreate2Connection");
   connection->setRelationship(core::Relationship("success", "description"));
 
   std::shared_ptr<minifi::Connection> connection2 = std::make_shared<
-      minifi::Connection>(repo,"listenhttp");
+      minifi::Connection>(repo, "listenhttp");
 
   connection2->setRelationship(core::Relationship("No Retry", "description"));
 
   // link the connections so that we can test results at the end for this
   connection->setSource(listenhttp);
 
-
   connection2->setSourceUUID(invokehttp_uuid);
   connection->setSourceUUID(processoruuid);
   connection->setDestinationUUID(invokehttp_uuid);
@@ -225,21 +214,22 @@ laConnection->setRelationship(core::Relationship("success", "description"));
   invokehttp->addConnection(connection);
   invokehttp->addConnection(connection2);
 
-
   core::ProcessorNode node(listenhttp);
   core::ProcessorNode node2(invokehttp);
-
-  core::ProcessContext context(node, repo);
-  core::ProcessContext context2(node2, repo);
+  std::shared_ptr<core::controller::ControllerServiceProvider> controller_services_provider =
+      nullptr;
+  core::ProcessContext context(node, controller_services_provider, repo);
+  core::ProcessContext context2(node2, controller_services_provider, repo);
   context.setProperty(org::apache::nifi::minifi::processors::ListenHTTP::Port,
                       "8686");
-  context.setProperty(org::apache::nifi::minifi::processors::ListenHTTP::BasePath,
-                        "/testytesttest");
+  context.setProperty(
+      org::apache::nifi::minifi::processors::ListenHTTP::BasePath,
+      "/testytesttest");
 
-  context2.setProperty(org::apache::nifi::minifi::processors::InvokeHTTP::Method,
-                        "POST");
+  context2.setProperty(
+      org::apache::nifi::minifi::processors::InvokeHTTP::Method, "POST");
   context2.setProperty(org::apache::nifi::minifi::processors::InvokeHTTP::URL,
-                          "http://localhost:8686/testytesttest");
+                       "http://localhost:8686/testytesttest");
   core::ProcessSession session(&context);
   core::ProcessSession session2(&context2);
 
@@ -264,7 +254,6 @@ laConnection->setRelationship(core::Relationship("success", "description"));
   REQUIRE(record == nullptr);
   REQUIRE(records.size() == 0);
 
-
   listenhttp->incrementActiveTasks();
   listenhttp->setScheduledState(core::ScheduledState::RUNNING);
   listenhttp->onTrigger(&context, &session);
@@ -281,32 +270,28 @@ laConnection->setRelationship(core::Relationship("success", "description"));
   session2.commit();
   records = reporter->getEvents();
 
-
-
   for (provenance::ProvenanceEventRecord *provEventRecord : records) {
     REQUIRE(provEventRecord->getComponentType() == listenhttp->getName());
   }
   std::shared_ptr<core::FlowFile> ffr = session2.get();
   std::string log_attribute_output = oss.str();
-std::cout << log_attribute_output << std::endl;
-  REQUIRE( log_attribute_output.find("exiting because method is POST") != std::string::npos );
+  REQUIRE(
+      log_attribute_output.find("exiting because method is POST")
+          != std::string::npos);
 
 }
 
-
-class CallBack : public minifi::OutputStreamCallback
-{
+class CallBack : public minifi::OutputStreamCallback {
  public:
-  CallBack()
-    {
+  CallBack() {
 
-    }
-  virtual ~CallBack(){
+  }
+  virtual ~CallBack() {
 
   }
-  virtual void process(std::ofstream *stream){
+  virtual void process(std::ofstream *stream) {
     std::string st = "we're gnna write some test stuff";
-    stream->write(st.c_str(),st.length());
+    stream->write(st.c_str(), st.length());
   }
 };
 
@@ -315,7 +300,8 @@ TEST_CASE("HTTPTestsWithResourceClaimPOST", "[httptest1]") {
   std::stringstream oss;
   std::unique_ptr<logging::BaseLogger> outputLogger = std::unique_ptr<
       logging::BaseLogger>(
-      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,0));
+      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,
+                                                                         0));
   std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
   logger->updateLogger(std::move(outputLogger));
 
@@ -323,19 +309,16 @@ TEST_CASE("HTTPTestsWithResourceClaimPOST", "[httptest1]") {
 
   testController.enableDebug();
 
-
-
-  std::shared_ptr<TestRepository> repo = std::make_shared<
-      TestRepository>();
+  std::shared_ptr<TestRepository> repo = std::make_shared<TestRepository>();
 
   std::shared_ptr<core::Processor> getfileprocessor = std::make_shared<
-        org::apache::nifi::minifi::processors::GetFile>("getfileCreate2");
+      org::apache::nifi::minifi::processors::GetFile>("getfileCreate2");
 
-    std::shared_ptr<core::Processor> logAttribute = std::make_shared<
-        org::apache::nifi::minifi::processors::LogAttribute>("logattribute");
+  std::shared_ptr<core::Processor> logAttribute = std::make_shared<
+      org::apache::nifi::minifi::processors::LogAttribute>("logattribute");
 
-    char format[] = "/tmp/gt.XXXXXX";
-    char *dir = testController.createTempDirectory(format);
+  char format[] = "/tmp/gt.XXXXXX";
+  char *dir = testController.createTempDirectory(format);
 
   std::shared_ptr<core::Processor> listenhttp = std::make_shared<
       org::apache::nifi::minifi::processors::ListenHTTP>("listenhttp");
@@ -348,23 +331,20 @@ TEST_CASE("HTTPTestsWithResourceClaimPOST", "[httptest1]") {
   uuid_t invokehttp_uuid;
   REQUIRE(true == invokehttp->getUUID(invokehttp_uuid));
 
-
   std::shared_ptr<minifi::Connection> gcConnection = std::make_shared<
-    minifi::Connection>(repo, "getfileCreate2Connection");
+      minifi::Connection>(repo, "getfileCreate2Connection");
   gcConnection->setRelationship(core::Relationship("success", "description"));
 
-std::shared_ptr<minifi::Connection> laConnection = std::make_shared<
-    minifi::Connection>(repo, "logattribute");
-laConnection->setRelationship(core::Relationship("success", "description"));
-
-
+  std::shared_ptr<minifi::Connection> laConnection = std::make_shared<
+      minifi::Connection>(repo, "logattribute");
+  laConnection->setRelationship(core::Relationship("success", "description"));
 
   std::shared_ptr<minifi::Connection> connection = std::make_shared<
-      minifi::Connection>(repo,"getfileCreate2Connection");
+      minifi::Connection>(repo, "getfileCreate2Connection");
   connection->setRelationship(core::Relationship("success", "description"));
 
   std::shared_ptr<minifi::Connection> connection2 = std::make_shared<
-      minifi::Connection>(repo,"listenhttp");
+      minifi::Connection>(repo, "listenhttp");
 
   connection2->setRelationship(core::Relationship("No Retry", "description"));
 
@@ -372,31 +352,31 @@ laConnection->setRelationship(core::Relationship("success", "description"));
   connection->setSource(listenhttp);
 
   connection->setSourceUUID(invokehttp_uuid);
-    connection->setDestinationUUID(processoruuid);
+  connection->setDestinationUUID(processoruuid);
 
   connection2->setSourceUUID(processoruuid);
   connection2->setSourceUUID(processoruuid);
 
-
   listenhttp->addConnection(connection);
   invokehttp->addConnection(connection);
   invokehttp->addConnection(connection2);
 
-
   core::ProcessorNode node(invokehttp);
   core::ProcessorNode node2(listenhttp);
-
-  core::ProcessContext context(node, repo);
-  core::ProcessContext context2(node2, repo);
+  std::shared_ptr<core::controller::ControllerServiceProvider> controller_services_provider =
+      nullptr;
+  core::ProcessContext context(node, controller_services_provider, repo);
+  core::ProcessContext context2(node2, controller_services_provider, repo);
   context.setProperty(org::apache::nifi::minifi::processors::ListenHTTP::Port,
                       "8680");
-  context.setProperty(org::apache::nifi::minifi::processors::ListenHTTP::BasePath,
-                        "/testytesttest");
+  context.setProperty(
+      org::apache::nifi::minifi::processors::ListenHTTP::BasePath,
+      "/testytesttest");
 
-  context2.setProperty(org::apache::nifi::minifi::processors::InvokeHTTP::Method,
-                        "POST");
+  context2.setProperty(
+      org::apache::nifi::minifi::processors::InvokeHTTP::Method, "POST");
   context2.setProperty(org::apache::nifi::minifi::processors::InvokeHTTP::URL,
-                          "http://localhost:8680/testytesttest");
+                       "http://localhost:8680/testytesttest");
   core::ProcessSession session(&context);
   core::ProcessSession session2(&context2);
 
@@ -410,13 +390,14 @@ laConnection->setRelationship(core::Relationship("success", "description"));
 
   /*
    explicit FlowFileRecord(std::shared_ptr<core::Repository> flow_repository,
-                          std::map<std::string, std::string> attributes,
-                          std::shared_ptr<ResourceClaim> claim = nullptr);
+   std::map<std::string, std::string> attributes,
+   std::shared_ptr<ResourceClaim> claim = nullptr);
    */
-  std::map<std::string,std::string> attributes;
+  std::map<std::string, std::string> attributes;
   attributes["testy"] = "test";
-  std::shared_ptr<minifi::FlowFileRecord> flow = std::make_shared<minifi::FlowFileRecord>(repo,attributes);
-  session2.write(flow,&callback);
+  std::shared_ptr<minifi::FlowFileRecord> flow = std::make_shared<
+      minifi::FlowFileRecord>(repo, attributes);
+  session2.write(flow, &callback);
 
   invokehttp->incrementActiveTasks();
   invokehttp->setScheduledState(core::ScheduledState::RUNNING);
@@ -429,15 +410,12 @@ laConnection->setRelationship(core::Relationship("success", "description"));
   listenhttp->onSchedule(&context, &factory);
   listenhttp->onTrigger(&context, &session);
 
-
-
   provenance::ProvenanceReporter *reporter = session.getProvenanceReporter();
   std::set<provenance::ProvenanceEventRecord*> records = reporter->getEvents();
   record = session.get();
   REQUIRE(record == nullptr);
   REQUIRE(records.size() == 0);
 
-
   listenhttp->incrementActiveTasks();
   listenhttp->setScheduledState(core::ScheduledState::RUNNING);
   listenhttp->onTrigger(&context, &session);
@@ -454,18 +432,14 @@ laConnection->setRelationship(core::Relationship("success", "description"));
   session2.commit();
   records = reporter->getEvents();
 
-
-
   for (provenance::ProvenanceEventRecord *provEventRecord : records) {
     REQUIRE(provEventRecord->getComponentType() == listenhttp->getName());
   }
   std::shared_ptr<core::FlowFile> ffr = session2.get();
   std::string log_attribute_output = oss.str();
-std::cout << log_attribute_output << std::endl;
-  REQUIRE( log_attribute_output.find("exiting because method is POST") != std::string::npos );
+  REQUIRE(
+      log_attribute_output.find("exiting because method is POST")
+          != std::string::npos);
 
 }
 
-
-
-

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/unit/MockClasses.h
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/MockClasses.h b/libminifi/test/unit/MockClasses.h
new file mode 100644
index 0000000..d32184b
--- /dev/null
+++ b/libminifi/test/unit/MockClasses.h
@@ -0,0 +1,139 @@
+/**
+ *
+ * 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.
+ */
+#ifndef LIBMINIFI_TEST_UNIT_MOCKCLASSES_H_
+#define LIBMINIFI_TEST_UNIT_MOCKCLASSES_H_
+
+#include "core/controller/ControllerService.h"
+#include "core/Processor.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+
+std::atomic<bool> disabled;
+
+class MockControllerService : public core::controller::ControllerService {
+ public:
+  explicit MockControllerService(const std::string &name, const std::string &id)
+      : ControllerService(name, id) {
+
+  }
+
+  explicit MockControllerService(const std::string &name, uuid_t uuid)
+      : ControllerService(name, uuid) {
+
+  }
+
+  explicit MockControllerService(const std::string &name)
+      : ControllerService(name, 0) {
+
+  }
+  MockControllerService() {
+
+  }
+
+  ~MockControllerService() {
+
+  }
+
+  virtual void initialize() {
+    core::controller::ControllerService::initialize();
+    enable();
+  }
+
+  std::string doSomething() {
+    return str;
+  }
+
+  virtual void enable() {
+    str = "pushitrealgood";
+  }
+
+  void yield() {
+
+  }
+
+  bool isRunning() {
+    return true;
+  }
+
+  bool isWorkAvailable() {
+    return true;
+  }
+ protected:
+  std::string str;
+};
+
+class MockProcessor : public core::Processor {
+ public:
+
+  explicit MockProcessor(const std::string &name, uuid_t uuid)
+      : Processor(name, uuid) {
+    setTriggerWhenEmpty(true);
+  }
+
+  explicit MockProcessor(const std::string &name)
+      : Processor(name, 0) {
+    setTriggerWhenEmpty(true);
+  }
+
+  ~MockProcessor() {
+
+  }
+
+  virtual void initialize() {
+    core::Property property("linkedService", "Linked service");
+    std::set<core::Property> properties;
+    properties.insert(property);
+    setSupportedProperties(properties);
+
+  }
+
+  // OnTrigger method, implemented by NiFi Processor Designer
+  virtual void onTrigger(core::ProcessContext *context,
+                         core::ProcessSession *session) {
+
+    std::string linked_service = "";
+    getProperty("linkedService", linked_service);
+    if (!IsNullOrEmpty(linked_service)) {
+
+      std::shared_ptr<core::controller::ControllerService> service = context
+          ->getControllerService(linked_service);
+
+      if (!disabled.load()) {
+        assert(true == context->isControllerServiceEnabled(linked_service));
+        assert(nullptr != service);
+        assert(
+            "pushitrealgood"
+                == std::static_pointer_cast<MockControllerService>(service)
+                    ->doSomething());
+      } else {
+        assert(false == context->isControllerServiceEnabled(linked_service));
+      }
+
+      //verify we have access to the controller service
+      // and verify that we can execute it.
+
+    }
+  }
+
+  bool isYield() {
+    return false;
+  }
+
+};
+
+#endif /* LIBMINIFI_TEST_UNIT_MOCKCLASSES_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/unit/ProvenanceTestHelper.h
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/ProvenanceTestHelper.h b/libminifi/test/unit/ProvenanceTestHelper.h
index 58ae870..67b5c65 100644
--- a/libminifi/test/unit/ProvenanceTestHelper.h
+++ b/libminifi/test/unit/ProvenanceTestHelper.h
@@ -93,7 +93,7 @@ class TestRepository : public core::Repository {
 class TestFlowRepository : public core::repository::FlowFileRepository {
  public:
   TestFlowRepository()
-      : core::repository::FlowFileRepository("./", 1000, 100, 0) {
+      : core::repository::FlowFileRepository("./dir", 1000, 100, 0) {
   }
   // initialize
   bool initialize() {
@@ -154,9 +154,7 @@ class TestFlowRepository : public core::repository::FlowFileRepository {
   std::map<std::string, std::string> repositoryResults;
 };
 
-
-
-class TestFlowController : public minifi::FlowController {
+class TestFlowController : public minifi::FlowController{
 
 public:
   TestFlowController(std::shared_ptr<core::Repository> repo,

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/unit/ProvenanceTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/ProvenanceTests.cpp b/libminifi/test/unit/ProvenanceTests.cpp
index 2e41cc8..6a134ed 100644
--- a/libminifi/test/unit/ProvenanceTests.cpp
+++ b/libminifi/test/unit/ProvenanceTests.cpp
@@ -17,8 +17,8 @@
  */
 
 #include "../TestBase.h"
+#include "../unit/ProvenanceTestHelper.h"
 
-#include "ProvenanceTestHelper.h"
 #include "provenance/Provenance.h"
 #include "FlowFileRecord.h"
 #include "core/Core.h"

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/unit/RepoTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/RepoTests.cpp b/libminifi/test/unit/RepoTests.cpp
index de51ead..c8deb89 100644
--- a/libminifi/test/unit/RepoTests.cpp
+++ b/libminifi/test/unit/RepoTests.cpp
@@ -17,8 +17,8 @@
  */
 
 #include "../TestBase.h"
+#include "../unit/ProvenanceTestHelper.h"
 
-#include "ProvenanceTestHelper.h"
 #include "provenance/Provenance.h"
 #include "FlowFileRecord.h"
 #include "core/Core.h"

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/unit/SerializationTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/SerializationTests.cpp b/libminifi/test/unit/SerializationTests.cpp
index 0841fbd..039ed57 100644
--- a/libminifi/test/unit/SerializationTests.cpp
+++ b/libminifi/test/unit/SerializationTests.cpp
@@ -22,11 +22,12 @@
 #include <uuid/uuid.h>
 #include "core/logging/LogAppenders.h"
 #include "core/logging/BaseLogger.h"
-#include "SiteToSiteHelper.h"
 #include <algorithm>
 #include <string>
 #include <memory>
+
 #include "../TestBase.h"
+#include "../unit/SiteToSiteHelper.h"
 #define FMT_DEFAULT fmt_lower
 
 using namespace org::apache::nifi::minifi::io;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/unit/Site2SiteTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/Site2SiteTests.cpp b/libminifi/test/unit/Site2SiteTests.cpp
index d82a3cf..5af86d5 100644
--- a/libminifi/test/unit/Site2SiteTests.cpp
+++ b/libminifi/test/unit/Site2SiteTests.cpp
@@ -22,11 +22,12 @@
 #include <uuid/uuid.h>
 #include "core/logging/LogAppenders.h"
 #include "core/logging/BaseLogger.h"
-#include "SiteToSiteHelper.h"
 #include <algorithm>
 #include <string>
 #include <memory>
+
 #include "../TestBase.h"
+#include "../unit/SiteToSiteHelper.h"
 #define FMT_DEFAULT fmt_lower
 
 using namespace org::apache::nifi::minifi::io;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/unit/ThreadPoolTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/ThreadPoolTests.cpp b/libminifi/test/unit/ThreadPoolTests.cpp
new file mode 100644
index 0000000..5c85e19
--- /dev/null
+++ b/libminifi/test/unit/ThreadPoolTests.cpp
@@ -0,0 +1,38 @@
+/**
+ *
+ * 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.
+ */
+
+#include <future>
+#include "../TestBase.h"
+#include "utils/ThreadPool.h"
+
+bool function() {
+  return true;
+}
+
+TEST_CASE("ThreadPoolTest1", "[TPT1]") {
+  utils::ThreadPool<bool> pool(5);
+  std::function<bool()> f_ex = function;
+  utils::Worker<bool> functor(f_ex);
+  pool.start();
+  std::future<bool> fut = pool.execute(std::move(functor));
+
+  fut.wait();
+
+  REQUIRE(true == fut.get());
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/unit/YamlCongifurationTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/YamlCongifurationTests.cpp b/libminifi/test/unit/YamlCongifurationTests.cpp
index 3f804ab..0c229d0 100644
--- a/libminifi/test/unit/YamlCongifurationTests.cpp
+++ b/libminifi/test/unit/YamlCongifurationTests.cpp
@@ -127,7 +127,7 @@ TEST_CASE("Test YAML Config 1", "[testyamlconfig1]") {
       "    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>()));
+  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);
 
@@ -178,7 +178,7 @@ TEST_CASE("Test YAML Config Missing Required Fields", "[testyamlconfig2]") {
       "            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>()));
+  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);

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/unit/resource/TestHTTPGet.yml
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/resource/TestHTTPGet.yml b/libminifi/test/unit/resource/TestHTTPGet.yml
deleted file mode 100644
index 0783b8e..0000000
--- a/libminifi/test/unit/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/c9940e94/libminifi/test/unit/resource/TestHTTPPost.yml
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/resource/TestHTTPPost.yml b/libminifi/test/unit/resource/TestHTTPPost.yml
deleted file mode 100644
index 837194d..0000000
--- a/libminifi/test/unit/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/c9940e94/main/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/main/CMakeLists.txt b/main/CMakeLists.txt
index a0e43b8..87506fa 100644
--- a/main/CMakeLists.txt
+++ b/main/CMakeLists.txt
@@ -23,7 +23,7 @@ IF(POLICY CMP0048)
   CMAKE_POLICY(SET CMP0048 OLD)
 ENDIF(POLICY CMP0048)
 
-include_directories(../include ../libminifi/include  ../libminifi/include/core/yaml  ../libminifi/include/core ../thirdparty/yaml-cpp-yaml-cpp-0.5.3/include ../thirdparty/civetweb-1.9.1/include ../thirdparty/jsoncpp/include ../thirdparty/leveldb-1.18/include ../thirdparty/)
+include_directories(../include ../libminifi/include  ../libminifi/include/processors/ ../libminifi/include/core/yaml  ../libminifi/include/core ../thirdparty/yaml-cpp-yaml-cpp-0.5.3/include ../thirdparty/civetweb-1.9.1/include ../thirdparty/leveldb-1.18/include ../thirdparty/)
 
 find_package(Boost REQUIRED)
 include_directories(${Boost_INCLUDE_DIRS})

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/main/MiNiFiMain.cpp
----------------------------------------------------------------------
diff --git a/main/MiNiFiMain.cpp b/main/MiNiFiMain.cpp
index daf4a8f..3eb16ae 100644
--- a/main/MiNiFiMain.cpp
+++ b/main/MiNiFiMain.cpp
@@ -60,7 +60,6 @@
 // Variables that allow us to avoid a timed wait.
 sem_t *running;
 //! Flow Controller
-static std::unique_ptr<minifi::FlowController> controller = nullptr;
 
 /**
  * Removed the stop command from the signal handler so that we could trigger
@@ -178,10 +177,11 @@ int main(int argc, char **argv) {
       core::createFlowConfiguration(prov_repo, flow_repo, configure, stream_factory,
                                    nifi_configuration_class_name));
 
-  controller = std::unique_ptr<minifi::FlowController>(
+  std::shared_ptr<minifi::FlowController> controller = std::unique_ptr<minifi::FlowController>(
       new minifi::FlowController(prov_repo, flow_repo, configure,
                                  std::move(flow_configuration)));
 
+  logger->log_info("Loading FlowController");
   // Load flow from specified configuration file
   controller->load();
   // Start Processing the flow


[5/5] nifi-minifi-cpp git commit: MINIFI-226: Add controller services capabilities along with unit tests Fix test failures Update Travis YML Update readme to link to MiNiFi licensing information

Posted by al...@apache.org.
MINIFI-226: Add controller services capabilities along with unit tests
Fix test failures
Update Travis YML
Update readme to link to MiNiFi licensing information

MINIFI-253 : Add basic Process Loader capability

This closes #83.

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/c9940e94
Tree: http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/tree/c9940e94
Diff: http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/diff/c9940e94

Branch: refs/heads/master
Commit: c9940e9454b02408a75ac47daae78d99cb9ef13d
Parents: 372f2d6
Author: Marc Parisi <ph...@apache.org>
Authored: Sat Apr 1 08:20:42 2017 -0400
Committer: Aldrin Piri <al...@apache.org>
Committed: Thu May 18 08:59:57 2017 -0400

----------------------------------------------------------------------
 .gitignore                                      |   3 +-
 .travis.yml                                     |   2 +-
 CMakeLists.txt                                  |  72 ++--
 README.md                                       |  31 ++
 libminifi/CMakeLists.txt                        |   3 +-
 libminifi/include/EventDrivenSchedulingAgent.h  |  10 +-
 libminifi/include/FlowController.h              | 132 +++++-
 libminifi/include/SchedulingAgent.h             |  33 +-
 libminifi/include/ThreadedSchedulingAgent.h     |  11 +-
 libminifi/include/TimerDrivenSchedulingAgent.h  |   7 +-
 .../include/controllers/SSLContextService.h     | 174 ++++++++
 libminifi/include/core/ClassLoader.h            | 259 ++++++++++++
 libminifi/include/core/ConfigurableComponent.h  |  11 +
 libminifi/include/core/ConfigurationFactory.h   |   7 +-
 libminifi/include/core/Connectable.h            |  12 +-
 libminifi/include/core/Core.h                   |   3 +-
 libminifi/include/core/FlowConfiguration.h      |  31 +-
 libminifi/include/core/ProcessContext.h         |  59 ++-
 libminifi/include/core/ProcessGroup.h           |  23 ++
 libminifi/include/core/Property.h               |  29 +-
 libminifi/include/core/Resource.h               |  54 +++
 .../include/core/controller/ControllerService.h | 142 +++++++
 .../core/controller/ControllerServiceLookup.h   |  89 ++++
 .../core/controller/ControllerServiceMap.h      | 120 ++++++
 .../core/controller/ControllerServiceNode.h     | 133 ++++++
 .../core/controller/ControllerServiceProvider.h | 306 ++++++++++++++
 .../controller/StandardControllerServiceNode.h  | 107 +++++
 .../StandardControllerServiceProvider.h         | 229 +++++++++++
 libminifi/include/core/yaml/YamlConfiguration.h |  54 ++-
 libminifi/include/io/StreamFactory.h            |   6 +-
 libminifi/include/io/TLSSocket.h                | 197 +++++++++
 libminifi/include/io/validation.h               |  14 +-
 libminifi/include/processors/AppendHostInfo.h   |   4 +
 libminifi/include/processors/ExecuteProcess.h   |   3 +
 libminifi/include/processors/GenerateFlowFile.h |   3 +
 libminifi/include/processors/GetFile.h          |   3 +
 libminifi/include/processors/InvokeHTTP.h       |  30 +-
 libminifi/include/processors/ListenHTTP.h       |   6 +-
 libminifi/include/processors/ListenSyslog.h     |   3 +
 libminifi/include/processors/LoadProcessors.h   |  34 ++
 libminifi/include/processors/LogAttribute.h     |   3 +
 libminifi/include/processors/PutFile.h          |   3 +
 libminifi/include/processors/TailFile.h         |   6 +-
 libminifi/include/properties/Configure.h        |  11 +-
 libminifi/include/utils/ThreadPool.h            | 355 ++++++++--------
 libminifi/src/Configure.cpp                     |  17 +-
 libminifi/src/EventDrivenSchedulingAgent.cpp    |   4 +-
 libminifi/src/FlowController.cpp                | 233 ++++++++++-
 libminifi/src/SchedulingAgent.cpp               |  35 +-
 libminifi/src/ThreadedSchedulingAgent.cpp       |  26 +-
 libminifi/src/TimerDrivenSchedulingAgent.cpp    |   4 +-
 libminifi/src/controllers/SSLContextService.cpp | 226 ++++++++++
 libminifi/src/core/ClassLoader.cpp              |  70 ++++
 libminifi/src/core/ConfigurableComponent.cpp    |  42 +-
 libminifi/src/core/ConfigurationFactory.cpp     |  13 +-
 libminifi/src/core/Connectable.cpp              |   6 +-
 libminifi/src/core/FlowConfiguration.cpp        |  66 +--
 libminifi/src/core/ProcessGroup.cpp             |  16 +
 libminifi/src/core/Processor.cpp                |  20 +-
 libminifi/src/core/Property.cpp                 |  24 +-
 .../core/controller/ControllerServiceNode.cpp   |  49 +++
 .../controller/ControllerServiceProvider.cpp    |  50 +++
 .../StandardControllerServiceNode.cpp           |  69 ++++
 libminifi/src/core/yaml/YamlConfiguration.cpp   | 212 +++++++---
 libminifi/src/processors/GetFile.cpp            |   4 +
 libminifi/src/processors/InvokeHTTP.cpp         |  75 ++--
 libminifi/test/FlowFileRecordTest.cpp           |  28 --
 libminifi/test/HttpGetIntegrationTest.cpp       | 120 ------
 libminifi/test/HttpPostIntegrationTest.cpp      | 120 ------
 libminifi/test/ProcessorTests.cpp               | 408 ------------------
 libminifi/test/SocketTests.cpp                  | 185 ---------
 libminifi/test/TestExecuteProcess.cpp           | 131 ------
 .../ControllerServiceIntegrationTests.cpp       | 186 +++++++++
 .../test/integration/HttpGetIntegrationTest.cpp | 128 ++++++
 .../integration/HttpPostIntegrationTest.cpp     | 126 ++++++
 libminifi/test/integration/ProcessorTests.cpp   | 411 +++++++++++++++++++
 libminifi/test/integration/SocketTests.cpp      | 185 +++++++++
 .../test/integration/TestExecuteProcess.cpp     | 137 +++++++
 .../test/resources/TestControllerServices.yml   |  62 +++
 libminifi/test/resources/TestHTTPGet.yml        |  73 ++++
 libminifi/test/resources/TestHTTPGetSecure.yml  |  88 ++++
 libminifi/test/resources/TestHTTPPost.yml       |  87 ++++
 libminifi/test/resources/cn.ckey.pem            |  31 ++
 libminifi/test/resources/cn.crt.pem             |  25 ++
 libminifi/test/resources/cn.pass                |   0
 libminifi/test/resources/nifi-cert.pem          |  20 +
 libminifi/test/unit/ClassLoaderTests.cpp        |  35 ++
 libminifi/test/unit/ControllerServiceTests.cpp  |  89 ++++
 libminifi/test/unit/InvokeHTTPTests.cpp         | 198 ++++-----
 libminifi/test/unit/MockClasses.h               | 139 +++++++
 libminifi/test/unit/ProvenanceTestHelper.h      |   6 +-
 libminifi/test/unit/ProvenanceTests.cpp         |   2 +-
 libminifi/test/unit/RepoTests.cpp               |   2 +-
 libminifi/test/unit/SerializationTests.cpp      |   3 +-
 libminifi/test/unit/Site2SiteTests.cpp          |   3 +-
 libminifi/test/unit/ThreadPoolTests.cpp         |  38 ++
 libminifi/test/unit/YamlCongifurationTests.cpp  |   4 +-
 libminifi/test/unit/resource/TestHTTPGet.yml    |  73 ----
 libminifi/test/unit/resource/TestHTTPPost.yml   |  87 ----
 main/CMakeLists.txt                             |   2 +-
 main/MiNiFiMain.cpp                             |   4 +-
 101 files changed, 5593 insertions(+), 1731 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 2da62d5..0f533cc 100644
--- a/.gitignore
+++ b/.gitignore
@@ -22,7 +22,8 @@ bin
 target
 thirdparty/**/*.o
 thirdparty/**/*.a
+libminifi/test/**/*.a
 docs/generated
 
 # Ignore source files that have been placed in the docker directory during build
-docker/minificppsource
\ No newline at end of file
+docker/minificppsource

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 0fa8aec..b93301c 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 && make test && make linter && make docs
+  - mkdir ./build && cd ./build && cmake .. && make VERBOSE=1 && make test ARGS="-V" && make linter && make docs

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 066169d..e441c44 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -143,29 +143,48 @@ include(CPack)
 
 enable_testing(test)
     file(GLOB LIBMINIFI_TEST_SOURCES "libminifi/test/unit/*.cpp")
-    add_executable(tests ${LIBMINIFI_TEST_SOURCES} ${SPD_SOURCES})
-    target_include_directories(tests PRIVATE BEFORE "thirdparty/catch")
-    target_include_directories(tests PRIVATE BEFORE "thirdparty/yaml-cpp-yaml-cpp-0.5.3/include")
-    target_include_directories(tests PRIVATE BEFORE "thirdparty/jsoncpp/include")
-    target_include_directories(tests PRIVATE BEFORE ${LEVELDB_INCLUDE_DIRS})
-    target_include_directories(tests PRIVATE BEFORE "include")
-    target_include_directories(tests PRIVATE BEFORE "libminifi/include/")
-    target_include_directories(tests PRIVATE BEFORE "libminifi/include/core")
-    target_include_directories(tests PRIVATE BEFORE "libminifi/include/core/repository")
-    target_include_directories(tests PRIVATE BEFORE "libminifi/include/core/yaml")
-    target_include_directories(tests PRIVATE BEFORE "libminifi/include/io")
-    target_include_directories(tests PRIVATE BEFORE "libminifi/include/utils")
-    target_include_directories(tests PRIVATE BEFORE "libminifi/include/processors")
-    target_include_directories(tests PRIVATE BEFORE "libminifi/include/provenance")
-    target_link_libraries(tests ${CMAKE_THREAD_LIBS_INIT} ${UUID_LIBRARIES} ${LEVELDB_LIBRARIES} ${OPENSSL_LIBRARIES} minifi yaml-cpp c-library civetweb-cpp ${JSON_CPP_LIB})
+    add_executable(unit_tests ${LIBMINIFI_TEST_SOURCES} ${SPD_SOURCES})
+    target_include_directories(unit_tests PRIVATE BEFORE "thirdparty/catch")
+    target_include_directories(unit_tests PRIVATE BEFORE "thirdparty/yaml-cpp-yaml-cpp-0.5.3/include")
+    target_include_directories(unit_tests PRIVATE BEFORE "thirdparty/jsoncpp/include")
+    target_include_directories(unit_tests PRIVATE BEFORE ${LEVELDB_INCLUDE_DIRS})
+    target_include_directories(unit_tests PRIVATE BEFORE "include")
+    target_include_directories(unit_tests PRIVATE BEFORE "libminifi/include/")
+    target_include_directories(unit_tests PRIVATE BEFORE "libminifi/include/core")
+    target_include_directories(unit_tests PRIVATE BEFORE "libminifi/include/core/controller")
+    target_include_directories(unit_tests PRIVATE BEFORE "libminifi/include/core/repository")
+    target_include_directories(unit_tests PRIVATE BEFORE "libminifi/include/core/yaml")
+    target_include_directories(unit_tests PRIVATE BEFORE "libminifi/include/io")
+    target_include_directories(unit_tests PRIVATE BEFORE "libminifi/include/utils")
+    target_include_directories(unit_tests PRIVATE BEFORE "libminifi/include/processors")
+    target_include_directories(unit_tests PRIVATE BEFORE "libminifi/include/provenance")
+    target_link_libraries(unit_tests ${CMAKE_THREAD_LIBS_INIT} ${UUID_LIBRARIES} ${LEVELDB_LIBRARIES} ${OPENSSL_LIBRARIES} minifi yaml-cpp c-library civetweb-cpp ${JSON_CPP_LIB})
     if (CURL_FOUND)
-        target_include_directories(tests PRIVATE BEFORE ${CURL_INCLUDE_DIRS})
-                target_link_libraries(tests ${CURL_LIBRARIES})
+        target_include_directories(unit_tests PRIVATE BEFORE ${CURL_INCLUDE_DIRS})
+                target_link_libraries(unit_tests ${CURL_LIBRARIES})
         endif(CURL_FOUND)
-    add_test(NAME LibMinifiTests COMMAND tests)
+    add_test(NAME LibMinifiUnitTests COMMAND unit_tests)
+            
+    
+    file(GLOB LIBMINIFI_TEST_CS "libminifi/test/integration/ControllerServiceIntegrationTests.cpp")
+    add_executable(testControllerServices ${LIBMINIFI_TEST_CS} ${SPD_SOURCES})
+    target_include_directories(testControllerServices PRIVATE BEFORE "thirdparty/catch")
+    target_include_directories(testControllerServices PRIVATE BEFORE "thirdparty/yaml-cpp-yaml-cpp-0.5.3/include")
+    target_include_directories(testControllerServices PRIVATE BEFORE ${LEVELDB_INCLUDE_DIRS})
+    target_include_directories(testControllerServices PRIVATE BEFORE "include")
+    target_include_directories(testControllerServices PRIVATE BEFORE "libminifi/include/")
+    target_include_directories(testControllerServices PRIVATE BEFORE "libminifi/include/core")
+    target_include_directories(testControllerServices PRIVATE BEFORE "libminifi/include/core/controller")
+    target_include_directories(testControllerServices PRIVATE BEFORE "libminifi/include/core/repository")
+    target_include_directories(testControllerServices PRIVATE BEFORE "libminifi/include/io")
+    target_include_directories(testControllerServices PRIVATE BEFORE "libminifi/include/utils")
+    target_include_directories(testControllerServices PRIVATE BEFORE "libminifi/include/processors")
+    target_include_directories(testControllerServices PRIVATE BEFORE "libminifi/include/provenance")
+    target_link_libraries(testControllerServices ${CMAKE_THREAD_LIBS_INIT} ${UUID_LIBRARIES} ${LEVELDB_LIBRARIES} ${OPENSSL_LIBRARIES} minifi yaml-cpp c-library civetweb-cpp ${JSON_CPP_LIB})
+    add_test(NAME TestControllerServices COMMAND testControllerServices "${CMAKE_SOURCE_DIR}/libminifi/test/resources/TestControllerServices.yml" "${CMAKE_SOURCE_DIR}/libminifi/test/resources/")
     
      
-    file(GLOB LIBMINIFI_TEST_CS "libminifi/test/HttpGetIntegrationTest.cpp")
+    file(GLOB LIBMINIFI_TEST_CS "libminifi/test/integration/HttpGetIntegrationTest.cpp")
     add_executable(testHttpGet ${LIBMINIFI_TEST_CS} ${SPD_SOURCES})
     target_include_directories(testHttpGet PRIVATE BEFORE "thirdparty/catch")
     target_include_directories(testHttpGet PRIVATE BEFORE "thirdparty/yaml-cpp-yaml-cpp-0.5.3/include")
@@ -184,9 +203,10 @@ enable_testing(test)
         target_include_directories(testHttpGet PRIVATE BEFORE ${CURL_INCLUDE_DIRS})
                 target_link_libraries(testHttpGet ${CURL_LIBRARIES})
         endif(CURL_FOUND)
-    add_test(NAME testHttpGet COMMAND testHttpGet "${CMAKE_SOURCE_DIR}/libminifi/test/unit/resource/TestHTTPGet.yml")
+    add_test(NAME testHttpGet COMMAND testHttpGet "${CMAKE_SOURCE_DIR}/libminifi/test/resources/TestHTTPGet.yml"  "${CMAKE_SOURCE_DIR}/libminifi/test/resources/")
+    add_test(NAME testHttpGetSecure COMMAND testHttpGet "${CMAKE_SOURCE_DIR}/libminifi/test/resources/TestHTTPGetSecure.yml"  "${CMAKE_SOURCE_DIR}/libminifi/test/resources/")
     
-    file(GLOB LIBMINIFI_TEST_CS "libminifi/test/HttpPostIntegrationTest.cpp")
+    file(GLOB LIBMINIFI_TEST_CS "libminifi/test/integration/HttpPostIntegrationTest.cpp")
     add_executable(testHttpPost ${LIBMINIFI_TEST_CS} ${SPD_SOURCES})
     target_include_directories(testHttpPost PRIVATE BEFORE "thirdparty/catch")
     target_include_directories(testHttpPost PRIVATE BEFORE "thirdparty/yaml-cpp-yaml-cpp-0.5.3/include")
@@ -205,16 +225,17 @@ enable_testing(test)
         target_include_directories(testHttpPost PRIVATE BEFORE ${CURL_INCLUDE_DIRS})
     	target_link_libraries(testHttpPost ${CURL_LIBRARIES})
     endif(CURL_FOUND)
-    add_test(NAME testHttpPost COMMAND testHttpPost "${CMAKE_SOURCE_DIR}/libminifi/test/unit/resource/TestHTTPPost.yml")
+    add_test(NAME testHttpPost COMMAND testHttpPost "${CMAKE_SOURCE_DIR}/libminifi/test/resources/TestHTTPPost.yml")
     
     
-    file(GLOB LIBMINIFI_TEST_EXECUTE_PROCESS "libminifi/test/TestExecuteProcess.cpp")
+    file(GLOB LIBMINIFI_TEST_EXECUTE_PROCESS "libminifi/test/integration/TestExecuteProcess.cpp")
     add_executable(testExecuteProcess ${LIBMINIFI_TEST_EXECUTE_PROCESS} ${SPD_SOURCES})
     target_include_directories(testExecuteProcess PRIVATE BEFORE "thirdparty/yaml-cpp-yaml-cpp-0.5.3/include")
     target_include_directories(testExecuteProcess PRIVATE BEFORE ${LEVELDB_INCLUDE_DIRS})
     target_include_directories(testExecuteProcess PRIVATE BEFORE "include")
     target_include_directories(testExecuteProcess PRIVATE BEFORE "libminifi/include/")
     target_include_directories(testExecuteProcess PRIVATE BEFORE "libminifi/include/core")
+    target_include_directories(testExecuteProcess PRIVATE BEFORE "libminifi/include/core/controller")
     target_include_directories(testExecuteProcess PRIVATE BEFORE "libminifi/include/core/repository")
     target_include_directories(testExecuteProcess PRIVATE BEFORE "libminifi/include/core/yaml")
     target_include_directories(testExecuteProcess PRIVATE BEFORE "libminifi/include/io")
@@ -228,7 +249,7 @@ enable_testing(test)
     target_link_libraries(testExecuteProcess ${CMAKE_THREAD_LIBS_INIT} ${UUID_LIBRARIES} ${LEVELDB_LIBRARIES} ${OPENSSL_LIBRARIES} minifi yaml-cpp c-library civetweb-cpp ${JSON_CPP_LIB})
     add_test(NAME ExecuteProcess COMMAND testExecuteProcess)
     
-    file(GLOB LIBMINIFI_TEST_SOCKETS "libminifi/test/SocketTests.cpp")
+    file(GLOB LIBMINIFI_TEST_SOCKETS "libminifi/test/integration/SocketTests.cpp")
     add_executable(testSockets ${LIBMINIFI_TEST_SOCKETS} ${SPD_SOURCES})
     target_include_directories(testSockets PRIVATE BEFORE "thirdparty/catch")
     target_include_directories(testSockets PRIVATE BEFORE "thirdparty/yaml-cpp-yaml-cpp-0.5.3/include")
@@ -248,7 +269,7 @@ enable_testing(test)
     target_link_libraries(testSockets ${CMAKE_THREAD_LIBS_INIT} ${UUID_LIBRARIES} ${LEVELDB_LIBRARIES} ${OPENSSL_LIBRARIES} minifi yaml-cpp c-library civetweb-cpp ${JSON_CPP_LIB})
     add_test(NAME TestSockets COMMAND testSockets)
     
-    file(GLOB LIBMINIFI_TEST_PROCESSORS "libminifi/test/ProcessorTests.cpp")
+    file(GLOB LIBMINIFI_TEST_PROCESSORS "libminifi/test/integration/ProcessorTests.cpp")
     add_executable(testProcessors ${LIBMINIFI_TEST_PROCESSORS} ${SPD_SOURCES})
     target_include_directories(testProcessors PRIVATE BEFORE "thirdparty/catch")
     target_include_directories(testProcessors PRIVATE BEFORE "thirdparty/yaml-cpp-yaml-cpp-0.5.3/include")
@@ -271,6 +292,7 @@ enable_testing(test)
     add_test(NAME TestProcessors COMMAND testProcessors)
     
 
+
 # Create a custom build target called "docker" that will invoke DockerBuild.sh and create the NiFi-MiNiFi-CPP Docker image
 add_custom_target(
     docker

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index a8b95b7..df89fb8 100644
--- a/README.md
+++ b/README.md
@@ -310,6 +310,24 @@ Additionally, users can utilize the MiNiFi Toolkit Converter (version 0.0.1 - sc
       host: localhost
       port uuid: 471deef6-2a6e-4a7d-912a-81cc17e3a204
       batch size: 100
+      
+### Controller Services
+ If you need to reference a controller service in your config.yml file, use the following template. In the example, below, ControllerServiceClass is the name of the class defining the controller Service. ControllerService1 
+ is linked to ControllerService2, and requires the latter to be started for ControllerService1 to start. 
+     
+	Controller Services:
+      - name: ControllerService1
+ 	    id: 2438e3c8-015a-1000-79ca-83af40ec1974
+	  	class: ControllerServiceClass
+	  	Properties:
+	      Property one: value
+	      Linked Services:
+	        - value: ControllerService2
+	  - name: ControllerService2
+	    id: 2438e3c8-015a-1000-79ca-83af40ec1992
+	  	class: ControllerServiceClass
+	  	Properties:
+        
 
 ### Running
 After completing a [build](#building), the application can be run by issuing the following from :
@@ -341,6 +359,16 @@ guide. It is located [here](https://github.com/google/styleguide/blob/gh-pages/e
 New contributions are expected to follow the Google style guide when it is reasonable. 
 Additionally, all new files must include a copy of the Apache License Header. 
 
+MiNiFi C++ contains a dynamic loading mechanism that loads arbitrary objects. To maintain
+consistency of development amongst the NiFi ecosystem, it is called a class loader. If you
+are contributing a custom Processor or Controller Service, the mechanism to register your class
+into the default class loader is a pragma definition named:
+    
+    REGISTER_RESOURCE(CLASSNAME);
+    
+To use this include REGISTER_RESOURCE(YourClassName); in your header file. The default class
+loader will make instnaces of YourClassName available for inclusion.  
+
 Once you have completed your changes, including source code and tests, you can verify that
 you follow the Google style guide by running the following command:
      $ make linter. 
@@ -350,6 +378,9 @@ This will provide output for all source files.
 Except as otherwise noted this software is licensed under the
 [Apache License, Version 2.0](http://www.apache.org/licenses/LICENSE-2.0.html)
 
+For additional information regarding the source of included projects and
+the corresponding licenses, you may visit the following [website](https://cwiki.apache.org/confluence/display/MINIFI/Licensing+Information)
+
 Licensed 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://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/libminifi/CMakeLists.txt b/libminifi/CMakeLists.txt
index a8da0e0..7e260d3 100644
--- a/libminifi/CMakeLists.txt
+++ b/libminifi/CMakeLists.txt
@@ -61,7 +61,8 @@ include_directories(../thirdparty/civetweb-1.9.1/include)
 include_directories(../thirdparty/jsoncpp/include)
 include_directories(include)
 
-file(GLOB SOURCES  "src/core/logging/*.cpp" "src/io/*.cpp" "src/io/tls/*.cpp" "src/core/*.cpp"  "src/core/repository/*.cpp" "src/core/yaml/*.cpp" "src/core/reporting/*.cpp" "src/provenance/*.cpp" "src/processors/*.cpp" "src/*.cpp")
+file(GLOB SOURCES  "src/core/logging/*.cpp"  "src/io/*.cpp" "src/io/tls/*.cpp" "src/core/controller/*.cpp" "src/controllers/*.cpp" "src/core/*.cpp"  "src/core/repository/*.cpp" "src/core/yaml/*.cpp" "src/core/reporting/*.cpp"  "src/provenance/*.cpp" "src/processors/*.cpp" "src/*.cpp")
+
 file(GLOB SPD_SOURCES "../include/spdlog/*")
 
 # Workaround the limitations of having a

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/EventDrivenSchedulingAgent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/EventDrivenSchedulingAgent.h b/libminifi/include/EventDrivenSchedulingAgent.h
index 22a68f3..2e49ddf 100644
--- a/libminifi/include/EventDrivenSchedulingAgent.h
+++ b/libminifi/include/EventDrivenSchedulingAgent.h
@@ -36,10 +36,14 @@ class EventDrivenSchedulingAgent : public ThreadedSchedulingAgent {
  public:
   // Constructor
   /*!
-   * Create a new processor
+   * Create a new event driven scheduling agent.
    */
-  EventDrivenSchedulingAgent(std::shared_ptr<core::Repository> repo, std::shared_ptr<Configure> configure)
-      : ThreadedSchedulingAgent(repo, configure) {
+  EventDrivenSchedulingAgent(
+      std::shared_ptr<core::controller::ControllerServiceProvider> controller_service_provider,
+      std::shared_ptr<core::Repository> repo,
+      std::shared_ptr<Configure> configuration)
+      : ThreadedSchedulingAgent(controller_service_provider, repo,
+                                configuration) {
   }
   // Destructor
   virtual ~EventDrivenSchedulingAgent() {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/FlowController.h
----------------------------------------------------------------------
diff --git a/libminifi/include/FlowController.h b/libminifi/include/FlowController.h
index 187448b..6c87653 100644
--- a/libminifi/include/FlowController.h
+++ b/libminifi/include/FlowController.h
@@ -38,6 +38,8 @@
 #include "core/ProcessSession.h"
 #include "core/ProcessGroup.h"
 #include "core/FlowConfiguration.h"
+#include "core/controller/ControllerServiceNode.h"
+#include "core/controller/ControllerServiceProvider.h"
 #include "TimerDrivenSchedulingAgent.h"
 #include "EventDrivenSchedulingAgent.h"
 #include "FlowControlProtocol.h"
@@ -56,7 +58,8 @@ namespace minifi {
  * Flow Controller class. Generally used by FlowController factory
  * as a singleton.
  */
-class FlowController : public core::CoreComponent {
+class FlowController : public core::controller::ControllerServiceProvider,
+    public std::enable_shared_from_this<FlowController> {
  public:
   static const int DEFAULT_MAX_TIMER_DRIVEN_THREAD = 10;
   static const int DEFAULT_MAX_EVENT_DRIVEN_THREAD = 5;
@@ -133,6 +136,122 @@ class FlowController : public core::CoreComponent {
     protocol_->setSerialNumber(number);
   }
 
+  /**
+   * Creates a controller service through the controller service provider impl.
+   * @param type class name
+   * @param id service identifier
+   * @param firstTimeAdded first time this CS was added
+   */
+  virtual std::shared_ptr<core::controller::ControllerServiceNode> createControllerService(
+      const std::string &type, const std::string &id,
+      bool firstTimeAdded);
+
+  /**
+   * controller service provider
+   */
+  /**
+   * removes controller service
+   * @param serviceNode service node to be removed.
+   */
+
+  virtual void removeControllerService(
+      const std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode);
+
+  /**
+   * Enables the controller service services
+   * @param serviceNode service node which will be disabled, along with linked services.
+   */
+  virtual void enableControllerService(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode);
+
+  /**
+   * Enables controller services
+   * @param serviceNoden vector of service nodes which will be enabled, along with linked services.
+   */
+  virtual void enableControllerServices(
+      std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> serviceNodes);
+
+  /**
+   * Disables controller services
+   * @param serviceNode service node which will be disabled, along with linked services.
+   */
+  virtual void disableControllerService(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode);
+
+  /**
+   * Gets all controller services.
+   */
+  virtual std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> getAllControllerServices();
+
+  /**
+   * Gets controller service node specified by <code>id</code>
+   * @param id service identifier
+   * @return shared pointer to the controller service node or nullptr if it does not exist.
+   */
+  virtual std::shared_ptr<core::controller::ControllerServiceNode> getControllerServiceNode(
+      const std::string &id);
+
+  virtual void verifyCanStopReferencingComponents(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode);
+
+  /**
+   * Unschedules referencing components.
+   */
+  virtual std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> unscheduleReferencingComponents(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode);
+
+  /**
+   * Verify can disable referencing components
+   * @param serviceNode service node whose referenced components will be scheduled.
+   */
+  virtual void verifyCanDisableReferencingServices(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode);
+
+  /**
+   * Disables referencing components
+   * @param serviceNode service node whose referenced components will be scheduled.
+   */
+  virtual std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> disableReferencingServices(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode);
+
+  /**
+   * Verify can enable referencing components
+   * @param serviceNode service node whose referenced components will be scheduled.
+   */
+  virtual void verifyCanEnableReferencingServices(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode);
+
+  /**
+   * Determines if the controller service specified by identifier is enabled.
+   */
+  bool isControllerServiceEnabled(const std::string &identifier);
+
+  /**
+   * Enables referencing components
+   * @param serviceNode service node whose referenced components will be scheduled.
+   */
+  virtual std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> enableReferencingServices(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode);
+
+  /**
+   * Schedules referencing components
+   * @param serviceNode service node whose referenced components will be scheduled.
+   */
+  virtual std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> scheduleReferencingComponents(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode);
+
+  /**
+   * Returns controller service components referenced by serviceIdentifier from the embedded
+   * controller service provider;
+   */
+  std::shared_ptr<core::controller::ControllerService> getControllerServiceForComponent(
+      const std::string &serviceIdentifier, const std::string &componentId);
+
+  /**
+   * Enables all controller services for the provider.
+   */
+  virtual void enableAllControllerServices();
+
  protected:
 
   // function to load the flow file repo.
@@ -151,7 +270,7 @@ class FlowController : public core::CoreComponent {
   // NiFi property File Name
   std::string properties_file_name_;
   // Root Process Group
-  std::unique_ptr<core::ProcessGroup> root_;
+  std::shared_ptr<core::ProcessGroup> root_;
   // MAX Timer Driven Threads
   int max_timer_driven_threads_;
   // MAX Event Driven Threads
@@ -171,9 +290,9 @@ class FlowController : public core::CoreComponent {
 
   // Flow Engines
   // Flow Timer Scheduler
-  TimerDrivenSchedulingAgent _timerScheduler;
+  std::shared_ptr<TimerDrivenSchedulingAgent> timer_scheduler_;
   // Flow Event Scheduler
-  EventDrivenSchedulingAgent _eventScheduler;
+  std::shared_ptr<EventDrivenSchedulingAgent> event_scheduler_;
   // Controller Service
   // Config
   // Site to Site Server Listener
@@ -181,6 +300,11 @@ class FlowController : public core::CoreComponent {
   // FlowControl Protocol
   FlowControlProtocol *protocol_;
 
+  std::shared_ptr<Configure> configuration_;
+
+  std::shared_ptr<core::controller::ControllerServiceMap> controller_service_map_;
+
+  std::shared_ptr<core::controller::ControllerServiceProvider> controller_service_provider_;
   // flow configuration object.
   std::unique_ptr<core::FlowConfiguration> flow_configuration_;
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/SchedulingAgent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/SchedulingAgent.h b/libminifi/include/SchedulingAgent.h
index 30df071..1198896 100644
--- a/libminifi/include/SchedulingAgent.h
+++ b/libminifi/include/SchedulingAgent.h
@@ -28,6 +28,7 @@
 #include <algorithm>
 #include <thread>
 #include "utils/TimeUtil.h"
+#include "utils/ThreadPool.h"
 #include "core/Core.h"
 #include "core/logging/Logger.h"
 #include "properties/Configure.h"
@@ -35,6 +36,8 @@
 #include "core/logging/Logger.h"
 #include "core/Processor.h"
 #include "core/ProcessContext.h"
+#include "core/controller/ControllerServiceProvider.h"
+#include "core/controller/ControllerServiceNode.h"
 #include "provenance/ProvenanceRepository.h"
 
 namespace org {
@@ -47,12 +50,22 @@ class SchedulingAgent {
  public:
   // Constructor
   /*!
-   * Create a new processor
+   * Create a new scheduling agent.
    */
-  SchedulingAgent(std::shared_ptr<core::Repository> repo) {
+  SchedulingAgent(
+      std::shared_ptr<core::controller::ControllerServiceProvider> controller_service_provider,
+      std::shared_ptr<core::Repository> repo, std::shared_ptr<Configure> configuration)
+      : configure_(configuration),
+        admin_yield_duration_(0),
+        bored_yield_duration_(0),
+        controller_service_provider_(controller_service_provider) {
     logger_ = logging::Logger::getLogger();
     running_ = false;
     repo_ = repo;
+    utils::ThreadPool<bool> pool = utils::ThreadPool<bool>(
+        configure_->getInt(Configure::nifi_flow_engine_threads, 8), true);
+    component_lifecycle_thread_pool_ = std::move(pool);
+    component_lifecycle_thread_pool_.start();
   }
   // Destructor
   virtual ~SchedulingAgent() {
@@ -69,13 +82,19 @@ class SchedulingAgent {
   // start
   void start() {
     running_ = true;
+
   }
   // stop
   void stop() {
     running_ = false;
+    component_lifecycle_thread_pool_.shutdown();
   }
 
  public:
+  virtual void enableControllerService(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode);
+  virtual void disableControllerService(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode);
   // schedule, overwritten by different DrivenSchedulingAgent
   virtual void schedule(std::shared_ptr<core::Processor> processor) = 0;
   // unschedule, overwritten by different DrivenSchedulingAgent
@@ -91,11 +110,17 @@ class SchedulingAgent {
   // Whether it is running
   std::atomic<bool> running_;
   // AdministrativeYieldDuration
-  int64_t _administrativeYieldDuration;
+  int64_t admin_yield_duration_;
   // BoredYieldDuration
-  int64_t _boredYieldDuration;
+  int64_t bored_yield_duration_;
+
+  std::shared_ptr<Configure> configure_;
 
   std::shared_ptr<core::Repository> repo_;
+  // thread pool for components.
+  utils::ThreadPool<bool> component_lifecycle_thread_pool_;
+  // controller service provider reference
+  std::shared_ptr<core::controller::ControllerServiceProvider> controller_service_provider_;
 
  private:
   // Prevent default copy constructor and assignment operation

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/ThreadedSchedulingAgent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/ThreadedSchedulingAgent.h b/libminifi/include/ThreadedSchedulingAgent.h
index 044b3c3..bf6f480 100644
--- a/libminifi/include/ThreadedSchedulingAgent.h
+++ b/libminifi/include/ThreadedSchedulingAgent.h
@@ -40,11 +40,13 @@ class ThreadedSchedulingAgent : public SchedulingAgent {
  public:
   // Constructor
   /*!
-   * Create a new processor
+   * Create a new threaded scheduling agent.
    */
-  ThreadedSchedulingAgent(std::shared_ptr<core::Repository> repo, std::shared_ptr<Configure> configure)
-      : SchedulingAgent(repo) {
-       configure_ = configure;
+  ThreadedSchedulingAgent(
+      std::shared_ptr<core::controller::ControllerServiceProvider> controller_service_provider,
+      std::shared_ptr<core::Repository> repo,
+      std::shared_ptr<Configure> configuration)
+      : SchedulingAgent(controller_service_provider, repo, configuration) {
   }
   // Destructor
   virtual ~ThreadedSchedulingAgent() {
@@ -70,7 +72,6 @@ class ThreadedSchedulingAgent : public SchedulingAgent {
   // Only support pass by reference or pointer
   ThreadedSchedulingAgent(const ThreadedSchedulingAgent &parent);
   ThreadedSchedulingAgent &operator=(const ThreadedSchedulingAgent &parent);
-  std::shared_ptr<Configure> configure_;
 };
 
 } /* namespace minifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/TimerDrivenSchedulingAgent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/TimerDrivenSchedulingAgent.h b/libminifi/include/TimerDrivenSchedulingAgent.h
index 31d089b..74096ee 100644
--- a/libminifi/include/TimerDrivenSchedulingAgent.h
+++ b/libminifi/include/TimerDrivenSchedulingAgent.h
@@ -37,8 +37,11 @@ class TimerDrivenSchedulingAgent : public ThreadedSchedulingAgent {
   /*!
    * Create a new processor
    */
-  TimerDrivenSchedulingAgent(std::shared_ptr<core::Repository> repo, std::shared_ptr<Configure> configure)
-      : ThreadedSchedulingAgent(repo, configure) {
+  TimerDrivenSchedulingAgent(
+      std::shared_ptr<core::controller::ControllerServiceProvider> controller_service_provider,
+      std::shared_ptr<core::Repository> repo,
+      std::shared_ptr<Configure> configure)
+      : ThreadedSchedulingAgent(controller_service_provider, repo, configure) {
   }
   //  Destructor
   virtual ~TimerDrivenSchedulingAgent() {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/controllers/SSLContextService.h
----------------------------------------------------------------------
diff --git a/libminifi/include/controllers/SSLContextService.h b/libminifi/include/controllers/SSLContextService.h
new file mode 100644
index 0000000..7b1c5b0
--- /dev/null
+++ b/libminifi/include/controllers/SSLContextService.h
@@ -0,0 +1,174 @@
+/**
+ *
+ * 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.
+ */
+#ifndef LIBMINIFI_INCLUDE_CONTROLLERS_SSLCONTEXTSERVICE_H_
+#define LIBMINIFI_INCLUDE_CONTROLLERS_SSLCONTEXTSERVICE_H_
+
+#include <openssl/err.h>
+#include <openssl/ssl.h>
+#include <iostream>
+#include <memory>
+#include "core/Resource.h"
+#include "utils/StringUtils.h"
+#include "io/validation.h"
+#include "../core/controller/ControllerService.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace controllers {
+
+class SSLContext {
+ public:
+  SSLContext(SSL_CTX *context)
+      : context_(context) {
+
+  }
+  ~SSLContext() {
+    if (context_) {
+      SSL_CTX_free(context_);
+    }
+  }
+ protected:
+  SSL_CTX *context_;
+};
+
+/**
+ * SSLContextService provides a configurable controller service from
+ * which we can provide an SSL Context or component parts that go
+ * into creating one.
+ *
+ * Justification: Abstracts SSL support out of processors into a
+ * configurable controller service.
+ */
+class SSLContextService : public core::controller::ControllerService {
+ public:
+  explicit SSLContextService(const std::string &name, const std::string &id)
+      : ControllerService(name, id),
+        initialized_(false),
+        valid_(false) {
+  }
+
+  explicit SSLContextService(const std::string &name, uuid_t uuid = 0)
+      : ControllerService(name, uuid),
+        initialized_(false),
+        valid_(false) {
+  }
+
+  virtual void initialize();
+
+  std::unique_ptr<SSLContext> createSSLContext();
+
+  const std::string &getCertificateFile();
+
+  const std::string &getPassphrase();
+
+  const std::string &getPassphraseFile();
+
+  const std::string &getPrivateKeyFile();
+
+  const std::string &getCACertificate();
+
+  void yield() {
+
+  }
+
+  bool isRunning() {
+    return getState() == core::controller::ControllerServiceState::ENABLED;
+  }
+
+  bool isWorkAvailable() {
+    return false;
+  }
+
+ bool configure_ssl_context(SSL_CTX *ctx)
+ {
+   if (SSL_CTX_use_certificate_file(ctx, certificate.c_str(), SSL_FILETYPE_PEM)
+         <= 0) {
+       logger_->log_error("Could not create load certificate, error : %s",
+                          std::strerror(errno));
+       return false;
+     }
+     if (!IsNullOrEmpty(passphrase_)) {
+       SSL_CTX_set_default_passwd_cb_userdata(ctx, &passphrase_);
+       SSL_CTX_set_default_passwd_cb(ctx, pemPassWordCb);
+     }
+
+     int retp = SSL_CTX_use_PrivateKey_file(ctx, private_key_.c_str(),
+                                            SSL_FILETYPE_PEM);
+     if (retp != 1) {
+       logger_->log_error("Could not create load private key,%i on %s error : %s",
+                          retp, private_key_, std::strerror(errno));
+       return false;
+     }
+
+     if (!SSL_CTX_check_private_key(ctx)) {
+       logger_->log_error(
+           "Private key does not match the public certificate, error : %s",
+           std::strerror(errno));
+       return false;
+     }
+
+     retp = SSL_CTX_load_verify_locations(ctx, ca_certificate_.c_str(), 0);
+     if (retp == 0) {
+       logger_->log_error("Can not load CA certificate, Exiting, error : %s",
+                          std::strerror(errno));
+       return false;
+     }
+
+     return true;
+ }
+
+ protected:
+
+  static int pemPassWordCb(char *buf, int size, int rwflag, void *userdata) {
+
+    std::string *pass = (std::string*) userdata;
+    if (pass->length() > 0) {
+
+      memset(buf, 0x00, size);
+      memcpy(buf, pass->c_str(), pass->length() - 1);
+
+      return pass->length() - 1;
+    }
+    return 0;
+  }
+
+  virtual void initializeTLS();
+
+  virtual void onEnable();
+
+  std::mutex initialization_mutex_;
+  std::atomic<bool> initialized_;
+  std::atomic<bool> valid_;
+  std::string certificate;
+  std::string private_key_;
+  std::string passphrase_;
+  std::string passphrase_file_;
+  std::string ca_certificate_;
+};
+typedef int (SSLContextService::*ptr)(char *, int, int, void *);
+REGISTER_RESOURCE(SSLContextService);
+
+} /* namespace controllers */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CONTROLLERS_SSLCONTEXTSERVICE_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/ClassLoader.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ClassLoader.h b/libminifi/include/core/ClassLoader.h
new file mode 100644
index 0000000..94c4425
--- /dev/null
+++ b/libminifi/include/core/ClassLoader.h
@@ -0,0 +1,259 @@
+/**
+ * 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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_CORE_CLASSLOADER_H_
+#define LIBMINIFI_INCLUDE_CORE_CLASSLOADER_H_
+
+#include <mutex>
+#include <vector>
+#include <map>
+#include "Connectable.h"
+#include "utils/StringUtils.h"
+#include <dlfcn.h>
+#include "core/Core.h"
+#include "core/logging/Logger.h"
+#include "io/DataStream.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+#define RESOURCE_FAILURE -1
+
+#define RESOURCE_SUCCESS 1
+
+/**
+ * Factory that is used as an interface for
+ * creating processors from shared objects.
+ */
+class ObjectFactory {
+
+ public:
+  /**
+   * Virtual destructor.
+   */
+  virtual ~ObjectFactory() {
+
+  }
+
+  /**
+   * Create a shared pointer to a new processor.
+   */
+  virtual std::shared_ptr<Connectable> create(const std::string &name) {
+    return nullptr;
+  }
+
+  /**
+   * Create a shared pointer to a new processor.
+   */
+  virtual std::shared_ptr<Connectable> create(const std::string &name,
+                                              uuid_t uuid) {
+    return nullptr;
+  }
+
+  /**
+   * Gets the name of the object.
+   * @return class name of processor
+   */
+  virtual std::string getName() = 0;
+
+  /**
+   * Gets the class name for the object
+   * @return class name for the processor.
+   */
+  virtual std::string getClassName() = 0;
+
+};
+
+/**
+ * Factory that is used as an interface for
+ * creating processors from shared objects.
+ */
+template<class T>
+class DefautObjectFactory : public ObjectFactory {
+
+ public:
+
+  DefautObjectFactory() {
+    className = core::getClassName<T>();
+  }
+  /**
+   * Virtual destructor.
+   */
+  virtual ~DefautObjectFactory() {
+
+  }
+
+  /**
+   * Create a shared pointer to a new processor.
+   */
+  virtual std::shared_ptr<Connectable> create(const std::string &name) {
+    std::shared_ptr<T> ptr = std::make_shared<T>(name);
+    return std::static_pointer_cast<Connectable>(ptr);
+  }
+
+  /**
+   * Create a shared pointer to a new processor.
+   */
+  virtual std::shared_ptr<Connectable> create(const std::string &name,
+                                              uuid_t uuid) {
+    std::shared_ptr<T> ptr = std::make_shared<T>(name, uuid);
+    return std::static_pointer_cast<Connectable>(ptr);
+  }
+
+  /**
+   * Gets the name of the object.
+   * @return class name of processor
+   */
+  virtual std::string getName() {
+    return className;
+  }
+
+  /**
+   * Gets the class name for the object
+   * @return class name for the processor.
+   */
+  virtual std::string getClassName() {
+    return className;
+  }
+
+ protected:
+  std::string className;
+
+};
+
+/**
+ * Function that is used to create the
+ * processor factory from the shared object.
+ */
+typedef ObjectFactory* createFactory();
+
+/**
+ * Processor class loader that accepts
+ * a variety of mechanisms to load in shared
+ * objects.
+ */
+class ClassLoader {
+
+ public:
+
+  static ClassLoader &getDefaultClassLoader();
+
+  /**
+   * Constructor.
+   */
+  ClassLoader()
+      : logger_(logging::Logger::getLogger()) {
+
+  }
+
+  ~ClassLoader() {
+    loaded_factories_.clear();
+    for (auto ptr : dl_handles_) {
+      dlclose(ptr);
+    }
+  }
+
+  /**
+   * Register the file system resource.
+   * This will attempt to load objects within this resource.
+   * @return return code: RESOURCE_FAILURE or RESOURCE_SUCCESS
+   */
+  uint16_t registerResource(const std::string &resource);
+
+  /**
+   * Register a class with the give ProcessorFactory
+   */
+  void registerClass(const std::string &name,
+                     std::unique_ptr<ObjectFactory> factory) {
+    if (loaded_factories_.find(name) != loaded_factories_.end()){
+      return;
+    }
+
+    std::lock_guard<std::mutex> lock(internal_mutex_);
+
+
+    loaded_factories_.insert(std::make_pair(name, std::move(factory)));
+  }
+
+  /**
+   * Instantiate object based on class_name
+   * @param class_name class to create
+   * @param uuid uuid of object
+   * @return nullptr or object created from class_name definition.
+   */
+  template<class T = Connectable>
+  std::shared_ptr<T> instantiate(const std::string &class_name,
+                                 const std::string &name);
+
+  /**
+   * Instantiate object based on class_name
+   * @param class_name class to create
+   * @param uuid uuid of object
+   * @return nullptr or object created from class_name definition.
+   */
+  template<class T = Connectable>
+  std::shared_ptr<T> instantiate(const std::string &class_name, uuid_t uuid);
+
+ protected:
+
+  // logger shared ptr
+  std::shared_ptr<org::apache::nifi::minifi::core::logging::Logger> logger_;
+
+  std::map<std::string, std::unique_ptr<ObjectFactory>> loaded_factories_;
+
+  std::mutex internal_mutex_;
+
+  std::vector<void *> dl_handles_;
+
+};
+
+template<class T>
+std::shared_ptr<T> ClassLoader::instantiate(const std::string &class_name,
+                                            const std::string &name) {
+  std::lock_guard<std::mutex> lock(internal_mutex_);
+  auto factory_entry = loaded_factories_.find(class_name);
+  if (factory_entry != loaded_factories_.end()) {
+    auto obj = factory_entry->second->create(name);
+    return std::static_pointer_cast<T>(obj);
+  } else {
+    return nullptr;
+  }
+}
+
+template<class T>
+std::shared_ptr<T> ClassLoader::instantiate(const std::string &class_name,
+                                            uuid_t uuid) {
+  std::lock_guard<std::mutex> lock(internal_mutex_);
+  auto factory_entry = loaded_factories_.find(class_name);
+  if (factory_entry != loaded_factories_.end()) {
+    auto obj = factory_entry->second->create(class_name, uuid);
+    return std::static_pointer_cast<T>(obj);
+  } else {
+    return nullptr;
+  }
+}
+
+}/* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_CLASSLOADER_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/ConfigurableComponent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ConfigurableComponent.h b/libminifi/include/core/ConfigurableComponent.h
index d46216b..bf886e8 100644
--- a/libminifi/include/core/ConfigurableComponent.h
+++ b/libminifi/include/core/ConfigurableComponent.h
@@ -54,6 +54,11 @@ class ConfigurableComponent {
    * @return result of getting property.
    */
   bool getProperty(const std::string name, std::string &value);
+
+  /**
+   * Provides a reference for the property.
+   */
+  bool getProperty(const std::string &name, Property &prop);
   /**
    * Sets the property using the provided name
    * @param property name
@@ -61,6 +66,12 @@ class ConfigurableComponent {
    * @return result of setting property.
    */
   bool setProperty(const std::string name, std::string value);
+
+  /**
+   * Updates the Property from the key (name), adding value
+   * to the collection of values within the Property.
+   */
+  bool updateProperty(const std::string &name, const std::string &value);
   /**
    * Sets the property using the provided name
    * @param property name

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/ConfigurationFactory.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ConfigurationFactory.h b/libminifi/include/core/ConfigurationFactory.h
index bf631d4..ed0bdb5 100644
--- a/libminifi/include/core/ConfigurationFactory.h
+++ b/libminifi/include/core/ConfigurationFactory.h
@@ -31,7 +31,8 @@ namespace core {
 template<typename T>
 typename std::enable_if<!class_operations<T>::value, T*>::type instantiate(
     const std::shared_ptr<core::Repository> &repo,
-    const std::shared_ptr<core::Repository> &flow_file_repo, const std::string path) {
+    const std::shared_ptr<core::Repository> &flow_file_repo,
+    std::shared_ptr<Configure> configuration, const std::string path) {
   throw std::runtime_error("Cannot instantiate class");
 }
 
@@ -40,8 +41,8 @@ typename std::enable_if<class_operations<T>::value, T*>::type instantiate(
     const std::shared_ptr<core::Repository> &repo,
     const std::shared_ptr<core::Repository> &flow_file_repo,
     const std::shared_ptr<io::StreamFactory> &stream_factory,
-    const std::string path) {
-  return new T(repo, flow_file_repo, stream_factory, path);
+    std::shared_ptr<Configure> configuration, const std::string path) {
+  return new T(repo, flow_file_repo, stream_factory, configuration, path);
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/Connectable.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Connectable.h b/libminifi/include/core/Connectable.h
index f7e425e..f690533 100644
--- a/libminifi/include/core/Connectable.h
+++ b/libminifi/include/core/Connectable.h
@@ -120,6 +120,16 @@ class Connectable : public CoreComponent {
    */
   virtual bool isWorkAvailable() = 0;
 
+  /**
+   * Verify that this connectable can be stopped.
+   * @return bool.
+   */
+  virtual bool verifyCanStop() {
+    if (isRunning())
+      return true;
+    return false;
+  }
+
  protected:
 
   // Penalization Period in MilliSecond
@@ -137,7 +147,7 @@ class Connectable : public CoreComponent {
   // Incoming connections
   std::set<std::shared_ptr<Connectable>> _incomingConnections;
   // Outgoing connections map based on Relationship name
-  std::map<std::string, std::set<std::shared_ptr<Connectable>>>_outGoingConnections;
+  std::map<std::string, std::set<std::shared_ptr<Connectable>>> out_going_connections_;
 
   // Mutex for protection
   std::mutex relationship_mutex_;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/Core.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Core.h b/libminifi/include/core/Core.h
index 1010be7..453a6a5 100644
--- a/libminifi/include/core/Core.h
+++ b/libminifi/include/core/Core.h
@@ -60,7 +60,7 @@ struct class_operations {
 
   typedef decltype(canDestruct<T>(0)) type;
 
-  static const bool value = type::value; /* Which is it? */
+  static const bool value = type::value;
 };
 
 template<typename T>
@@ -99,6 +99,7 @@ class CoreComponent {
     uuidStr_ = uuidStr;
   }
 
+
   /**
    * Move Constructor.
    */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/FlowConfiguration.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/FlowConfiguration.h b/libminifi/include/core/FlowConfiguration.h
index 96f0b4a..2e704b5 100644
--- a/libminifi/include/core/FlowConfiguration.h
+++ b/libminifi/include/core/FlowConfiguration.h
@@ -21,6 +21,8 @@
 #include "core/Core.h"
 #include "Connection.h"
 #include "RemoteProcessorGroupPort.h"
+#include "core/controller/ControllerServiceNode.h"
+#include "core/controller/StandardControllerServiceProvider.h"
 #include "provenance/Provenance.h"
 #include "core/reporting/SiteToSiteProvenanceReportingTask.h"
 #include "processors/GetFile.h"
@@ -56,14 +58,19 @@ class FlowConfiguration : public CoreComponent {
    * Constructor that will be used for configuring
    * the flow controller.
    */
-  FlowConfiguration(std::shared_ptr<core::Repository> repo,
-                    std::shared_ptr<core::Repository> flow_file_repo,
-                    std::shared_ptr<io::StreamFactory> stream_factory,
-                    const std::string path)
+  explicit FlowConfiguration(std::shared_ptr<core::Repository> repo,
+                             std::shared_ptr<core::Repository> flow_file_repo,
+                             std::shared_ptr<io::StreamFactory> stream_factory,
+                             std::shared_ptr<Configure> configuration,
+                             const std::string path)
       : CoreComponent(core::getClassName<FlowConfiguration>()),
         flow_file_repo_(flow_file_repo),
         config_path_(path) {
-
+    controller_services_ = std::make_shared<
+        core::controller::ControllerServiceMap>();
+    service_provider_ = std::make_shared<
+        core::controller::StandardControllerServiceProvider>(
+        controller_services_, nullptr, configuration);
   }
 
   virtual ~FlowConfiguration();
@@ -74,6 +81,10 @@ class FlowConfiguration : public CoreComponent {
   // Create Root Processor Group
   std::unique_ptr<core::ProcessGroup> createRootProcessGroup(std::string name,
                                                              uuid_t uuid);
+
+  std::shared_ptr<core::controller::ControllerServiceNode> createControllerService(
+      const std::string &class_name, const std::string &name, uuid_t uuid);
+
   // Create Remote Processor Group
   std::unique_ptr<core::ProcessGroup> createRemoteProcessGroup(std::string name,
                                                                uuid_t uuid);
@@ -105,11 +116,21 @@ class FlowConfiguration : public CoreComponent {
     return nullptr;
   }
 
+  std::shared_ptr<core::controller::StandardControllerServiceProvider> &getControllerServiceProvider() {
+    return service_provider_;
+  }
+
  protected:
+
+  // service provider reference.
+  std::shared_ptr<core::controller::StandardControllerServiceProvider> service_provider_;
+  // based, shared controller service map.
+  std::shared_ptr<core::controller::ControllerServiceMap> controller_services_;
   // configuration path
   std::string config_path_;
   // flow file repo
   std::shared_ptr<core::Repository> flow_file_repo_;
+  // stream factory
   std::shared_ptr<io::StreamFactory> stream_factory_;
 };
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/ProcessContext.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ProcessContext.h b/libminifi/include/core/ProcessContext.h
index 1da85cd..79b7704 100644
--- a/libminifi/include/core/ProcessContext.h
+++ b/libminifi/include/core/ProcessContext.h
@@ -27,6 +27,8 @@
 #include <algorithm>
 
 #include "Property.h"
+#include "core/controller/ControllerServiceProvider.h"
+#include "core/controller/ControllerServiceLookup.h"
 #include "core/logging/Logger.h"
 #include "ProcessorNode.h"
 #include "core/Repository.h"
@@ -38,15 +40,18 @@ namespace minifi {
 namespace core {
 
 // ProcessContext Class
-class ProcessContext {
+class ProcessContext : public controller::ControllerServiceLookup {
  public:
   // Constructor
   /*!
    * Create a new process context associated with the processor/controller service/state manager
    */
-  ProcessContext(ProcessorNode &processor,
-                 std::shared_ptr<core::Repository> repo)
-      : processor_node_(processor) {
+  ProcessContext(
+      ProcessorNode &processor,
+      std::shared_ptr<controller::ControllerServiceProvider> &controller_service_provider,
+      std::shared_ptr<core::Repository> repo)
+      : processor_node_(processor),
+        controller_service_provider_(controller_service_provider) {
     logger_ = logging::Logger::getLogger();
     repo_ = repo;
   }
@@ -95,8 +100,54 @@ class ProcessContext {
   ProcessContext(const ProcessContext &parent) = delete;
   ProcessContext &operator=(const ProcessContext &parent) = delete;
 
+  // controller services
+
+  /**
+   * @param identifier of controller service
+   * @return the ControllerService that is registered with the given
+   * identifier
+   */
+  std::shared_ptr<core::controller::ControllerService> getControllerService(
+      const std::string &identifier) {
+    return controller_service_provider_->getControllerServiceForComponent(
+        identifier, processor_node_.getUUIDStr());
+  }
+
+  /**
+   * @param identifier identifier of service to check
+   * @return <code>true</code> if the Controller Service with the given
+   * identifier is enabled, <code>false</code> otherwise. If the given
+   * identifier is not known by this ControllerServiceLookup, returns
+   * <code>false</code>
+   */
+  bool isControllerServiceEnabled(const std::string &identifier) {
+    return controller_service_provider_->isControllerServiceEnabled(identifier);
+  }
+
+  /**
+   * @param identifier identifier of service to check
+   * @return <code>true</code> if the Controller Service with the given
+   * identifier has been enabled but is still in the transitioning state,
+   * otherwise returns <code>false</code>. If the given identifier is not
+   * known by this ControllerServiceLookup, returns <code>false</code>
+   */
+  bool isControllerServiceEnabling(const std::string &identifier) {
+    return controller_service_provider_->isControllerServiceEnabling(identifier);
+  }
+
+  /**
+   * @param identifier identifier to look up
+   * @return the name of the Controller service with the given identifier. If
+   * no service can be found with this identifier, returns {@code null}
+   */
+  const std::string getControllerServiceName(const std::string &identifier) {
+    return controller_service_provider_->getControllerServiceName(identifier);
+  }
+
  private:
 
+  // controller service provider.
+  std::shared_ptr<controller::ControllerServiceProvider> controller_service_provider_;
   // repository shared pointer.
   std::shared_ptr<core::Repository> repo_;
   // Processor

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/ProcessGroup.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ProcessGroup.h b/libminifi/include/core/ProcessGroup.h
index 75bb0ba..f2f9a63 100644
--- a/libminifi/include/core/ProcessGroup.h
+++ b/libminifi/include/core/ProcessGroup.h
@@ -32,6 +32,8 @@
 #include "TimerDrivenSchedulingAgent.h"
 #include "EventDrivenSchedulingAgent.h"
 #include "core/logging/Logger.h"
+#include "controller/ControllerServiceNode.h"
+#include "controller/ControllerServiceMap.h"
 
 namespace org {
 namespace apache {
@@ -140,6 +142,23 @@ class ProcessGroup {
   std::shared_ptr<Processor> findProcessor(uuid_t uuid);
   // findProcessor based on name
   std::shared_ptr<Processor> findProcessor(const std::string &processorName);
+  /**
+   * Add controller service
+   * @param nodeId node identifier
+   * @param node controller service node.
+   */
+  void addControllerService(
+      const std::string &nodeId,
+      std::shared_ptr<core::controller::ControllerServiceNode> &node);
+
+  /**
+   * Find controllerservice node will search child groups until the nodeId is found.
+   * @param node node identifier
+   * @return controller service node, if it exists.
+   */
+  std::shared_ptr<core::controller::ControllerServiceNode> findControllerService(
+      const std::string &nodeId);
+
   // removeConnection
   void removeConnection(std::shared_ptr<Connection> connection);
   // update property value
@@ -171,6 +190,10 @@ class ProcessGroup {
   // Transmitting
   std::atomic<bool> transmitting_;
 
+  // controller services
+
+  core::controller::ControllerServiceMap controller_service_map_;
+
  private:
 
   // Mutex for protection

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/Property.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Property.h b/libminifi/include/core/Property.h
index c681449..fd940c5 100644
--- a/libminifi/include/core/Property.h
+++ b/libminifi/include/core/Property.h
@@ -30,6 +30,7 @@
 #include <set>
 #include <stdlib.h>
 #include <math.h>
+#include "utils/StringUtils.h"
 
 namespace org {
 namespace apache {
@@ -58,11 +59,24 @@ class Property {
   Property(const std::string name, const std::string description,
            const std::string value)
       : name_(name),
-        description_(description),
-        value_(value) {
+        isCollection(false),
+        description_(description) {
+    values_.push_back(std::string(value.c_str()));
   }
-  Property() {
+
+  Property(const std::string name, const std::string description)
+      : name_(name),
+        isCollection(true),
+        description_(description) {
+  }
+
+  Property()
+      : isCollection(false),
+        name_(""),
+        description_("") {
+
   }
+
   // Destructor
   virtual ~Property() {
   }
@@ -72,8 +86,14 @@ class Property {
   std::string getDescription();
   // Get value for the property
   std::string getValue() const;
+  std::vector<std::string> &getValues();
+
   // Set value for the property
   void setValue(std::string value);
+  /**
+   * Add value to the collection of values.
+   */
+  void addValue(const std::string &value);
   const Property &operator=(const Property &other);
   // Compare
   bool operator <(const Property & right) const;
@@ -244,12 +264,13 @@ class Property {
   }
 
  protected:
+  bool isCollection;
   // Name
   std::string name_;
   // Description
   std::string description_;
   // Value
-  std::string value_;
+  std::vector<std::string> values_;
 
  private:
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/Resource.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Resource.h b/libminifi/include/core/Resource.h
new file mode 100644
index 0000000..8f110a1
--- /dev/null
+++ b/libminifi/include/core/Resource.h
@@ -0,0 +1,54 @@
+/**
+ *
+ * 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.
+ */
+#ifndef LIBMINIFI_INCLUDE_CORE_RESOURCE_H_
+#define LIBMINIFI_INCLUDE_CORE_RESOURCE_H_
+
+#include "ClassLoader.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+template<class T>
+class StaticClassType {
+ public:
+
+  StaticClassType(const std::string &name) {
+    // Notify when the static member is created
+    ClassLoader::getDefaultClassLoader().registerClass(
+        name, std::unique_ptr<ObjectFactory>(new DefautObjectFactory<T>()));
+  }
+};
+
+#define REGISTER_RESOURCE(CLASSNAME) \
+        static core::StaticClassType<CLASSNAME> \
+        CLASSNAME##_registrar( #CLASSNAME );
+
+#define REGISTER_RESOURCE_AS(CLASSNAME,NAME) \
+        static core::StaticClassType<CLASSNAME> \
+        CLASSNAME##_registrar( #NAME );
+
+}/* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_RESOURCE_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/controller/ControllerService.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/controller/ControllerService.h b/libminifi/include/core/controller/ControllerService.h
new file mode 100644
index 0000000..86c8f7a
--- /dev/null
+++ b/libminifi/include/core/controller/ControllerService.h
@@ -0,0 +1,142 @@
+/**
+ *
+ * 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.
+ */
+#ifndef LIBMINIFI_INCLUDE_CORE_CONTROLLER_CONTROLLERSERVICE_H_
+#define LIBMINIFI_INCLUDE_CORE_CONTROLLER_CONTROLLERSERVICE_H_
+
+#include <set>
+#include "properties/Configure.h"
+#include "core/Core.h"
+#include "core/ConfigurableComponent.h"
+#include "core/Connectable.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace controller {
+
+enum ControllerServiceState {
+  /**
+   * Controller Service is disabled and cannot be used.
+   */
+  DISABLED,
+  /**
+   * Controller Service is in the process of being disabled.
+   */
+  DISABLING,
+  /**
+   * Controller Service is being enabled.
+   */
+  ENABLING,
+  /**
+   * Controller Service is enabled.
+   */
+  ENABLED
+};
+
+/**
+ * Controller Service base class that contains some pure virtual methods.
+ *
+ * Design: OnEnable is executed when the controller service is being enabled.
+ * Note that keeping state here must be protected  in this function.
+ */
+class ControllerService : public ConfigurableComponent, public Connectable {
+ public:
+
+  /**
+   * Controller Service constructor.
+   */
+  explicit ControllerService()
+      : Connectable(core::getClassName<ControllerService>(), 0),
+        ConfigurableComponent(logging::Logger::getLogger()),
+        configuration_(std::make_shared<Configure>()) {
+    current_state_ = DISABLED;
+  }
+
+  /**
+   * Controller Service constructor.
+   */
+  explicit ControllerService(const std::string &name, const std::string &id)
+      : Connectable(name, 0),
+        ConfigurableComponent(logging::Logger::getLogger()),
+        configuration_(std::make_shared<Configure>()) {
+    current_state_ = DISABLED;
+    uuid_parse(id.c_str(), uuid_);
+    char uuidStr[37];
+    uuid_unparse_lower(uuid_, uuidStr);
+    uuidStr_ = uuidStr;
+  }
+
+  /**
+   * Controller Service constructor.
+   */
+  explicit ControllerService(const std::string &name, uuid_t uuid)
+      : Connectable(name, uuid),
+        ConfigurableComponent(logging::Logger::getLogger()),
+        configuration_(std::make_shared<Configure>()) {
+    current_state_ = DISABLED;
+  }
+
+  virtual void initialize() {
+    // set base supported properties
+    Property property("Linked Services", "Referenced Controller Services");
+    std::set<Property> supportedProperties;
+    supportedProperties.insert(property);
+    setSupportedProperties(supportedProperties);
+    current_state_ = ENABLED;
+  }
+
+  /**
+   * Replaces the configuration object within the controller service.
+   */
+  void setConfiguration(const std::shared_ptr<Configure> &configuration) {
+    configuration_ = configuration;
+  }
+
+  ControllerServiceState getState() {
+    return current_state_.load();
+  }
+
+  /**
+   * Function is called when Controller Services are enabled and being run
+   */
+  virtual void onEnable() {
+
+  }
+
+  void setState(ControllerServiceState state) {
+    current_state_ = state;
+  }
+ protected:
+
+  std::shared_ptr<Configure> configuration_;
+  std::atomic<ControllerServiceState> current_state_;
+  virtual bool canEdit() {
+    return true;
+  }
+};
+
+} /* namespace controller */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_CONTROLLER_CONTROLLERSERVICE_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/controller/ControllerServiceLookup.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/controller/ControllerServiceLookup.h b/libminifi/include/core/controller/ControllerServiceLookup.h
new file mode 100644
index 0000000..6f23e34
--- /dev/null
+++ b/libminifi/include/core/controller/ControllerServiceLookup.h
@@ -0,0 +1,89 @@
+/**
+ *
+ * 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.
+ */
+#ifndef LIBMINIFI_INCLUDE_CORE_CONTROLLER_CONTROLLERSERVICELOOKUP_H_
+#define LIBMINIFI_INCLUDE_CORE_CONTROLLER_CONTROLLERSERVICELOOKUP_H_
+
+#include <map>
+#include "core/Core.h"
+#include "core/ConfigurableComponent.h"
+#include "ControllerService.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace controller {
+
+/**
+ * Controller Service Lookup pure virtual class.
+ *
+ * Purpose: Provide a mechanism that controllers can lookup information about
+ * controller services.
+ *
+ */
+class ControllerServiceLookup {
+ public:
+
+  ControllerServiceLookup() {
+
+  }
+
+  virtual ~ControllerServiceLookup() {
+
+  }
+
+  /**
+   * Gets the controller service via the provided identifier.
+   * @param identifier reference string for controller service.
+   * @return controller service reference.
+   */
+  virtual std::shared_ptr<ControllerService> getControllerService(
+      const std::string &identifier) = 0;
+
+  /**
+   * Detects if controller service is enabled.
+   * @param identifier reference string for controller service.
+   * @return true if controller service is enabled.
+   */
+  virtual bool isControllerServiceEnabled(const std::string &identifier) = 0;
+
+  /**
+   * Detects if controller service is being enabled.
+   * @param identifier reference string for controller service.
+   * @return true if controller service is enabled.
+   */
+  virtual bool isControllerServiceEnabling(const std::string &identifier) = 0;
+
+  /**
+   * Gets the controller service name for the provided reference identifier
+   * @param identifier reference string for the controller service.
+   */
+  virtual const std::string getControllerServiceName(
+      const std::string &identifier) = 0;
+
+};
+
+} /* namespace controller */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_CONTROLLER_CONTROLLERSERVICELOOKUP_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/controller/ControllerServiceMap.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/controller/ControllerServiceMap.h b/libminifi/include/core/controller/ControllerServiceMap.h
new file mode 100644
index 0000000..f1cc2cf
--- /dev/null
+++ b/libminifi/include/core/controller/ControllerServiceMap.h
@@ -0,0 +1,120 @@
+/**
+ *
+ * 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.
+ */
+#ifndef LIBMINIFI_INCLUDE_CORE_CONTROLLER_CONTROLLERSERVICEMAP_H_
+#define LIBMINIFI_INCLUDE_CORE_CONTROLLER_CONTROLLERSERVICEMAP_H_
+
+#include <map>
+#include <string>
+#include "ControllerServiceNode.h"
+#include "io/validation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace controller {
+
+/**
+ * Purpose: Controller service map is the mapping between service names
+ * and ControllerService Nodes.
+ * Justification: This abstracts the map, the controller for the map, and the
+ * accounting into an object that will be used amongst the separate Controller
+ * Service classes. This will help avoid help when sending the map as a reference.
+ */
+class ControllerServiceMap {
+ public:
+
+  ControllerServiceMap() {
+  }
+
+  virtual ~ControllerServiceMap() {
+  }
+
+  /**
+   * Gets the controller service node using the <code>id</code>
+   * @param id identifier for controller service.
+   * @return nullptr if node does not exist or controller service node shared pointer.
+   */
+  virtual std::shared_ptr<ControllerServiceNode> getControllerServiceNode(
+      const std::string &id) {
+    std::lock_guard<std::mutex> lock(mutex_);
+    auto exists = controller_services_.find(id);
+    if (exists != controller_services_.end())
+      return exists->second;
+    else
+      return nullptr;
+  }
+
+  /**
+   * Removes the controller service.
+   * @param serviceNode service node to remove
+   *
+   */
+  virtual bool removeControllerService(
+      const std::shared_ptr<ControllerServiceNode> &serviceNode) {
+    if (IsNullOrEmpty(serviceNode.get()))
+      return false;
+    std::lock_guard<std::mutex> lock(mutex_);
+    controller_services_[serviceNode->getName()] = nullptr;
+    controller_services_list_.erase(serviceNode);
+    return true;
+  }
+
+  /**
+   * Puts the service node into the mapping using <code>id</code> as the identifier
+   * @param id service identifier
+   * @param serviceNode controller service node shared pointer.
+   *
+   */
+  virtual bool put(const std::string &id,
+                   const std::shared_ptr<ControllerServiceNode> &serviceNode) {
+    if (IsNullOrEmpty(id) || IsNullOrEmpty(serviceNode.get()))
+      return false;
+    std::lock_guard<std::mutex> lock(mutex_);
+    controller_services_[id] = serviceNode;
+    controller_services_list_.insert(serviceNode);
+    return true;
+  }
+
+  /**
+   * Gets all controller services.
+   * @return controller service node shared pointers.
+   */
+  std::vector<std::shared_ptr<ControllerServiceNode>> getAllControllerServices() {
+    std::lock_guard<std::mutex> lock(mutex_);
+    return std::vector<std::shared_ptr<ControllerServiceNode>>(
+        controller_services_list_.begin(), controller_services_list_.end());
+  }
+
+  ControllerServiceMap(const ControllerServiceMap &other) = delete;
+
+ protected:
+  std::mutex mutex_;
+  std::set<std::shared_ptr<ControllerServiceNode>> controller_services_list_;
+  std::map<std::string, std::shared_ptr<ControllerServiceNode>> controller_services_;
+};
+
+} /* namespace controller */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_CONTROLLER_CONTROLLERSERVICEMAP_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/controller/ControllerServiceNode.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/controller/ControllerServiceNode.h b/libminifi/include/core/controller/ControllerServiceNode.h
new file mode 100644
index 0000000..55913cb
--- /dev/null
+++ b/libminifi/include/core/controller/ControllerServiceNode.h
@@ -0,0 +1,133 @@
+/**
+ *
+ * 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.
+ */
+#ifndef LIBMINIFI_INCLUDE_CORE_CONTROLLER_CONTROLLERSERVICENODE_H_
+#define LIBMINIFI_INCLUDE_CORE_CONTROLLER_CONTROLLERSERVICENODE_H_
+
+#include "core/Core.h"
+#include "core/ConfigurableComponent.h"
+#include "core/logging/Logger.h"
+#include "properties/Configure.h"
+#include "ControllerService.h"
+#include "io/validation.h"
+#include "Exception.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace controller {
+
+class ControllerServiceNode : public CoreComponent, public ConfigurableComponent {
+ public:
+
+  /**
+   * Constructor for the controller service node.
+   * @param service controller service reference
+   * @param id identifier for this node.
+   * @param configuration shared pointer configuration.
+   */
+  explicit ControllerServiceNode(std::shared_ptr<ControllerService> service,
+                        const std::string &id, std::shared_ptr<Configure> configuration)
+      : CoreComponent(id),
+        ConfigurableComponent(logging::Logger::getLogger()),
+        controller_service_(service),
+        configuration_(configuration),
+        active(false) {
+    if (service == nullptr || IsNullOrEmpty(service.get())) {
+      throw Exception(GENERAL_EXCEPTION, "Service must be properly configured");
+    }
+    if (IsNullOrEmpty(configuration)) {
+      throw Exception(GENERAL_EXCEPTION,
+                      "Configuration must be properly configured");
+    }
+    service->setConfiguration(configuration);
+  }
+
+  virtual void initialize() {
+    controller_service_->initialize();
+    // set base supported properties
+    Property property("Linked Services", "Referenced Controller Services");
+    std::set<Property> supportedProperties;
+    supportedProperties.insert(property);
+    setSupportedProperties(supportedProperties);
+  }
+  void setName(const std::string name) {
+    CoreComponent::setName(name);
+    controller_service_->setName(name);
+  }
+
+  void setUUID(uuid_t uuid) {
+    CoreComponent::setUUID(uuid);
+    controller_service_->setUUID(uuid);
+  }
+
+  /**
+   * Returns the implementation of the Controller Service that this ControllerServiceNode
+   * maintains
+   * @return the implementation of the Controller Service
+   */
+  std::shared_ptr<ControllerService> &getControllerServiceImplementation();
+  std::vector<std::shared_ptr<ControllerServiceNode> > &getLinkedControllerServices();
+  std::vector<std::shared_ptr<ConfigurableComponent> > &getLinkedComponents();
+
+  /**
+   * Returns true if we can be enabled.
+   * Returns false if this ControllerServiceNode cannot be enabled.
+   */
+  virtual bool canEnable()=0;
+
+  virtual bool enabled() {
+    return active.load();
+  }
+
+  /**
+   * Function to enable the controller service node.
+   */
+  virtual bool enable() = 0;
+
+  /**
+   * Function to disable the controller service node.
+   */
+  virtual bool disable() = 0;
+
+  ControllerServiceNode(const ControllerServiceNode &other) = delete;
+  ControllerServiceNode &operator=(const ControllerServiceNode &parent) = delete;
+ protected:
+
+  bool canEdit() {
+    return true;
+  }
+
+  std::atomic<bool> active;
+  std::shared_ptr<Configure> configuration_;
+  // controller service.
+  std::shared_ptr<ControllerService> controller_service_;
+  // linked controller services.
+  std::vector<std::shared_ptr<ControllerServiceNode> > linked_controller_services_;
+  std::vector<std::shared_ptr<ConfigurableComponent> > linked_components_;
+};
+
+} /* namespace controller */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_CONTROLLER_CONTROLLERSERVICENODE_H_ */


[4/5] nifi-minifi-cpp git commit: MINIFI-226: Add controller services capabilities along with unit tests Fix test failures Update Travis YML Update readme to link to MiNiFi licensing information

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/controller/ControllerServiceProvider.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/controller/ControllerServiceProvider.h b/libminifi/include/core/controller/ControllerServiceProvider.h
new file mode 100644
index 0000000..a749475
--- /dev/null
+++ b/libminifi/include/core/controller/ControllerServiceProvider.h
@@ -0,0 +1,306 @@
+/**
+ *
+ * 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.
+ */
+#ifndef LIBMINIFI_INCLUDE_CORE_CONTROLLER_CONTROLLERSERVICEPROVIDER_H_
+#define LIBMINIFI_INCLUDE_CORE_CONTROLLER_CONTROLLERSERVICEPROVIDER_H_
+
+#include <vector>
+#include "core/Core.h"
+#include "ControllerServiceLookup.h"
+#include "core/ConfigurableComponent.h"
+#include "ControllerServiceNode.h"
+#include "ControllerServiceMap.h"
+#include "core/ClassLoader.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace controller {
+
+class ControllerServiceProvider : public CoreComponent,
+    public ConfigurableComponent, public ControllerServiceLookup {
+ public:
+
+  explicit ControllerServiceProvider(const std::string &name)
+      : CoreComponent(name),
+        ConfigurableComponent(logging::Logger::getLogger()) {
+    controller_map_ = std::make_shared<ControllerServiceMap>();
+  }
+
+  explicit ControllerServiceProvider(
+      std::shared_ptr<ControllerServiceMap> services)
+      : CoreComponent(core::getClassName<ControllerServiceProvider>()),
+        ConfigurableComponent(logging::Logger::getLogger()),
+        controller_map_(services) {
+  }
+
+  explicit ControllerServiceProvider(
+      const std::string &name, std::shared_ptr<ControllerServiceMap> services)
+      : CoreComponent(name),
+        ConfigurableComponent(logging::Logger::getLogger()),
+        controller_map_(services) {
+  }
+
+  explicit ControllerServiceProvider(const ControllerServiceProvider &&other)
+      : CoreComponent(std::move(other)),
+        ConfigurableComponent(std::move(other)),
+        controller_map_(std::move(other.controller_map_)) {
+  }
+
+  virtual ~ControllerServiceProvider() {
+  }
+
+  /**
+   * Creates a controller service node wrapping the controller service
+   *
+   * @param type service type.
+   * @param id controller service identifier.
+   * @return shared pointer to the controller service node.
+   */
+  virtual std::shared_ptr<ControllerServiceNode> createControllerService(
+      const std::string &type, const std::string &id,
+      bool firstTimeAdded) = 0;
+  /**
+   * Gets a controller service node wrapping the controller service
+   *
+   * @param type service type.
+   * @param id controller service identifier.
+   * @return shared pointer to the controller service node.
+   */
+  virtual std::shared_ptr<ControllerServiceNode> getControllerServiceNode(
+      const std::string &id) {
+    return controller_map_->getControllerServiceNode(id);
+  }
+
+  /**
+   * Removes a controller service.
+   * @param serviceNode controller service node.
+   */
+  virtual void removeControllerService(
+      const std::shared_ptr<ControllerServiceNode> &serviceNode) {
+    controller_map_->removeControllerService(serviceNode);
+  }
+
+  /**
+   * Enables the provided controller service
+   * @param serviceNode controller service node.
+   */
+  virtual void enableControllerService(
+      std::shared_ptr<ControllerServiceNode> &serviceNode) = 0;
+
+  /**
+   * Enables the provided controller service nodes
+   * @param serviceNode controller service node.
+   */
+  virtual void enableControllerServices(
+      std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> serviceNodes) = 0;
+
+  /**
+   * Disables the provided controller service node
+   * @param serviceNode controller service node.
+   */
+  virtual void disableControllerService(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) = 0;
+
+  /**
+   * Gets a list of all controller services.
+   */
+  virtual std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> getAllControllerServices() {
+    return controller_map_->getAllControllerServices();
+  }
+
+  /**
+   * Verifies that referencing components can be stopped for the controller service
+   */
+  virtual void verifyCanStopReferencingComponents(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) = 0;
+
+  /**
+   *  Unschedules referencing components.
+   */
+  virtual std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> unscheduleReferencingComponents(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) = 0;
+
+  /**
+   * Verifies referencing components for <code>serviceNode</code> can be disabled.
+   * @param serviceNode shared pointer to a controller service node.
+   */
+  virtual void verifyCanDisableReferencingServices(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) = 0;
+
+  /**
+   * Disables referencing components for <code>serviceNode</code> can be disabled.
+   * @param serviceNode shared pointer to a controller service node.
+   */
+  virtual std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> disableReferencingServices(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+    return std::vector<std::shared_ptr<core::controller::ControllerServiceNode>>();
+  }
+
+  /**
+   * Verifies referencing components for <code>serviceNode</code> can be enabled.
+   * @param serviceNode shared pointer to a controller service node.
+   */
+  virtual void verifyCanEnableReferencingServices(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+    std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> references =
+        findLinkedComponents(serviceNode);
+    for (auto ref : references) {
+      ref->canEnable();
+    }
+  }
+
+  /**
+   * Enables referencing components for <code>serviceNode</code> can be Enabled.
+   * @param serviceNode shared pointer to a controller service node.
+   */
+  virtual std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> enableReferencingServices(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) = 0;
+
+  /**
+   * Schedules the service node and referencing components.
+   * @param serviceNode shared pointer to a controller service node.
+   */
+  virtual std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> scheduleReferencingComponents(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) = 0;
+
+  /**
+   * Returns a controller service for the service identifier and componentID
+   * @param service Identifier service identifier.
+   */
+  virtual std::shared_ptr<ControllerService> getControllerServiceForComponent(
+      const std::string &serviceIdentifier, const std::string &componentId) {
+    std::shared_ptr<ControllerService> node = getControllerService(
+        serviceIdentifier);
+    return node;
+  }
+
+  /**
+   * Gets the controller service for the provided identifier
+   * @param identifier service identifier.
+   */
+  virtual std::shared_ptr<ControllerService> getControllerService(
+      const std::string &identifier);
+
+  /**
+   * Determines if Controller service is enabled.
+   * @param identifier service identifier.
+   */
+  virtual bool isControllerServiceEnabled(const std::string &identifier) {
+    std::shared_ptr<ControllerServiceNode> node = getControllerServiceNode(
+        identifier);
+    if (nullptr != node) {
+      return linkedServicesAre(ENABLED, node);
+    } else
+      return false;
+  }
+
+  /**
+   * Determines if Controller service is being enabled.
+   * @param identifier service identifier.
+   */
+  virtual bool isControllerServiceEnabling(const std::string &identifier) {
+    std::shared_ptr<ControllerServiceNode> node = getControllerServiceNode(
+        identifier);
+    if (nullptr != node) {
+      return linkedServicesAre(ENABLING, node);
+    } else
+      return false;
+  }
+
+  virtual const std::string getControllerServiceName(
+      const std::string &identifier) {
+    std::shared_ptr<ControllerService> node = getControllerService(identifier);
+    if (nullptr != node) {
+      return node->getName();
+    } else
+      return "";
+  }
+
+  virtual void enableAllControllerServices() = 0;
+
+ protected:
+
+  /**
+   * verifies that linked services match the provided state.
+   */
+  inline bool linkedServicesAre(
+      ControllerServiceState state,
+      const std::shared_ptr<ControllerServiceNode> &node) {
+    if (node->getControllerServiceImplementation()->getState() == state) {
+      for (auto child_service : node->getLinkedControllerServices()) {
+        if (child_service->getControllerServiceImplementation()->getState()
+            != state) {
+          return false;
+        }
+      }
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  bool canEdit() {
+    return true;
+  }
+
+  /**
+   * Finds linked components
+   * @param referenceNode reference node from whcih we will find linked references.
+   */
+  std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> findLinkedComponents(
+      std::shared_ptr<core::controller::ControllerServiceNode> &referenceNode) {
+
+    std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> references;
+
+    for (std::shared_ptr<core::controller::ControllerServiceNode> linked_node : referenceNode
+        ->getLinkedControllerServices()) {
+      references.push_back(linked_node);
+      std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> linked_references =
+          findLinkedComponents(linked_node);
+
+      auto removal_predicate =
+          [&linked_references](std::shared_ptr<core::controller::ControllerServiceNode> key) ->bool
+          {
+            return std::find(linked_references.begin(), linked_references.end(), key) != linked_references.end();
+          };
+
+      references.erase(
+          std::remove_if(references.begin(), references.end(),
+                         removal_predicate),
+          references.end());
+
+      references.insert(std::end(references), linked_references.begin(),
+                        linked_references.end());
+    }
+    return references;
+  }
+
+  std::shared_ptr<ControllerServiceMap> controller_map_;
+
+};
+
+} /* namespace controller */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_CONTROLLER_CONTROLLERSERVICEPROVIDER_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/controller/StandardControllerServiceNode.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/controller/StandardControllerServiceNode.h b/libminifi/include/core/controller/StandardControllerServiceNode.h
new file mode 100644
index 0000000..f599217
--- /dev/null
+++ b/libminifi/include/core/controller/StandardControllerServiceNode.h
@@ -0,0 +1,107 @@
+/**
+ *
+ * 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.
+ */
+#ifndef LIBMINIFI_INCLUDE_CORE_CONTROLLER_STANDARDCONTROLLERSERVICENODE_H_
+#define LIBMINIFI_INCLUDE_CORE_CONTROLLER_STANDARDCONTROLLERSERVICENODE_H_
+
+#include "core/Core.h"
+#include "ControllerServiceNode.h"
+#include "core/logging/Logger.h"
+#include "core/ProcessGroup.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace controller {
+
+class StandardControllerServiceNode : public ControllerServiceNode {
+ public:
+
+  explicit StandardControllerServiceNode(
+      std::shared_ptr<ControllerService> service,
+      std::shared_ptr<ControllerServiceProvider> provider,
+      const std::string &id, std::shared_ptr<Configure> configuration)
+      : ControllerServiceNode(service, id, configuration),
+        provider(provider) {
+  }
+
+  explicit StandardControllerServiceNode(
+      std::shared_ptr<ControllerService> service, const std::string &id,
+      std::shared_ptr<Configure> configuration)
+      : ControllerServiceNode(service, id, configuration),
+        provider(nullptr) {
+  }
+
+  std::shared_ptr<core::ProcessGroup> &getProcessGroup();
+
+  void setProcessGroup(std::shared_ptr<ProcessGroup> &processGroup);
+
+  StandardControllerServiceNode(const StandardControllerServiceNode &other) = delete;
+  StandardControllerServiceNode &operator=(
+      const StandardControllerServiceNode &parent) = delete;
+
+  /**
+   * Initializes the controller service node.
+   */
+  virtual void initialize() {
+    ControllerServiceNode::initialize();
+    active = false;
+  }
+
+  bool canEnable() {
+    if (!active.load()) {
+      for (auto linked_service : linked_controller_services_) {
+        if (!linked_service->canEnable()) {
+          return false;
+        }
+      }
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  bool enable();
+
+  bool disable() {
+    controller_service_->setState(DISABLED);
+    active = false;
+    return true;
+  }
+
+ protected:
+
+  // controller service provider.
+  std::shared_ptr<ControllerServiceProvider> provider;
+
+  // process group.
+  std::shared_ptr<core::ProcessGroup> process_group_;
+
+  std::mutex mutex_;
+
+};
+
+} /* namespace controller */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_CONTROLLER_STANDARDCONTROLLERSERVICENODE_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/controller/StandardControllerServiceProvider.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/controller/StandardControllerServiceProvider.h b/libminifi/include/core/controller/StandardControllerServiceProvider.h
new file mode 100644
index 0000000..ba8af56
--- /dev/null
+++ b/libminifi/include/core/controller/StandardControllerServiceProvider.h
@@ -0,0 +1,229 @@
+/**
+ *
+ * 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.
+ */
+#ifndef LIBMINIFI_INCLUDE_CORE_CONTROLLER_STANDARDStandardControllerServiceProvider_H_
+#define LIBMINIFI_INCLUDE_CORE_CONTROLLER_STANDARDStandardControllerServiceProvider_H_
+
+#include <iostream>
+#include <memory>
+#include <vector>
+#include "core/ProcessGroup.h"
+#include "SchedulingAgent.h"
+#include "core/ClassLoader.h"
+#include "ControllerService.h"
+#include "ControllerServiceMap.h"
+#include "ControllerServiceNode.h"
+#include "StandardControllerServiceNode.h"
+#include "ControllerServiceProvider.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace controller {
+
+class StandardControllerServiceProvider : public ControllerServiceProvider,
+    public std::enable_shared_from_this<StandardControllerServiceProvider> {
+ public:
+
+  explicit StandardControllerServiceProvider(
+      std::shared_ptr<ControllerServiceMap> services,
+      std::shared_ptr<ProcessGroup> root_group, std::shared_ptr<Configure> configuration,
+      std::shared_ptr<minifi::SchedulingAgent> agent, ClassLoader &loader =
+          ClassLoader::getDefaultClassLoader())
+      : ControllerServiceProvider(services),
+        root_group_(root_group),
+        agent_(agent),
+        extension_loader_(loader),
+        configuration_(configuration) {
+  }
+
+  explicit StandardControllerServiceProvider(
+      std::shared_ptr<ControllerServiceMap> services,
+      std::shared_ptr<ProcessGroup> root_group, std::shared_ptr<Configure> configuration,
+      ClassLoader &loader = ClassLoader::getDefaultClassLoader())
+      : ControllerServiceProvider(services),
+        root_group_(root_group),
+        agent_(0),
+        extension_loader_(loader),
+        configuration_(configuration) {
+  }
+
+  explicit StandardControllerServiceProvider(
+      const StandardControllerServiceProvider && other)
+      : ControllerServiceProvider(std::move(other)),
+        root_group_(std::move(other.root_group_)),
+        agent_(std::move(other.agent_)),
+        extension_loader_(other.extension_loader_),
+        configuration_(other.configuration_) {
+
+  }
+
+  void setRootGroup(std::shared_ptr<ProcessGroup> rg) {
+    root_group_ = rg;
+  }
+
+  void setSchedulingAgent(std::shared_ptr<minifi::SchedulingAgent> agent) {
+    agent_ = agent;
+  }
+
+  std::shared_ptr<ControllerServiceNode> createControllerService(
+      const std::string &type, const std::string &id,
+      bool firstTimeAdded) {
+
+    std::shared_ptr<ControllerService> new_controller_service =
+        extension_loader_.instantiate<ControllerService>(type, id);
+
+    if (nullptr == new_controller_service) {
+      return nullptr;
+    }
+
+    std::shared_ptr<ControllerServiceNode> new_service_node = std::make_shared<
+        StandardControllerServiceNode>(
+        new_controller_service,
+        std::static_pointer_cast<ControllerServiceProvider>(shared_from_this()),
+        id, configuration_);
+
+    controller_map_->put(id, new_service_node);
+    return new_service_node;
+
+  }
+
+
+  void enableControllerService(
+      std::shared_ptr<ControllerServiceNode> &serviceNode) {
+    if (serviceNode->canEnable()) {
+      agent_->enableControllerService(serviceNode);
+    }
+  }
+
+
+  virtual void enableAllControllerServices() {
+    logger_->log_info("Enabling %d controller services",
+                      controller_map_->getAllControllerServices().size());
+    for (auto service : controller_map_->getAllControllerServices()) {
+
+      if (service->canEnable()) {
+        logger_->log_info("Enabling %s", service->getName());
+        agent_->enableControllerService(service);
+      } else {
+        logger_->log_info("Could not enable %s", service->getName());
+      }
+    }
+  }
+
+
+  void enableControllerServices(
+      std::vector<std::shared_ptr<ControllerServiceNode>> serviceNodes) {
+    for (auto node : serviceNodes) {
+      enableControllerService(node);
+    }
+  }
+
+
+  void disableControllerService(
+      std::shared_ptr<ControllerServiceNode> &serviceNode) {
+    if (!IsNullOrEmpty(serviceNode.get()) && serviceNode->enabled()) {
+      agent_->disableControllerService(serviceNode);
+    }
+  }
+
+
+  void verifyCanStopReferencingComponents(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+  }
+
+
+  std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> unscheduleReferencingComponents(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+    std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> references =
+        findLinkedComponents(serviceNode);
+    for (auto ref : references) {
+      agent_->disableControllerService(ref);
+    }
+    return references;
+  }
+
+
+  void verifyCanDisableReferencingServices(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+    std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> references =
+        findLinkedComponents(serviceNode);
+    for (auto ref : references) {
+      if (!ref->canEnable()) {
+        logger_->log_info("Cannot disable %s", ref->getName());
+      }
+    }
+  }
+
+
+  virtual std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> disableReferencingServices(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+    std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> references =
+        findLinkedComponents(serviceNode);
+    for (auto ref : references) {
+      agent_->disableControllerService(ref);
+    }
+
+    return references;
+  }
+
+  std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> enableReferencingServices(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+    std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> references =
+        findLinkedComponents(serviceNode);
+    for (auto ref : references) {
+      agent_->enableControllerService(ref);
+    }
+    return references;
+  }
+
+  std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> scheduleReferencingComponents(
+      std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+    std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> references =
+        findLinkedComponents(serviceNode);
+    for (auto ref : references) {
+      agent_->enableControllerService(ref);
+    }
+    return references;
+  }
+
+ protected:
+
+  bool canEdit() {
+    return false;
+  }
+
+  std::shared_ptr<minifi::SchedulingAgent> agent_;
+
+  ClassLoader &extension_loader_;
+
+  std::shared_ptr<Configure> configuration_;
+
+  std::shared_ptr<ProcessGroup> root_group_;
+
+};
+
+} /* namespace controller */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_CONTROLLER_STANDARDStandardControllerServiceProvider_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/core/yaml/YamlConfiguration.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/yaml/YamlConfiguration.h b/libminifi/include/core/yaml/YamlConfiguration.h
index 793cdb9..6226a4e 100644
--- a/libminifi/include/core/yaml/YamlConfiguration.h
+++ b/libminifi/include/core/yaml/YamlConfiguration.h
@@ -20,6 +20,7 @@
 
 #include "core/ProcessorConfig.h"
 #include "yaml-cpp/yaml.h"
+#include "processors/LoadProcessors.h"
 #include "../FlowConfiguration.h"
 #include "Site2SiteClientProtocol.h"
 #include <string>
@@ -36,18 +37,21 @@ namespace core {
 #define CONFIG_YAML_FLOW_CONTROLLER_KEY "Flow Controller"
 #define CONFIG_YAML_PROCESSORS_KEY "Processors"
 #define CONFIG_YAML_CONNECTIONS_KEY "Connections"
+#define CONFIG_YAML_CONTROLLER_SERVICES_KEY "Controller Services"
 #define CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY "Remote Processing Groups"
 #define CONFIG_YAML_PROVENANCE_REPORT_KEY "Provenance Reporting"
 
 class YamlConfiguration : public FlowConfiguration {
 
  public:
-  YamlConfiguration(std::shared_ptr<core::Repository> repo,
+  explicit YamlConfiguration(std::shared_ptr<core::Repository> repo,
                     std::shared_ptr<core::Repository> flow_file_repo,
                     std::shared_ptr<io::StreamFactory> stream_factory,
+                    std::shared_ptr<Configure> configuration,
                     const std::string path = DEFAULT_FLOW_YAML_FILE_NAME)
-      : FlowConfiguration(repo, flow_file_repo, stream_factory, path) {
-       stream_factory_ = stream_factory;
+      : FlowConfiguration(repo, flow_file_repo, stream_factory, configuration,
+                          path) {
+    stream_factory_ = stream_factory;
     if (IsNullOrEmpty(config_path_)) {
       config_path_ = DEFAULT_FLOW_YAML_FILE_NAME;
     }
@@ -67,7 +71,8 @@ class YamlConfiguration : public FlowConfiguration {
    * @return              the root ProcessGroup node of the flow
    *                        configuration tree
    */
-  std::unique_ptr<core::ProcessGroup> getRoot(const std::string &yamlConfigStr) {
+  std::unique_ptr<core::ProcessGroup> getRoot(
+      const std::string &yamlConfigStr) {
     YAML::Node rootYamlNode = YAML::LoadFile(yamlConfigStr);
     return getRoot(&rootYamlNode);
   }
@@ -106,9 +111,14 @@ class YamlConfiguration : public FlowConfiguration {
     YAML::Node flowControllerNode = rootYaml[CONFIG_YAML_FLOW_CONTROLLER_KEY];
     YAML::Node processorsNode = rootYaml[CONFIG_YAML_PROCESSORS_KEY];
     YAML::Node connectionsNode = rootYaml[CONFIG_YAML_CONNECTIONS_KEY];
-    YAML::Node remoteProcessingGroupsNode = rootYaml[CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY];
-    YAML::Node provenanceReportNode = rootYaml[CONFIG_YAML_PROVENANCE_REPORT_KEY];
+    YAML::Node controllerServiceNode =
+        rootYaml[CONFIG_YAML_CONTROLLER_SERVICES_KEY];
+    YAML::Node remoteProcessingGroupsNode =
+        rootYaml[CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY];
+    YAML::Node provenanceReportNode =
+        rootYaml[CONFIG_YAML_PROVENANCE_REPORT_KEY];
 
+    parseControllerServices(&controllerServiceNode);
     // Create the root process group
     core::ProcessGroup * root = parseRootProcessGroupYaml(flowControllerNode);
     parseProcessorNodeYaml(processorsNode, root);
@@ -116,6 +126,15 @@ class YamlConfiguration : public FlowConfiguration {
     parseConnectionYaml(&connectionsNode, root);
     parseProvenanceReportingYaml(&provenanceReportNode, root);
 
+    // set the controller services into the root group.
+    for (auto controller_service : controller_services_
+        ->getAllControllerServices()) {
+      root->addControllerService(controller_service->getName(),
+                                 controller_service);
+      root->addControllerService(controller_service->getUUIDStr(),
+                                 controller_service);
+    }
+
     return std::unique_ptr<core::ProcessGroup>(root);
   }
 
@@ -147,7 +166,6 @@ class YamlConfiguration : public FlowConfiguration {
   void parsePortYaml(YAML::Node *portNode, core::ProcessGroup *parent,
                      TransferDirection direction);
 
-
   /**
    * Parses the root level YAML node for the flow configuration and
    * returns a ProcessGroup containing the tree of flow configuration
@@ -158,6 +176,17 @@ class YamlConfiguration : public FlowConfiguration {
    */
   core::ProcessGroup *parseRootProcessGroupYaml(YAML::Node rootNode);
 
+  // Process Property YAML
+  void parseProcessorPropertyYaml(YAML::Node *doc, YAML::Node *node,
+                                  std::shared_ptr<core::Processor> processor);
+  /**
+   * Parse controller services
+   * @param controllerServicesNode controller services YAML node.
+   * @param parent parent process group.
+   */
+  void parseControllerServices(YAML::Node *controllerServicesNode);
+  // Process connection YAML
+
   /**
    * Parses the Connections section of a configuration YAML.
    * The resulting Connections are added to the parent ProcessGroup.
@@ -200,8 +229,9 @@ class YamlConfiguration : public FlowConfiguration {
    * @param propertiesNode the YAML::Node containing the properties
    * @param processor      the Processor to which to add the resulting properties
    */
-  void parsePropertiesNodeYaml(YAML::Node *propertiesNode,
-                               std::shared_ptr<core::Processor> processor);
+  void parsePropertiesNodeYaml(
+      YAML::Node *propertiesNode,
+      std::shared_ptr<core::ConfigurableComponent> processor);
 
   /**
    * A helper function for parsing or generating optional id fields.
@@ -219,7 +249,8 @@ class YamlConfiguration : public FlowConfiguration {
    *                   is optional and defaults to 'id'
    * @return         the parsed or generated UUID string
    */
-  std::string getOrGenerateId(YAML::Node *yamlNode, const std::string &idField = "id");
+  std::string getOrGenerateId(YAML::Node *yamlNode, const std::string &idField =
+                                  "id");
 
   /**
    * This is a helper function for verifying the existence of a required
@@ -239,8 +270,7 @@ class YamlConfiguration : public FlowConfiguration {
    * @throws std::invalid_argument if the required field 'fieldName' is
    *                               not present in 'yamlNode'
    */
-  void checkRequiredField(YAML::Node *yamlNode,
-                          const std::string &fieldName,
+  void checkRequiredField(YAML::Node *yamlNode, const std::string &fieldName,
                           const std::string &yamlSection = "",
                           const std::string &errorMessage = "");
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/io/StreamFactory.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/StreamFactory.h b/libminifi/include/io/StreamFactory.h
index db4625a..3faee45 100644
--- a/libminifi/include/io/StreamFactory.h
+++ b/libminifi/include/io/StreamFactory.h
@@ -30,7 +30,11 @@ namespace io {
 
 class AbstractStreamFactory {
  public:
-  virtual std::unique_ptr<Socket> createSocket(const std::string &host, const uint16_t port) = 0;
+  virtual ~AbstractStreamFactory() {
+  }
+
+  virtual std::unique_ptr<Socket> createSocket(const std::string &host,
+                                               const uint16_t port) = 0;
 };
 
 /**

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/io/TLSSocket.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/TLSSocket.h b/libminifi/include/io/TLSSocket.h
new file mode 100644
index 0000000..011a012
--- /dev/null
+++ b/libminifi/include/io/TLSSocket.h
@@ -0,0 +1,197 @@
+/**
+ *
+ * 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.
+ */
+#ifndef LIBMINIFI_INCLUDE_IO_TLSSOCKET_H_
+#define LIBMINIFI_INCLUDE_IO_TLSSOCKET_H_
+
+#include <cstdint>
+#include "ClientSocket.h"
+#include <atomic>
+#include <mutex>
+
+#include "properties/Configure.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
+
+#include <openssl/ssl.h>
+#include <openssl/err.h>
+
+#define TLS_ERROR_CONTEXT 1
+#define TLS_ERROR_PEM_MISSING 2
+#define TLS_ERROR_CERT_MISSING 3
+#define TLS_ERROR_KEY_ERROR 4
+#define TLS_ERROR_CERT_ERROR 5
+
+class TLSContext {
+
+ public:
+
+  /**
+   * Build an instance, creating a memory fence, which
+   * allows us to avoid locking. This is tantamount to double checked locking.
+   * @returns new TLSContext;
+   */
+  static TLSContext *getInstance() {
+    TLSContext* atomic_context = context_instance.load(
+        std::memory_order_relaxed);
+    std::atomic_thread_fence(std::memory_order_acquire);
+    if (atomic_context == nullptr) {
+      std::lock_guard<std::mutex> lock(context_mutex);
+      atomic_context = context_instance.load(std::memory_order_relaxed);
+      if (atomic_context == nullptr) {
+        atomic_context = new TLSContext();
+        atomic_context->initialize();
+        std::atomic_thread_fence(std::memory_order_release);
+        context_instance.store(atomic_context, std::memory_order_relaxed);
+      }
+    }
+    return atomic_context;
+  }
+
+  virtual ~TLSContext() {
+    if (0 != ctx)
+      SSL_CTX_free(ctx);
+  }
+
+  SSL_CTX *getContext() {
+    return ctx;
+  }
+
+  short getError() {
+    return error_value;
+  }
+
+  short initialize();
+
+ private:
+
+  static int pemPassWordCb(char *buf, int size, int rwflag, void *userdata) {
+    std::string passphrase;
+
+    if (Configure::getConfigure()->get(
+        Configure::nifi_security_client_pass_phrase, passphrase)) {
+
+      std::ifstream file(passphrase.c_str(), std::ifstream::in);
+      if (!file.good()) {
+        memset(buf, 0x00, size);
+        return 0;
+      }
+
+      std::string password;
+      password.assign((std::istreambuf_iterator<char>(file)),
+                      std::istreambuf_iterator<char>());
+      file.close();
+      memset(buf, 0x00, size);
+      memcpy(buf, password.c_str(), password.length() - 1);
+
+      return password.length() - 1;
+    }
+    return 0;
+  }
+
+  TLSContext();
+
+  std::shared_ptr<logging::Logger> logger_;
+  Configure *configuration;
+  SSL_CTX *ctx;
+
+  short error_value;
+
+  static std::atomic<TLSContext*> context_instance;
+  static std::mutex context_mutex;
+};
+
+class TLSSocket : public Socket {
+ public:
+
+  /**
+   * Constructor that accepts host name, port and listeners. With this
+   * contructor we will be creating a server socket
+   * @param hostname our host name
+   * @param port connecting port
+   * @param listeners number of listeners in the queue
+   */
+  explicit TLSSocket(const std::string &hostname, const uint16_t port,
+                     const uint16_t listeners);
+
+  /**
+   * Constructor that creates a client socket.
+   * @param hostname hostname we are connecting to.
+   * @param port port we are connecting to.
+   */
+  explicit TLSSocket(const std::string &hostname, const uint16_t port);
+
+  /**
+   * Move constructor.
+   */
+  explicit TLSSocket(const TLSSocket &&);
+
+  virtual ~TLSSocket();
+
+  /**
+   * Initializes the socket
+   * @return result of the creation operation.
+   */
+  short initialize();
+
+  /**
+   * Attempt to select the socket file descriptor
+   * @param msec timeout interval to wait
+   * @returns file descriptor
+   */
+  virtual short select_descriptor(const uint16_t msec);
+
+  /**
+   * Reads data and places it into buf
+   * @param buf buffer in which we extract data
+   * @param buflen
+   */
+  virtual int readData(uint8_t *buf, int buflen);
+
+  /**
+   * Write value to the stream using std::vector
+   * @param buf incoming buffer
+   * @param buflen buffer to write
+   *
+   */
+  int writeData(std::vector<uint8_t> &buf, int buflen);
+
+  /**
+   * Write value to the stream using uint8_t ptr
+   * @param buf incoming buffer
+   * @param buflen buffer to write
+   *
+   */
+  int writeData(uint8_t *value, int size);
+
+ protected:
+
+  SSL* ssl;
+
+};
+
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_IO_TLSSOCKET_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/io/validation.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/validation.h b/libminifi/include/io/validation.h
index 9dd1b8a..a5e1bc5 100644
--- a/libminifi/include/io/validation.h
+++ b/libminifi/include/io/validation.h
@@ -56,7 +56,7 @@ static auto IsNullOrEmpty(
 template<typename T>
 static auto IsNullOrEmpty(
     T *object) -> typename std::enable_if<size_function_functor_checker<T>::has_size_function==1, bool>::type {
-  return (0 == object || object->size() == 0);
+  return (nullptr == object || object->size() == 0);
 }
 
 /**
@@ -65,8 +65,18 @@ static auto IsNullOrEmpty(
 template<typename T>
 static auto IsNullOrEmpty(
     T *object) -> typename std::enable_if<not size_function_functor_checker<T>::has_size_function , bool>::type {
-  return (0 == object);
+  return (nullptr == object);
 }
+
+/**
+ * Determines if the variable is null or ::size() == 0
+ */
+template<typename T>
+static auto IsNullOrEmpty(
+    std::shared_ptr<T> object) -> typename std::enable_if<not size_function_functor_checker<T>::has_size_function , bool>::type {
+  return (nullptr == object || nullptr == object.get());
+}
+
 /**
  * Determines if the variable is null or strlen(str) == 0
  */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/processors/AppendHostInfo.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/AppendHostInfo.h b/libminifi/include/processors/AppendHostInfo.h
index a16dff3..d33c717 100644
--- a/libminifi/include/processors/AppendHostInfo.h
+++ b/libminifi/include/processors/AppendHostInfo.h
@@ -25,6 +25,7 @@
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
 #include "core/Core.h"
+#include "core/Resource.h"
 
 namespace org {
 namespace apache {
@@ -70,6 +71,9 @@ class AppendHostInfo : public core::Processor {
   std::shared_ptr<logging::Logger> logger_;
 };
 
+REGISTER_RESOURCE(AppendHostInfo);
+
+
 } /* namespace processors */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/processors/ExecuteProcess.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/ExecuteProcess.h b/libminifi/include/processors/ExecuteProcess.h
index f74f489..dbf2d15 100644
--- a/libminifi/include/processors/ExecuteProcess.h
+++ b/libminifi/include/processors/ExecuteProcess.h
@@ -35,6 +35,7 @@
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
 #include "core/Core.h"
+#include "core/Resource.h"
 
 namespace org {
 namespace apache {
@@ -115,6 +116,8 @@ class ExecuteProcess : public core::Processor {
   pid_t _pid;
 };
 
+REGISTER_RESOURCE(ExecuteProcess);
+
 } /* namespace processors */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/processors/GenerateFlowFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/GenerateFlowFile.h b/libminifi/include/processors/GenerateFlowFile.h
index d15a02c..2f24e64 100644
--- a/libminifi/include/processors/GenerateFlowFile.h
+++ b/libminifi/include/processors/GenerateFlowFile.h
@@ -24,6 +24,7 @@
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
 #include "core/Core.h"
+#include "core/Resource.h"
 
 namespace org {
 namespace apache {
@@ -89,6 +90,8 @@ class GenerateFlowFile : public core::Processor {
   uint64_t _dataSize;
 };
 
+REGISTER_RESOURCE(GenerateFlowFile);
+
 } /* namespace processors */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/processors/GetFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/GetFile.h b/libminifi/include/processors/GetFile.h
index 5345404..e25e7db 100644
--- a/libminifi/include/processors/GetFile.h
+++ b/libminifi/include/processors/GetFile.h
@@ -23,6 +23,7 @@
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
 #include "core/Core.h"
+#include "core/Resource.h"
 
 namespace org {
 namespace apache {
@@ -134,6 +135,8 @@ class GetFile : public core::Processor {
 
 };
 
+REGISTER_RESOURCE(GetFile);
+
 } /* namespace processors */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/processors/InvokeHTTP.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/InvokeHTTP.h b/libminifi/include/processors/InvokeHTTP.h
index 789b3b5..ca4fef6 100644
--- a/libminifi/include/processors/InvokeHTTP.h
+++ b/libminifi/include/processors/InvokeHTTP.h
@@ -28,6 +28,8 @@
 #include "core/ProcessSession.h"
 #include "core/Core.h"
 #include "core/Property.h"
+#include "core/Resource.h"
+#include "controllers/SSLContextService.h"
 #include "utils/ByteInputCallBack.h"
 
 namespace org {
@@ -109,7 +111,8 @@ class InvokeHTTP : public core::Processor {
         connect_timeout_(20000),
         penalize_no_retry_(false),
         read_timeout_(20000),
-        always_output_response_(false) {
+        always_output_response_(false),
+        ssl_context_service_(nullptr) {
     curl_global_init(CURL_GLOBAL_DEFAULT);
   }
   // Destructor
@@ -161,6 +164,23 @@ class InvokeHTTP : public core::Processor {
  protected:
 
   /**
+   * Configures the SSL Context. Relies on the Context service and OpenSSL's installation
+   */
+  static CURLcode configure_ssl_context(CURL *curl, void *ctx, void *param);
+
+  /**
+   * Determines if a secure connection is required
+   * @param url url we will be connecting to
+   * @returns true if secure connection is allowed/required
+   */
+  bool isSecure(const std::string &url);
+
+  /**
+   * Configures a secure connection
+   */
+  void configure_secure_connection(CURL *http_session);
+
+  /**
    * Generate a transaction ID
    * @return transaction ID string.
    */
@@ -190,13 +210,17 @@ class InvokeHTTP : public core::Processor {
   void route(std::shared_ptr<FlowFileRecord> &request,
              std::shared_ptr<FlowFileRecord> &response,
              core::ProcessSession *session, core::ProcessContext *context,
-             bool isSuccess, int statusCode);
+             bool isSuccess,
+             int statusCode);
   /**
    * Determine if we should emit a new flowfile based on our activity
    * @param method method type
    * @return result of the evaluation.
    */
   bool emitFlowFile(const std::string &method);
+
+  std::shared_ptr<minifi::controllers::SSLContextService> ssl_context_service_;
+
   CURLcode res;
 
   // http method
@@ -219,6 +243,8 @@ class InvokeHTTP : public core::Processor {
   bool penalize_no_retry_;
 };
 
+REGISTER_RESOURCE(InvokeHTTP)
+
 } /* namespace processors */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/processors/ListenHTTP.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/ListenHTTP.h b/libminifi/include/processors/ListenHTTP.h
index 69432be..586a699 100644
--- a/libminifi/include/processors/ListenHTTP.h
+++ b/libminifi/include/processors/ListenHTTP.h
@@ -29,6 +29,7 @@
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
 #include "core/Core.h"
+#include "core/Resource.h"
 
 namespace org {
 namespace apache {
@@ -105,8 +106,6 @@ class ListenHTTP : public core::Processor {
     const struct mg_request_info *_reqInfo;
   };
 
- protected:
-
  private:
   // Logger
   std::shared_ptr<logging::Logger> _logger;
@@ -115,6 +114,9 @@ class ListenHTTP : public core::Processor {
   std::unique_ptr<Handler> _handler;
 };
 
+
+REGISTER_RESOURCE(ListenHTTP);
+
 } /* namespace processors */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/processors/ListenSyslog.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/ListenSyslog.h b/libminifi/include/processors/ListenSyslog.h
index cbbdf41..4e642e8 100644
--- a/libminifi/include/processors/ListenSyslog.h
+++ b/libminifi/include/processors/ListenSyslog.h
@@ -36,6 +36,7 @@
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
 #include "core/Core.h"
+#include "core/Resource.h"
 
 namespace org {
 namespace apache {
@@ -206,6 +207,8 @@ class ListenSyslog : public core::Processor {
   char _buffer[2048];
 };
 
+REGISTER_RESOURCE(ListenSyslog);
+
 } /* namespace processors */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/processors/LoadProcessors.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/LoadProcessors.h b/libminifi/include/processors/LoadProcessors.h
new file mode 100644
index 0000000..7a16773
--- /dev/null
+++ b/libminifi/include/processors/LoadProcessors.h
@@ -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
+ *
+ *     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.
+ */
+#ifndef LIBMINIFI_INCLUDE_PROCESSORS_LOADPROCESSORS_H_
+#define LIBMINIFI_INCLUDE_PROCESSORS_LOADPROCESSORS_H_
+
+#include "core/Core.h"
+#include "core/Resource.h"
+
+#include "AppendHostInfo.h"
+#include "ExecuteProcess.h"
+#include "GenerateFlowFile.h"
+#include "GetFile.h"
+#include "ListenHTTP.h"
+#include "LogAttribute.h"
+#include "PutFile.h"
+#include "TailFile.h"
+
+
+#endif /* LIBMINIFI_INCLUDE_PROCESSORS_LOADPROCESSORS_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/processors/LogAttribute.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/LogAttribute.h b/libminifi/include/processors/LogAttribute.h
index dcc802d..56864c7 100644
--- a/libminifi/include/processors/LogAttribute.h
+++ b/libminifi/include/processors/LogAttribute.h
@@ -24,6 +24,7 @@
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
 #include "core/Core.h"
+#include "core/Resource.h"
 
 namespace org {
 namespace apache {
@@ -120,6 +121,8 @@ class LogAttribute : public core::Processor {
   std::shared_ptr<logging::Logger> logger_;
 };
 
+REGISTER_RESOURCE(LogAttribute);
+
 } /* namespace processors */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/processors/PutFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/PutFile.h b/libminifi/include/processors/PutFile.h
index cc5dfca..a51f6b5 100644
--- a/libminifi/include/processors/PutFile.h
+++ b/libminifi/include/processors/PutFile.h
@@ -24,6 +24,7 @@
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
 #include "core/Core.h"
+#include "core/Resource.h"
 
 namespace org {
 namespace apache {
@@ -103,6 +104,8 @@ class PutFile : public core::Processor {
                const std::string &tmpFile, const std::string &destFile);
 };
 
+REGISTER_RESOURCE(PutFile);
+
 } /* namespace processors */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/processors/TailFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/TailFile.h b/libminifi/include/processors/TailFile.h
index c6349a0..ac7db5a 100644
--- a/libminifi/include/processors/TailFile.h
+++ b/libminifi/include/processors/TailFile.h
@@ -24,6 +24,7 @@
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
 #include "core/Core.h"
+#include "core/Resource.h"
 
 namespace org {
 namespace apache {
@@ -73,8 +74,9 @@ class TailFile : public core::Processor {
   std::string _stateFile;
   // State related to the tailed file
   std::string _currentTailFileName;
-  uint64_t _currentTailFilePosition;
+  // determine if state is recovered;
   bool _stateRecovered;
+  uint64_t _currentTailFilePosition;
   uint64_t _currentTailFileCreatedTime;
   static const int BUFFER_SIZE = 512;
 
@@ -89,6 +91,8 @@ class TailFile : public core::Processor {
 
 };
 
+REGISTER_RESOURCE(TailFile);
+
 // Matched File Item for Roll over check
 typedef struct {
   std::string fileName;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/properties/Configure.h
----------------------------------------------------------------------
diff --git a/libminifi/include/properties/Configure.h b/libminifi/include/properties/Configure.h
index cb1b412..4119edd 100644
--- a/libminifi/include/properties/Configure.h
+++ b/libminifi/include/properties/Configure.h
@@ -28,6 +28,7 @@
 #include <iostream>
 #include <fstream>
 #include "core/Core.h"
+#include "utils/StringUtils.h"
 #include "core/logging/Logger.h"
 
 namespace org {
@@ -38,7 +39,9 @@ namespace minifi {
 class Configure {
  public:
   // nifi.flow.configuration.file
+  static const char *nifi_default_directory;
   static const char *nifi_flow_configuration_file;
+  static const char *nifi_flow_engine_threads;
   static const char *nifi_administrative_yield_duration;
   static const char *nifi_bored_yield_duration;
   static const char *nifi_graceful_shutdown_seconds;
@@ -80,7 +83,7 @@ class Configure {
   // Set the config value
   void set(std::string key, std::string value) {
     std::lock_guard<std::mutex> lock(mutex_);
-    properties_[key] = value;
+    properties_[key] = std::string(value.c_str());
   }
   // Check whether the config value existed
   bool has(std::string key) {
@@ -89,6 +92,12 @@ class Configure {
   }
   // Get the config value
   bool get(std::string key, std::string &value);
+
+  /**
+   * Returns the configuration value or an empty string.
+   * @return value corresponding to key or empty value.
+   */
+  int getInt(const std::string &key, int default_value);
   // Parse one line in configure file like key=value
   void parseConfigureFileLine(char *buf);
   // Load Configure File

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/include/utils/ThreadPool.h
----------------------------------------------------------------------
diff --git a/libminifi/include/utils/ThreadPool.h b/libminifi/include/utils/ThreadPool.h
index 7508900..e3c15d8 100644
--- a/libminifi/include/utils/ThreadPool.h
+++ b/libminifi/include/utils/ThreadPool.h
@@ -35,58 +35,54 @@ namespace utils {
  * purpose: Provides a wrapper for the functor
  * and returns a future based on the template argument.
  */
-template< typename T>
-class Worker{
-public:
-  explicit Worker(std::function<T()> &task) : task(task)
-  {
+template<typename T>
+class Worker {
+ public:
+  explicit Worker(std::function<T()> &task)
+      : task(task) {
     promise = std::make_shared<std::promise<T>>();
   }
 
   /**
    * Move constructor for worker tasks
    */
-  Worker(Worker &&other) : task (std::move(other.task)),
-						promise(other.promise)
-  {
+  Worker(Worker &&other)
+      : task(std::move(other.task)),
+        promise(other.promise) {
   }
 
-
   /**
    * Runs the task and takes the output from the funtor
    * setting the result into the promise
    */
-  void run()
-  {
+  void run() {
     T result = task();
     promise->set_value(result);
   }
 
-   Worker<T>(const Worker<T>&) = delete;
-    Worker<T>& operator = (const Worker<T>&) = delete;
+  Worker<T>(const Worker<T>&) = delete;
+  Worker<T>& operator =(const Worker<T>&) = delete;
 
-  Worker<T>& operator = (Worker<T>&&) ;
+  Worker<T>& operator =(Worker<T> &&);
 
   std::shared_ptr<std::promise<T>> getPromise();
 
-private:
-   std::function<T()> task;
-   std::shared_ptr<std::promise<T>> promise;
+ private:
+  std::function<T()> task;
+  std::shared_ptr<std::promise<T>> promise;
 };
 
-template< typename T>
-Worker<T>&  Worker<T>::operator = (Worker<T>&& other)
-{
-    task = std::move(other.task);
-    promise = other.promise;
-    return *this;
+template<typename T>
+Worker<T>& Worker<T>::operator =(Worker<T> && other) {
+  task = std::move(other.task);
+  promise = other.promise;
+  return *this;
 }
 
-
 template<typename T>
-std::shared_ptr<std::promise<T>> Worker<T>::getPromise(){
-    return promise;
-  }
+std::shared_ptr<std::promise<T>> Worker<T>::getPromise() {
+  return promise;
+}
 
 /**
  * Thread pool
@@ -95,171 +91,190 @@ std::shared_ptr<std::promise<T>> Worker<T>::getPromise(){
  * Design: Locked control over a manager thread that controls the worker threads
  */
 template<typename T>
-class ThreadPool
-    {
-    public:
-        ThreadPool(int max_worker_threads, bool daemon_threads=false) : max_worker_threads_(max_worker_threads)
-	,daemon_threads_(daemon_threads), running_(false){
-	  current_workers_ = 0;
-	}
-        virtual ~ThreadPool(){
-	  shutdown();
-	}
-
-	/**
-	 * Execute accepts a worker task and returns
-	 * a future
-	 * @param task this thread pool will subsume ownership of
-	 * the worker task
-	 * @return future with the impending result.
-	 */
-        std::future<T> execute(Worker<T> &&task);
-	/**
-	 * Starts the Thread Pool
-	 */
-        void start();
-	/**
-	 * Shutdown the thread pool and clear any
-	 * currently running activities
-	 */
-	void shutdown();
-	/**
-	 * Set the max concurrent tasks. When this is done
-	 * we must start and restart the thread pool if
-	 * the number of tasks is less than the currently configured number
-	 */
-	void setMaxConcurrentTasks(uint16_t max)
-	{
-	  std::lock_guard<std::recursive_mutex> lock(manager_mutex_);
-	  if (running_)
-	  {
-	    shutdown();
-	  }
-	  max_worker_threads_= max;
-	  if (!running_)
-	    start();
-	}
-
-    protected:
-
-	/**
-	* Drain will notify tasks to stop following notification
-	*/
-	void drain()
-	{
-	  while(current_workers_ > 0)
-	  {
-	    tasks_available_.notify_one();
-	  }
-	}
-	// determines if threads are detached
-	bool daemon_threads_;
-	// max worker threads
-        int max_worker_threads_;
-	// current worker tasks.
-	std::atomic<int> current_workers_;
-	// thread queue
-        std::vector<std::thread> thread_queue_;
-	// manager thread
-        std::thread manager_thread_;
-	// atomic running boolean
-	std::atomic<bool> running_;
-	// worker queue of worker objects
-        std::queue<Worker<T>> worker_queue_;
-	// notification for available work
-        std::condition_variable tasks_available_;
-	// manager mutex
-	std::recursive_mutex manager_mutex_;
-	// work queue mutex
-        std::mutex worker_queue_mutex_;
-
-	/**
-	 * Call for the manager to start worker threads
-	 */
-	void startWorkers();
-
-	/**
-	 * Runs worker tasks
-	 */
-        void run_tasks();
-    };
+class ThreadPool {
+ public:
+
+  ThreadPool(int max_worker_threads = 8, bool daemon_threads = false)
+      : max_worker_threads_(max_worker_threads),
+        daemon_threads_(daemon_threads),
+        running_(false) {
+    current_workers_ = 0;
+  }
+
+  ThreadPool(const ThreadPool<T> &&other)
+      : max_worker_threads_(std::move(other.max_worker_threads_)),
+        daemon_threads_(std::move(other.daemon_threads_)),
+        running_(false) {
+    current_workers_ = 0;
+  }
+  virtual ~ThreadPool() {
+    shutdown();
+  }
+
+  /**
+   * Execute accepts a worker task and returns
+   * a future
+   * @param task this thread pool will subsume ownership of
+   * the worker task
+   * @return future with the impending result.
+   */
+  std::future<T> execute(Worker<T> &&task);
+  /**
+   * Starts the Thread Pool
+   */
+  void start();
+  /**
+   * Shutdown the thread pool and clear any
+   * currently running activities
+   */
+  void shutdown();
+  /**
+   * Set the max concurrent tasks. When this is done
+   * we must start and restart the thread pool if
+   * the number of tasks is less than the currently configured number
+   */
+  void setMaxConcurrentTasks(uint16_t max) {
+    std::lock_guard<std::recursive_mutex> lock(manager_mutex_);
+    if (running_) {
+      shutdown();
+    }
+    max_worker_threads_ = max;
+    if (!running_)
+      start();
+  }
+
+  ThreadPool<T> operator=(const ThreadPool<T> &other) = delete;
+  ThreadPool(const ThreadPool<T> &other) = delete;
+
+  ThreadPool<T> &operator=(ThreadPool<T> &&other) {
+    std::lock_guard<std::recursive_mutex> lock(manager_mutex_);
+    if (other.running_) {
+      other.shutdown();
+    }
+    if (running_) {
+      shutdown();
+    }
+    max_worker_threads_ = std::move(other.max_worker_threads_);
+    daemon_threads_ = std::move(other.daemon_threads_);
+    current_workers_ = 0;
+
+    thread_queue_ = std::move(other.thread_queue_);
+    worker_queue_ = std::move(other.worker_queue_);
+    if (!running_) {
+      start();
+    }
+    return *this;
+  }
+
+ protected:
+
+  /**
+   * Drain will notify tasks to stop following notification
+   */
+  void drain() {
+    while (current_workers_ > 0) {
+      tasks_available_.notify_one();
+    }
+  }
+// determines if threads are detached
+  bool daemon_threads_;
+// max worker threads
+  int max_worker_threads_;
+// current worker tasks.
+  std::atomic<int> current_workers_;
+// thread queue
+  std::vector<std::thread> thread_queue_;
+// manager thread
+  std::thread manager_thread_;
+// atomic running boolean
+  std::atomic<bool> running_;
+// worker queue of worker objects
+  std::queue<Worker<T>> worker_queue_;
+// notification for available work
+  std::condition_variable tasks_available_;
+// manager mutex
+  std::recursive_mutex manager_mutex_;
+// work queue mutex
+  std::mutex worker_queue_mutex_;
+
+  /**
+   * Call for the manager to start worker threads
+   */
+  void startWorkers();
+
+  /**
+   * Runs worker tasks
+   */
+  void run_tasks();
+}
+;
 
 template<typename T>
-std::future<T> ThreadPool<T>::execute(Worker<T> &&task){
+std::future<T> ThreadPool<T>::execute(Worker<T> &&task) {
 
   std::unique_lock<std::mutex> lock(worker_queue_mutex_);
   bool wasEmpty = worker_queue_.empty();
   std::future<T> future = task.getPromise()->get_future();
   worker_queue_.push(std::move(task));
-  if (wasEmpty)
-  {
-      tasks_available_.notify_one();
+  if (wasEmpty) {
+    tasks_available_.notify_one();
   }
   return future;
 }
 
-template< typename T>
-void  ThreadPool<T>::startWorkers(){
-    for (int i = 0; i < max_worker_threads_; i++)
-    {
-      thread_queue_.push_back( std::thread(&ThreadPool::run_tasks, this));
-      current_workers_++;
-    }
+template<typename T>
+void ThreadPool<T>::startWorkers() {
+  for (int i = 0; i < max_worker_threads_; i++) {
+    thread_queue_.push_back(std::thread(&ThreadPool::run_tasks, this));
+    current_workers_++;
+  }
 
-     if (daemon_threads_)
-     {
-	for (auto &thread : thread_queue_){
-	    thread.detach();
-	}
-     }
-    for (auto &thread : thread_queue_)
-    {
-	if (thread.joinable())
-	  thread.join();
+  if (daemon_threads_) {
+    for (auto &thread : thread_queue_) {
+      thread.detach();
     }
+  }
+  for (auto &thread : thread_queue_) {
+    if (thread.joinable())
+      thread.join();
+  }
 }
-template< typename T>
-void  ThreadPool<T>::run_tasks()
-{
-  while (running_.load())
-    {
-	std::unique_lock<std::mutex> lock(worker_queue_mutex_);
-	if (worker_queue_.empty())
-	{
+template<typename T>
+void ThreadPool<T>::run_tasks() {
+  while (running_.load()) {
+    std::unique_lock<std::mutex> lock(worker_queue_mutex_);
+    if (worker_queue_.empty()) {
 
-	    tasks_available_.wait(lock);
-	}
+      tasks_available_.wait(lock);
+    }
 
-	if (!running_.load())
-	  break;
+    if (!running_.load())
+      break;
 
-	if (worker_queue_.empty())
-	  continue;
-	Worker<T> task = std::move(worker_queue_.front());
-	worker_queue_.pop();
-	task.run();
-    }
-    current_workers_--;
+    if (worker_queue_.empty())
+      continue;
+    Worker<T> task = std::move(worker_queue_.front());
+    worker_queue_.pop();
+    task.run();
+  }
+  current_workers_--;
 
 }
-template< typename T>
- void ThreadPool<T>::start()
-{
+template<typename T>
+void ThreadPool<T>::start() {
   std::lock_guard<std::recursive_mutex> lock(manager_mutex_);
-  if (!running_)
-  {
+  if (!running_) {
     running_ = true;
     manager_thread_ = std::thread(&ThreadPool::startWorkers, this);
 
   }
 }
 
-template< typename T>
-void ThreadPool<T>::shutdown(){
+template<typename T>
+void ThreadPool<T>::shutdown() {
 
   std::lock_guard<std::recursive_mutex> lock(manager_mutex_);
-  if (running_.load())
-  {
+  if (running_.load()) {
 
     running_.store(false);
 
@@ -267,21 +282,19 @@ void ThreadPool<T>::shutdown(){
     if (manager_thread_.joinable())
       manager_thread_.join();
     {
-    std::unique_lock<std::mutex> lock(worker_queue_mutex_);
-    thread_queue_.clear();
-    current_workers_ = 0;
-    while(!worker_queue_.empty())
-      worker_queue_.pop();
+      std::unique_lock<std::mutex> lock(worker_queue_mutex_);
+      thread_queue_.clear();
+      current_workers_ = 0;
+      while (!worker_queue_.empty())
+        worker_queue_.pop();
     }
   }
 }
 
-
 } /* namespace utils */
 } /* namespace minifi */
 } /* namespace nifi */
 } /* namespace apache */
 } /* namespace org */
 
-
 #endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/Configure.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/Configure.cpp b/libminifi/src/Configure.cpp
index 95562c3..f35e88a 100644
--- a/libminifi/src/Configure.cpp
+++ b/libminifi/src/Configure.cpp
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 #include "properties/Configure.h"
+#include <cstdlib>
 #include <string>
 #include "utils/StringUtils.h"
 #include "core/Core.h"
@@ -25,8 +26,10 @@ namespace apache {
 namespace nifi {
 namespace minifi {
 
+const char *Configure::nifi_default_directory = "nifi.default.directory";
 const char *Configure::nifi_flow_configuration_file =
     "nifi.flow.configuration.file";
+const char *Configure::nifi_flow_engine_threads = "nifi.flow.engine.threads";
 const char *Configure::nifi_administrative_yield_duration =
     "nifi.administrative.yield.duration";
 const char *Configure::nifi_bored_yield_duration = "nifi.bored.yield.duration";
@@ -82,6 +85,17 @@ bool Configure::get(std::string key, std::string &value) {
   }
 }
 
+int Configure::getInt(const std::string &key, int default_value) {
+  std::lock_guard<std::mutex> lock(mutex_);
+  auto it = properties_.find(key);
+
+  if (it != properties_.end()) {
+    return std::atol(it->second.c_str());
+  } else {
+    return default_value;
+  }
+}
+
 // Parse one line in configure file like key=value
 void Configure::parseConfigureFileLine(char *buf) {
   char *line = buf;
@@ -124,8 +138,7 @@ void Configure::loadConfigureFile(const char *fileName) {
   if (fileName) {
     // perform a naive determination if this is a relative path
     if (fileName[0] != '/') {
-      adjustedFilename = adjustedFilename + getHome() + "/"
-          + fileName;
+      adjustedFilename = adjustedFilename + getHome() + "/" + fileName;
     } else {
       adjustedFilename += fileName;
     }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/EventDrivenSchedulingAgent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/EventDrivenSchedulingAgent.cpp b/libminifi/src/EventDrivenSchedulingAgent.cpp
index cbb60ea..fa2171b 100644
--- a/libminifi/src/EventDrivenSchedulingAgent.cpp
+++ b/libminifi/src/EventDrivenSchedulingAgent.cpp
@@ -44,10 +44,10 @@ void EventDrivenSchedulingAgent::run(
       // Honor the yield
       std::this_thread::sleep_for(
           std::chrono::milliseconds(processor->getYieldTime()));
-    } else if (shouldYield && this->_boredYieldDuration > 0) {
+    } else if (shouldYield && this->bored_yield_duration_ > 0) {
       // No work to do or need to apply back pressure
       std::this_thread::sleep_for(
-          std::chrono::milliseconds(this->_boredYieldDuration));
+          std::chrono::milliseconds(this->bored_yield_duration_));
     }
 
     // Block until work is available

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/FlowController.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/FlowController.cpp b/libminifi/src/FlowController.cpp
index 6785a9d..5f6e014 100644
--- a/libminifi/src/FlowController.cpp
+++ b/libminifi/src/FlowController.cpp
@@ -37,6 +37,7 @@
 #include "core/ProcessGroup.h"
 #include "utils/StringUtils.h"
 #include "core/Core.h"
+#include "core/controller/ControllerServiceProvider.h"
 #include "core/repository/FlowFileRepository.h"
 
 namespace org {
@@ -52,7 +53,8 @@ FlowController::FlowController(
     std::shared_ptr<Configure> configure,
     std::unique_ptr<core::FlowConfiguration> flow_configuration,
     const std::string name, bool headless_mode)
-    : CoreComponent(core::getClassName<FlowController>()),
+    : core::controller::ControllerServiceProvider(
+          core::getClassName<FlowController>()),
       root_(nullptr),
       max_timer_driven_threads_(0),
       max_event_driven_threads_(0),
@@ -61,14 +63,20 @@ FlowController::FlowController(
       provenance_repo_(provenance_repo),
       flow_file_repo_(flow_file_repo),
       protocol_(0),
-      _timerScheduler(provenance_repo_, configure),
-      _eventScheduler(provenance_repo_, configure),
-      flow_configuration_(std::move(flow_configuration)) {
+      controller_service_map_(
+          std::make_shared<core::controller::ControllerServiceMap>()),
+      timer_scheduler_(nullptr),
+      event_scheduler_(nullptr),
+      controller_service_provider_(nullptr),
+      flow_configuration_(std::move(flow_configuration)),
+      configuration_(configure) {
   if (provenance_repo == nullptr)
     throw std::runtime_error("Provenance Repo should not be null");
   if (flow_file_repo == nullptr)
     throw std::runtime_error("Flow Repo should not be null");
-
+  if (IsNullOrEmpty(configuration_)) {
+    throw std::runtime_error("Must supply a configuration.");
+  }
   uuid_generate(uuid_);
   setUUID(uuid_);
 
@@ -80,14 +88,14 @@ FlowController::FlowController(
   max_timer_driven_threads_ = DEFAULT_MAX_TIMER_DRIVEN_THREAD;
   running_ = false;
   initialized_ = false;
-  root_ = NULL;
+  root_ = nullptr;
 
   protocol_ = new FlowControlProtocol(this, configure);
 
   if (!headless_mode) {
     std::string rawConfigFileString;
     configure->get(Configure::nifi_flow_configuration_file,
-                    rawConfigFileString);
+                   rawConfigFileString);
 
     if (!rawConfigFileString.empty()) {
       configuration_filename_ = rawConfigFileString;
@@ -162,15 +170,15 @@ void FlowController::stop(bool force) {
     running_ = false;
 
     logger_->log_info("Stop Flow Controller");
-    this->_timerScheduler.stop();
-    this->_eventScheduler.stop();
+    this->timer_scheduler_->stop();
+    this->event_scheduler_->stop();
     this->flow_file_repo_->stop();
     this->provenance_repo_->stop();
     // Wait for sometime for thread stop
     std::this_thread::sleep_for(std::chrono::milliseconds(1000));
     if (this->root_)
-      this->root_->stopProcessing(&this->_timerScheduler,
-                                  &this->_eventScheduler);
+      this->root_->stopProcessing(this->timer_scheduler_.get(),
+                                  this->event_scheduler_.get());
   }
 }
 
@@ -221,14 +229,40 @@ void FlowController::load() {
     stop(true);
   }
   if (!initialized_) {
+    logger_->log_info("Initializing timers");
+    if (nullptr == timer_scheduler_) {
+      timer_scheduler_ = std::make_shared<TimerDrivenSchedulingAgent>(
+          std::static_pointer_cast<core::controller::ControllerServiceProvider>(
+              shared_from_this()),
+          provenance_repo_, configuration_);
+    }
+    if (nullptr == event_scheduler_) {
+      event_scheduler_ = std::make_shared<EventDrivenSchedulingAgent>(
+          std::static_pointer_cast<core::controller::ControllerServiceProvider>(
+              shared_from_this()),
+          provenance_repo_, configuration_);
+    }
     logger_->log_info("Load Flow Controller from file %s",
                       configuration_filename_.c_str());
 
-    this->root_ = flow_configuration_->getRoot(configuration_filename_);
+    this->root_ = std::shared_ptr<core::ProcessGroup>(
+        flow_configuration_->getRoot(configuration_filename_));
+
+    logger_->log_info("Loaded root processor Group");
+
+    controller_service_provider_ = flow_configuration_
+        ->getControllerServiceProvider();
 
+    std::static_pointer_cast<core::controller::StandardControllerServiceProvider>(
+        controller_service_provider_)->setRootGroup(root_);
+    std::static_pointer_cast<core::controller::StandardControllerServiceProvider>(
+        controller_service_provider_)->setSchedulingAgent(
+        std::static_pointer_cast<minifi::SchedulingAgent>(event_scheduler_));
+
+    logger_->log_info("Loaded controller service provider");
     // Load Flow File from Repo
     loadFlowRepo();
-
+    logger_->log_info("Loaded flow repository");
     initialized_ = true;
   }
 }
@@ -255,15 +289,20 @@ void FlowController::reload(std::string yamlFile) {
 }
 
 void FlowController::loadFlowRepo() {
-  if (this->flow_file_repo_) {
+  if (this->flow_file_repo_ != nullptr) {
+    logger_->log_debug("Getting connection map");
     std::map<std::string, std::shared_ptr<Connection>> connectionMap;
     if (this->root_ != nullptr) {
       this->root_->getConnections(connectionMap);
     }
+    logger_->log_debug("Number of connections from connectionMap %d",
+                       connectionMap.size());
     auto rep = std::static_pointer_cast<core::repository::FlowFileRepository>(
         flow_file_repo_);
     rep->setConnectionMap(connectionMap);
     flow_file_repo_->loadComponent();
+  } else {
+    logger_->log_debug("Flow file repository is not set");
   }
 }
 
@@ -276,11 +315,12 @@ bool FlowController::start() {
   } else {
     if (!running_) {
       logger_->log_info("Starting Flow Controller");
-      this->_timerScheduler.start();
-      this->_eventScheduler.start();
+      controller_service_provider_->enableAllControllerServices();
+      this->timer_scheduler_->start();
+      this->event_scheduler_->start();
       if (this->root_ != nullptr) {
-        this->root_->startProcessing(&this->_timerScheduler,
-                                     &this->_eventScheduler);
+        this->root_->startProcessing(this->timer_scheduler_.get(),
+                                     this->event_scheduler_.get());
       }
       running_ = true;
       this->protocol_->start();
@@ -291,6 +331,163 @@ bool FlowController::start() {
     return true;
   }
 }
+/**
+ * Controller Service functions
+ *
+ */
+
+/**
+ * Creates a controller service through the controller service provider impl.
+ * @param type class name
+ * @param id service identifier
+ * @param firstTimeAdded first time this CS was added
+ */
+std::shared_ptr<core::controller::ControllerServiceNode> FlowController::createControllerService(
+    const std::string &type, const std::string &id,
+    bool firstTimeAdded) {
+  return controller_service_provider_->createControllerService(type, id,
+                                                               firstTimeAdded);
+}
+
+/**
+ * controller service provider
+ */
+/**
+ * removes controller service
+ * @param serviceNode service node to be removed.
+ */
+
+void FlowController::removeControllerService(
+    const std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+  controller_map_->removeControllerService(serviceNode);
+}
+
+/**
+ * Enables the controller service services
+ * @param serviceNode service node which will be disabled, along with linked services.
+ */
+void FlowController::enableControllerService(
+    std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+  return controller_service_provider_->enableControllerService(serviceNode);
+}
+
+/**
+ * Enables controller services
+ * @param serviceNoden vector of service nodes which will be enabled, along with linked services.
+ */
+void FlowController::enableControllerServices(
+    std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> serviceNodes) {
+}
+
+/**
+ * Disables controller services
+ * @param serviceNode service node which will be disabled, along with linked services.
+ */
+void FlowController::disableControllerService(
+    std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+  controller_service_provider_->disableControllerService(serviceNode);
+}
+
+/**
+ * Gets all controller services.
+ */
+std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> FlowController::getAllControllerServices() {
+  return controller_service_provider_->getAllControllerServices();
+}
+
+/**
+ * Gets controller service node specified by <code>id</code>
+ * @param id service identifier
+ * @return shared pointer to the controller service node or nullptr if it does not exist.
+ */
+std::shared_ptr<core::controller::ControllerServiceNode> FlowController::getControllerServiceNode(
+    const std::string &id) {
+  return controller_service_provider_->getControllerServiceNode(id);
+}
+
+void FlowController::verifyCanStopReferencingComponents(
+    std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+}
+
+/**
+ * Unschedules referencing components.
+ */
+std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> FlowController::unscheduleReferencingComponents(
+    std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+  return controller_service_provider_->unscheduleReferencingComponents(
+      serviceNode);
+}
+
+/**
+ * Verify can disable referencing components
+ * @param serviceNode service node whose referenced components will be scheduled.
+ */
+void FlowController::verifyCanDisableReferencingServices(
+    std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+  controller_service_provider_->verifyCanDisableReferencingServices(
+      serviceNode);
+}
+
+/**
+ * Disables referencing components
+ * @param serviceNode service node whose referenced components will be scheduled.
+ */
+std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> FlowController::disableReferencingServices(
+    std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+  return controller_service_provider_->disableReferencingServices(serviceNode);
+}
+
+/**
+ * Verify can enable referencing components
+ * @param serviceNode service node whose referenced components will be scheduled.
+ */
+void FlowController::verifyCanEnableReferencingServices(
+    std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+  controller_service_provider_->verifyCanEnableReferencingServices(serviceNode);
+}
+
+/**
+ * Determines if the controller service specified by identifier is enabled.
+ */
+bool FlowController::isControllerServiceEnabled(const std::string &identifier) {
+  return controller_service_provider_->isControllerServiceEnabled(identifier);
+}
+
+/**
+ * Enables referencing components
+ * @param serviceNode service node whose referenced components will be scheduled.
+ */
+std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> FlowController::enableReferencingServices(
+    std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+  return controller_service_provider_->enableReferencingServices(serviceNode);
+}
+
+/**
+ * Schedules referencing components
+ * @param serviceNode service node whose referenced components will be scheduled.
+ */
+std::vector<std::shared_ptr<core::controller::ControllerServiceNode>> FlowController::scheduleReferencingComponents(
+    std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+  return controller_service_provider_->scheduleReferencingComponents(
+      serviceNode);
+}
+
+/**
+ * Returns controller service components referenced by serviceIdentifier from the embedded
+ * controller service provider;
+ */
+std::shared_ptr<core::controller::ControllerService> FlowController::getControllerServiceForComponent(
+    const std::string &serviceIdentifier, const std::string &componentId) {
+  return controller_service_provider_->getControllerServiceForComponent(
+      serviceIdentifier, componentId);
+}
+
+/**
+ * Enables all controller services for the provider.
+ */
+void FlowController::enableAllControllerServices() {
+  controller_service_provider_->enableAllControllerServices();
+}
 
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/SchedulingAgent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/SchedulingAgent.cpp b/libminifi/src/SchedulingAgent.cpp
index d69ba00..fc979fd 100644
--- a/libminifi/src/SchedulingAgent.cpp
+++ b/libminifi/src/SchedulingAgent.cpp
@@ -20,6 +20,7 @@
 #include "SchedulingAgent.h"
 #include <chrono>
 #include <thread>
+#include <utility>
 #include <memory>
 #include <iostream>
 #include "Exception.h"
@@ -39,6 +40,36 @@ bool SchedulingAgent::hasWorkToDo(std::shared_ptr<core::Processor> processor) {
     return false;
 }
 
+void SchedulingAgent::enableControllerService(
+    std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+
+  logger_->log_trace("Enabling CSN in SchedulingAgent %s",
+                     serviceNode->getName());
+  // reference the enable function from serviceNode
+  std::function<bool()> f_ex = [serviceNode] {
+    return serviceNode->enable();
+  };
+  // create a functor that will be submitted to the thread pool.
+  utils::Worker<bool> functor(f_ex);
+  // move the functor into the thread pool. While a future is returned
+  // we aren't terribly concerned with the result.
+  component_lifecycle_thread_pool_.execute(std::move(functor));
+}
+
+void SchedulingAgent::disableControllerService(
+    std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
+
+  // reference the disable function from serviceNode
+  std::function<bool()> f_ex = [serviceNode] {
+    return serviceNode->disable();
+  };
+  // create a functor that will be submitted to the thread pool.
+  utils::Worker<bool> functor(f_ex);
+  // move the functor into the thread pool. While a future is returned
+  // we aren't terribly concerned with the result.
+  component_lifecycle_thread_pool_.execute(std::move(functor));
+}
+
 bool SchedulingAgent::hasTooMuchOutGoing(
     std::shared_ptr<core::Processor> processor) {
   return processor->flowFilesOutGoingFull();
@@ -71,11 +102,11 @@ bool SchedulingAgent::onTrigger(std::shared_ptr<core::Processor> processor,
     processor->decrementActiveTask();
   } catch (std::exception &exception) {
     logger_->log_debug("Caught Exception %s", exception.what());
-    processor->yield(_administrativeYieldDuration);
+    processor->yield(admin_yield_duration_);
     processor->decrementActiveTask();
   } catch (...) {
     logger_->log_debug("Caught Exception during SchedulingAgent::onTrigger");
-    processor->yield(_administrativeYieldDuration);
+    processor->yield(admin_yield_duration_);
     processor->decrementActiveTask();
   }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/ThreadedSchedulingAgent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/ThreadedSchedulingAgent.cpp b/libminifi/src/ThreadedSchedulingAgent.cpp
index 65e7531..7e9bb03 100644
--- a/libminifi/src/ThreadedSchedulingAgent.cpp
+++ b/libminifi/src/ThreadedSchedulingAgent.cpp
@@ -39,30 +39,28 @@ void ThreadedSchedulingAgent::schedule(
     std::shared_ptr<core::Processor> processor) {
   std::lock_guard<std::mutex> lock(mutex_);
 
-  _administrativeYieldDuration = 0;
+  admin_yield_duration_ = 0;
   std::string yieldValue;
 
   if (configure_->get(Configure::nifi_administrative_yield_duration,
                       yieldValue)) {
     core::TimeUnit unit;
-    if (core::Property::StringToTime(yieldValue, _administrativeYieldDuration,
-                                     unit)
-        && core::Property::ConvertTimeUnitToMS(_administrativeYieldDuration,
-                                               unit,
-                                               _administrativeYieldDuration)) {
+    if (core::Property::StringToTime(yieldValue, admin_yield_duration_, unit)
+        && core::Property::ConvertTimeUnitToMS(admin_yield_duration_, unit,
+                                               admin_yield_duration_)) {
       logger_->log_debug("nifi_administrative_yield_duration: [%d] ms",
-                         _administrativeYieldDuration);
+                         admin_yield_duration_);
     }
   }
 
-  _boredYieldDuration = 0;
+  bored_yield_duration_ = 0;
   if (configure_->get(Configure::nifi_bored_yield_duration, yieldValue)) {
     core::TimeUnit unit;
-    if (core::Property::StringToTime(yieldValue, _boredYieldDuration, unit)
-        && core::Property::ConvertTimeUnitToMS(_boredYieldDuration, unit,
-                                               _boredYieldDuration)) {
+    if (core::Property::StringToTime(yieldValue, bored_yield_duration_, unit)
+        && core::Property::ConvertTimeUnitToMS(bored_yield_duration_, unit,
+                                               bored_yield_duration_)) {
       logger_->log_debug("nifi_bored_yield_duration: [%d] ms",
-                         _boredYieldDuration);
+                         bored_yield_duration_);
     }
   }
 
@@ -82,8 +80,8 @@ void ThreadedSchedulingAgent::schedule(
   }
 
   core::ProcessorNode processor_node(processor);
-  auto processContext = std::make_shared<core::ProcessContext>(processor_node,
-                                                               repo_);
+  auto processContext = std::make_shared<core::ProcessContext>(
+      processor_node, controller_service_provider_, repo_);
   auto sessionFactory = std::make_shared<core::ProcessSessionFactory>(
       processContext.get());
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/TimerDrivenSchedulingAgent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/TimerDrivenSchedulingAgent.cpp b/libminifi/src/TimerDrivenSchedulingAgent.cpp
index 8d10658..8610e64 100644
--- a/libminifi/src/TimerDrivenSchedulingAgent.cpp
+++ b/libminifi/src/TimerDrivenSchedulingAgent.cpp
@@ -40,10 +40,10 @@ void TimerDrivenSchedulingAgent::run(
       // Honor the yield
       std::this_thread::sleep_for(
           std::chrono::milliseconds(processor->getYieldTime()));
-    } else if (shouldYield && this->_boredYieldDuration > 0) {
+    } else if (shouldYield && this->bored_yield_duration_ > 0) {
       // No work to do or need to apply back pressure
       std::this_thread::sleep_for(
-          std::chrono::milliseconds(this->_boredYieldDuration));
+          std::chrono::milliseconds(this->bored_yield_duration_));
     }
     std::this_thread::sleep_for(
         std::chrono::nanoseconds(processor->getSchedulingPeriodNano()));


[3/5] nifi-minifi-cpp git commit: MINIFI-226: Add controller services capabilities along with unit tests Fix test failures Update Travis YML Update readme to link to MiNiFi licensing information

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/controllers/SSLContextService.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/controllers/SSLContextService.cpp b/libminifi/src/controllers/SSLContextService.cpp
new file mode 100644
index 0000000..51a7cc4
--- /dev/null
+++ b/libminifi/src/controllers/SSLContextService.cpp
@@ -0,0 +1,226 @@
+/**
+ *
+ * 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.
+ */
+
+#include "controllers/SSLContextService.h"
+#include <openssl/err.h>
+#include <openssl/ssl.h>
+#include <string>
+#include <memory>
+#include <set>
+#include "core/Property.h"
+#include "io/validation.h"
+#include "properties/Configure.h"
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace controllers {
+
+void SSLContextService::initialize() {
+  if (initialized_)
+    return;
+
+  std::lock_guard<std::mutex> lock(initialization_mutex_);
+
+  ControllerService::initialize();
+
+  initializeTLS();
+}
+
+std::unique_ptr<SSLContext> SSLContextService::createSSLContext() {
+  SSL_library_init();
+  const SSL_METHOD *method;
+
+  OpenSSL_add_all_algorithms();
+  SSL_load_error_strings();
+  method = TLSv1_2_client_method();
+  SSL_CTX *ctx = SSL_CTX_new(method);
+
+  if (SSL_CTX_use_certificate_file(ctx, certificate.c_str(), SSL_FILETYPE_PEM)
+      <= 0) {
+    logger_->log_error("Could not create load certificate, error : %s",
+                       std::strerror(errno));
+    return nullptr;
+  }
+  if (!IsNullOrEmpty(passphrase_)) {
+    SSL_CTX_set_default_passwd_cb_userdata(ctx, &passphrase_);
+    SSL_CTX_set_default_passwd_cb(ctx, pemPassWordCb);
+  }
+
+  int retp = SSL_CTX_use_PrivateKey_file(ctx, private_key_.c_str(),
+                                         SSL_FILETYPE_PEM);
+  if (retp != 1) {
+    logger_->log_error("Could not create load private key,%i on %s error : %s",
+                       retp, private_key_, std::strerror(errno));
+    return nullptr;
+  }
+
+  if (!SSL_CTX_check_private_key(ctx)) {
+    logger_->log_error(
+        "Private key does not match the public certificate, error : %s",
+        std::strerror(errno));
+    return nullptr;
+  }
+
+  retp = SSL_CTX_load_verify_locations(ctx, ca_certificate_.c_str(), 0);
+  if (retp == 0) {
+    logger_->log_error("Can not load CA certificate, Exiting, error : %s",
+                       std::strerror(errno));
+  }
+  return std::unique_ptr<SSLContext>(new SSLContext(ctx));
+}
+
+const std::string &SSLContextService::getCertificateFile() {
+  std::lock_guard<std::mutex> lock(initialization_mutex_);
+  return certificate;
+}
+
+const std::string &SSLContextService::getPassphrase() {
+  std::lock_guard<std::mutex> lock(initialization_mutex_);
+  return passphrase_;
+}
+
+const std::string &SSLContextService::getPassphraseFile() {
+  std::lock_guard<std::mutex> lock(initialization_mutex_);
+  return passphrase_file_;
+}
+
+const std::string &SSLContextService::getPrivateKeyFile() {
+  std::lock_guard<std::mutex> lock(initialization_mutex_);
+  return private_key_;
+}
+
+const std::string &SSLContextService::getCACertificate() {
+  std::lock_guard<std::mutex> lock(initialization_mutex_);
+  return ca_certificate_;
+}
+
+void SSLContextService::onEnable() {
+  valid_ = true;
+  core::Property property("Client Certificate", "Client Certificate");
+  core::Property privKey("Private Key", "Private Key file");
+  core::Property passphrase_prop(
+      "Passphrase", "Client passphrase. Either a file or unencrypted text");
+  core::Property caCert("CA Certificate", "CA certificate file");
+  std::string default_dir;
+  if (nullptr != configuration_)
+  configuration_->get(Configure::nifi_default_directory, default_dir);
+
+  logger_->log_trace("onEnable()");
+
+  if (getProperty(property.getName(), certificate)
+      && getProperty(privKey.getName(), private_key_)) {
+    logger_->log_error(
+        "Certificate and Private Key PEM file not configured, error: %s.",
+        std::strerror(errno));
+
+    std::ifstream cert_file(certificate);
+    std::ifstream priv_file(private_key_);
+    if (!cert_file.good()) {
+      logger_->log_info("%s not good", certificate);
+      std::string test_cert = default_dir + certificate;
+      std::ifstream cert_file_test(test_cert);
+      if (cert_file_test.good()) {
+        certificate = test_cert;
+        logger_->log_debug("%s now good", certificate);
+      } else {
+        logger_->log_debug("%s still not good", test_cert);
+        valid_ = false;
+      }
+      cert_file_test.close();
+    }
+
+    if (!priv_file.good()) {
+      std::string test_priv = default_dir + private_key_;
+      std::ifstream private_file_test(test_priv);
+      if (private_file_test.good()) {
+        private_key_ = test_priv;
+      } else {
+        valid_ = false;
+      }
+      private_file_test.close();
+    }
+    cert_file.close();
+    priv_file.close();
+
+  } else {
+    logger_->log_debug("Certificate empty");
+  }
+  if (!getProperty(passphrase_prop.getName(), passphrase_)) {
+    logger_->log_debug("No pass phrase for %s", certificate);
+  } else {
+    std::ifstream passphrase_file(passphrase_);
+    if (passphrase_file.good()) {
+      passphrase_file_ = passphrase_;
+      // we should read it from the file
+      passphrase_.assign((std::istreambuf_iterator<char>(passphrase_file)),
+                         std::istreambuf_iterator<char>());
+    } else {
+      std::string test_passphrase = default_dir + passphrase_;
+      std::ifstream passphrase_file_test(test_passphrase);
+      if (passphrase_file_test.good()) {
+        passphrase_ = test_passphrase;
+        passphrase_file_ = test_passphrase;
+        passphrase_.assign(
+            (std::istreambuf_iterator<char>(passphrase_file_test)),
+            std::istreambuf_iterator<char>());
+      } else {
+        valid_ = false;
+      }
+      passphrase_file_test.close();
+    }
+    passphrase_file.close();
+  }
+  // load CA certificates
+  if (!getProperty(caCert.getName(), ca_certificate_)) {
+    logger_->log_error("Can not load CA certificate.");
+  } else {
+    std::ifstream cert_file(ca_certificate_);
+    if (!cert_file.good()) {
+      std::string test_ca_cert = default_dir + ca_certificate_;
+      std::ifstream ca_cert_file_file_test(test_ca_cert);
+      if (ca_cert_file_file_test.good()) {
+        ca_certificate_ = test_ca_cert;
+      } else {
+        valid_ = false;
+      }
+      ca_cert_file_file_test.close();
+    }
+    cert_file.close();
+  }
+}
+
+void SSLContextService::initializeTLS() {
+  core::Property property("Client Certificate", "Client Certificate");
+  core::Property privKey("Private Key", "Private Key file");
+  core::Property passphrase_prop(
+      "Passphrase", "Client passphrase. Either a file or unencrypted text");
+  core::Property caCert("CA Certificate", "CA certificate file");
+  std::set<core::Property> supportedProperties;
+  supportedProperties.insert(property);
+  supportedProperties.insert(privKey);
+  supportedProperties.insert(passphrase_prop);
+  supportedProperties.insert(caCert);
+  setSupportedProperties(supportedProperties);
+}
+
+} /* namespace controllers */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/core/ClassLoader.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ClassLoader.cpp b/libminifi/src/core/ClassLoader.cpp
new file mode 100644
index 0000000..77c4a85
--- /dev/null
+++ b/libminifi/src/core/ClassLoader.cpp
@@ -0,0 +1,70 @@
+/**
+ * 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.
+ */
+
+#include "core/ClassLoader.h"
+#include <sys/mman.h>
+#include <memory>
+#include <string>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+ClassLoader &ClassLoader::getDefaultClassLoader() {
+  static ClassLoader ret;
+  // populate ret
+  return ret;
+}
+uint16_t ClassLoader::registerResource(const std::string &resource) {
+  void* resource_ptr = dlopen(resource.c_str(), RTLD_LAZY);
+  if (!resource_ptr) {
+    logger_->log_error("Cannot load library: %s", dlerror());
+    return RESOURCE_FAILURE;
+  } else {
+    std::lock_guard<std::mutex> lock(internal_mutex_);
+    dl_handles_.push_back(resource_ptr);
+  }
+
+  // reset errors
+  dlerror();
+
+  // load the symbols
+  createFactory* create_factory_func = reinterpret_cast<createFactory*>(dlsym(
+      resource_ptr, "createFactory"));
+  const char* dlsym_error = dlerror();
+  if (dlsym_error) {
+    logger_->log_error("Cannot load library: %s", dlsym_error);
+    return RESOURCE_FAILURE;
+  }
+
+  ObjectFactory *factory = create_factory_func();
+
+  std::lock_guard<std::mutex> lock(internal_mutex_);
+
+  loaded_factories_[factory->getClassName()] = std::unique_ptr<ObjectFactory>(
+      factory);
+
+  return RESOURCE_SUCCESS;
+}
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/core/ConfigurableComponent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ConfigurableComponent.cpp b/libminifi/src/core/ConfigurableComponent.cpp
index fa5ff7d..cf2089e 100644
--- a/libminifi/src/core/ConfigurableComponent.cpp
+++ b/libminifi/src/core/ConfigurableComponent.cpp
@@ -43,6 +43,20 @@ ConfigurableComponent::ConfigurableComponent(
 ConfigurableComponent::~ConfigurableComponent() {
 }
 
+bool ConfigurableComponent::getProperty(const std::string &name,
+                                        Property &prop) {
+  std::lock_guard<std::mutex> lock(configuration_mutex_);
+
+  auto &&it = properties_.find(name);
+
+  if (it != properties_.end()) {
+    prop = it->second;
+    return true;
+  } else {
+    return false;
+  }
+}
+
 /**
  * Get property using the provided name.
  * @param name property name.
@@ -54,12 +68,11 @@ bool ConfigurableComponent::getProperty(const std::string name,
   std::lock_guard<std::mutex> lock(configuration_mutex_);
 
   auto &&it = properties_.find(name);
-
   if (it != properties_.end()) {
     Property item = it->second;
     value = item.getValue();
-    my_logger_->log_info("Processor %s property name %s value %s", name.c_str(),
-                         item.getName().c_str(), value.c_str());
+    my_logger_->log_info("Processor %s property name %s value %s", name,
+                         item.getName(), value);
     return true;
   } else {
     return false;
@@ -92,6 +105,29 @@ bool ConfigurableComponent::setProperty(const std::string name,
  * Sets the property using the provided name
  * @param property name
  * @param value property value.
+ * @return result of setting property.
+ */
+bool ConfigurableComponent::updateProperty(const std::string &name,
+                                           const std::string &value) {
+  std::lock_guard<std::mutex> lock(configuration_mutex_);
+  auto &&it = properties_.find(name);
+
+  if (it != properties_.end()) {
+    Property item = it->second;
+    item.addValue(value);
+    properties_[item.getName()] = item;
+    my_logger_->log_info("Component %s property name %s value %s", name.c_str(),
+                         item.getName().c_str(), value.c_str());
+    return true;
+  } else {
+    return false;
+  }
+}
+
+/**
+ * Sets the property using the provided name
+ * @param property name
+ * @param value property value.
  * @return whether property was set or not
  */
 bool ConfigurableComponent::setProperty(Property &prop, std::string value) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/core/ConfigurationFactory.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ConfigurationFactory.cpp b/libminifi/src/core/ConfigurationFactory.cpp
index 4ccead2..6aa42e3 100644
--- a/libminifi/src/core/ConfigurationFactory.cpp
+++ b/libminifi/src/core/ConfigurationFactory.cpp
@@ -21,6 +21,7 @@
 #include <memory>
 #include <algorithm>
 #include <set>
+#include "core/Core.h"
 #include "core/ConfigurationFactory.h"
 #include "core/FlowConfiguration.h"
 #include "io/StreamFactory.h"
@@ -53,17 +54,20 @@ std::unique_ptr<core::FlowConfiguration> createFlowConfiguration(
     if (class_name_lc == "flowconfiguration") {
       // load the base configuration.
       return std::unique_ptr<core::FlowConfiguration>(
-          new core::FlowConfiguration(repo, flow_file_repo, stream_factory, path));
+          new core::FlowConfiguration(repo, flow_file_repo, stream_factory,
+                                      configure, path));
 
     } else if (class_name_lc == "yamlconfiguration") {
       // only load if the class is defined.
       return std::unique_ptr<core::FlowConfiguration>(
-          instantiate<core::YamlConfiguration>(repo, flow_file_repo, stream_factory, path));
+          instantiate<core::YamlConfiguration>(repo, flow_file_repo,
+                                               stream_factory, configure, path));
 
     } else {
       if (fail_safe) {
         return std::unique_ptr<core::FlowConfiguration>(
-            new core::FlowConfiguration(repo, flow_file_repo, stream_factory, path));
+            new core::FlowConfiguration(repo, flow_file_repo, stream_factory,
+                                        configure, path));
       } else {
         throw std::runtime_error(
             "Support for the provided configuration class could not be found");
@@ -72,7 +76,8 @@ std::unique_ptr<core::FlowConfiguration> createFlowConfiguration(
   } catch (const std::runtime_error &r) {
     if (fail_safe) {
       return std::unique_ptr<core::FlowConfiguration>(
-          new core::FlowConfiguration(repo, flow_file_repo, stream_factory, path));
+          new core::FlowConfiguration(repo, flow_file_repo, stream_factory,
+                                      configure, path));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/core/Connectable.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Connectable.cpp b/libminifi/src/core/Connectable.cpp
index e234a06..58ae6d4 100644
--- a/libminifi/src/core/Connectable.cpp
+++ b/libminifi/src/core/Connectable.cpp
@@ -148,9 +148,9 @@ std::set<std::shared_ptr<Connectable>> Connectable::getOutGoingConnections(
     std::string relationship) {
   std::set<std::shared_ptr<Connectable>> empty;
 
-  auto &&it = _outGoingConnections.find(relationship);
-  if (it != _outGoingConnections.end()) {
-    return _outGoingConnections[relationship];
+  auto &&it = out_going_connections_.find(relationship);
+  if (it != out_going_connections_.end()) {
+    return out_going_connections_[relationship];
   } else {
     return empty;
   }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/core/FlowConfiguration.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/FlowConfiguration.cpp b/libminifi/src/core/FlowConfiguration.cpp
index 90058d2..1ed9176 100644
--- a/libminifi/src/core/FlowConfiguration.cpp
+++ b/libminifi/src/core/FlowConfiguration.cpp
@@ -19,6 +19,7 @@
 #include "core/FlowConfiguration.h"
 #include <memory>
 #include <string>
+#include "core/ClassLoader.h"
 
 namespace org {
 namespace apache {
@@ -31,54 +32,12 @@ FlowConfiguration::~FlowConfiguration() {
 
 std::shared_ptr<core::Processor> FlowConfiguration::createProcessor(
     std::string name, uuid_t uuid) {
-  std::shared_ptr<core::Processor> processor = nullptr;
-  if (name
-      == org::apache::nifi::minifi::processors::GenerateFlowFile::ProcessorName) {
-    processor = std::make_shared<
-        org::apache::nifi::minifi::processors::GenerateFlowFile>(name, uuid);
-  } else if (name
-      == org::apache::nifi::minifi::processors::LogAttribute::ProcessorName) {
-    processor = std::make_shared<
-        org::apache::nifi::minifi::processors::LogAttribute>(name, uuid);
-  } else if (name
-      == org::apache::nifi::minifi::processors::GetFile::ProcessorName) {
-    processor =
-        std::make_shared<org::apache::nifi::minifi::processors::GetFile>(name,
-                                                                         uuid);
-  } else if (name
-      == org::apache::nifi::minifi::processors::PutFile::ProcessorName) {
-    processor =
-        std::make_shared<org::apache::nifi::minifi::processors::PutFile>(name,
-                                                                         uuid);
-  } else if (name
-      == org::apache::nifi::minifi::processors::TailFile::ProcessorName) {
-    processor =
-        std::make_shared<org::apache::nifi::minifi::processors::TailFile>(name,
-                                                                          uuid);
-  } else if (name
-      == org::apache::nifi::minifi::processors::ListenSyslog::ProcessorName) {
-    processor = std::make_shared<
-        org::apache::nifi::minifi::processors::ListenSyslog>(name, uuid);
-  } else if (name
-        == org::apache::nifi::minifi::processors::ListenHTTP::ProcessorName) {
-      processor = std::make_shared<
-          org::apache::nifi::minifi::processors::ListenHTTP>(name, uuid);
-  } else if (name
-          == org::apache::nifi::minifi::processors::InvokeHTTP::ProcessorName) {
-        processor = std::make_shared<
-            org::apache::nifi::minifi::processors::InvokeHTTP>(name, uuid);
-  } else if (name
-      == org::apache::nifi::minifi::processors::ExecuteProcess::ProcessorName) {
-    processor = std::make_shared<
-        org::apache::nifi::minifi::processors::ExecuteProcess>(name, uuid);
-  } else if (name
-      == org::apache::nifi::minifi::processors::AppendHostInfo::ProcessorName) {
-    processor = std::make_shared<
-        org::apache::nifi::minifi::processors::AppendHostInfo>(name, uuid);
-  } else {
+  auto ptr = core::ClassLoader::getDefaultClassLoader().instantiate(name, uuid);
+  if (nullptr == ptr) {
     logger_->log_error("No Processor defined for %s", name.c_str());
-    return nullptr;
   }
+  std::shared_ptr<core::Processor> processor = std::static_pointer_cast<
+      core::Processor>(ptr);
 
   // initialize the processor
   processor->initialize();
@@ -89,8 +48,10 @@ std::shared_ptr<core::Processor> FlowConfiguration::createProcessor(
 std::shared_ptr<core::Processor> FlowConfiguration::createProvenanceReportTask() {
   std::shared_ptr<core::Processor> processor = nullptr;
 
-  processor = std::make_shared<
-        org::apache::nifi::minifi::core::reporting::SiteToSiteProvenanceReportingTask>(stream_factory_);
+  processor =
+      std::make_shared<
+          org::apache::nifi::minifi::core::reporting::SiteToSiteProvenanceReportingTask>(
+          stream_factory_);
   // initialize the processor
   processor->initialize();
 
@@ -114,6 +75,15 @@ std::shared_ptr<minifi::Connection> FlowConfiguration::createConnection(
   return std::make_shared<minifi::Connection>(flow_file_repo_, name, uuid);
 }
 
+std::shared_ptr<core::controller::ControllerServiceNode> FlowConfiguration::createControllerService(
+    const std::string &class_name, const std::string &name, uuid_t uuid) {
+  std::shared_ptr<core::controller::ControllerServiceNode> controllerServicesNode =
+      service_provider_->createControllerService(class_name, name, true);
+  if (nullptr != controllerServicesNode)
+    controllerServicesNode->setUUID(uuid);
+  return controllerServicesNode;
+}
+
 } /* namespace core */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/core/ProcessGroup.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ProcessGroup.cpp b/libminifi/src/core/ProcessGroup.cpp
index 1a6e729..1b8ec3a 100644
--- a/libminifi/src/core/ProcessGroup.cpp
+++ b/libminifi/src/core/ProcessGroup.cpp
@@ -202,6 +202,22 @@ std::shared_ptr<Processor> ProcessGroup::findProcessor(uuid_t uuid) {
   return ret;
 }
 
+void ProcessGroup::addControllerService(
+    const std::string &nodeId,
+    std::shared_ptr<core::controller::ControllerServiceNode> &node) {
+  controller_service_map_.put(nodeId, node);
+}
+
+/**
+ * Find controllerservice node will search child groups until the nodeId is found.
+ * @param node node identifier
+ * @return controller service node, if it exists.
+ */
+std::shared_ptr<core::controller::ControllerServiceNode> ProcessGroup::findControllerService(
+    const std::string &nodeId) {
+  return controller_service_map_.getControllerServiceNode(nodeId);
+}
+
 std::shared_ptr<Processor> ProcessGroup::findProcessor(
     const std::string &processorName) {
   std::shared_ptr<Processor> ret = NULL;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/core/Processor.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Processor.cpp b/libminifi/src/core/Processor.cpp
index e124992..7464af2 100644
--- a/libminifi/src/core/Processor.cpp
+++ b/libminifi/src/core/Processor.cpp
@@ -114,15 +114,15 @@ bool Processor::addConnection(std::shared_ptr<Connectable> conn) {
   if (my_uuid == source_uuid) {
     std::string relationship = connection->getRelationship().getName();
     // Connection is source from the current processor
-    auto &&it = _outGoingConnections.find(relationship);
-    if (it != _outGoingConnections.end()) {
+    auto &&it = out_going_connections_.find(relationship);
+    if (it != out_going_connections_.end()) {
       // We already has connection for this relationship
       std::set<std::shared_ptr<Connectable>> existedConnection = it->second;
       if (existedConnection.find(connection) == existedConnection.end()) {
         // We do not have the same connection for this relationship yet
         existedConnection.insert(connection);
         connection->setSource(shared_from_this());
-        _outGoingConnections[relationship] = existedConnection;
+        out_going_connections_[relationship] = existedConnection;
         logger_->log_info(
             "Add connection %s into Processor %s outgoing connection for relationship %s",
             connection->getName().c_str(), name_.c_str(), relationship.c_str());
@@ -133,7 +133,7 @@ bool Processor::addConnection(std::shared_ptr<Connectable> conn) {
       std::set<std::shared_ptr<Connectable>> newConnection;
       newConnection.insert(connection);
       connection->setSource(shared_from_this());
-      _outGoingConnections[relationship] = newConnection;
+      out_going_connections_[relationship] = newConnection;
       logger_->log_info(
           "Add connection %s into Processor %s outgoing connection for relationship %s",
           connection->getName().c_str(), name_.c_str(), relationship.c_str());
@@ -178,13 +178,13 @@ void Processor::removeConnection(std::shared_ptr<Connectable> conn) {
   if (uuid_compare(uuid_, srcUUID) == 0) {
     std::string relationship = connection->getRelationship().getName();
     // Connection is source from the current processor
-    auto &&it = _outGoingConnections.find(relationship);
-    if (it == _outGoingConnections.end()) {
+    auto &&it = out_going_connections_.find(relationship);
+    if (it == out_going_connections_.end()) {
       return;
     } else {
-      if (_outGoingConnections[relationship].find(connection)
-          != _outGoingConnections[relationship].end()) {
-        _outGoingConnections[relationship].erase(connection);
+      if (out_going_connections_[relationship].find(connection)
+          != out_going_connections_[relationship].end()) {
+        out_going_connections_[relationship].erase(connection);
         connection->setSource(NULL);
         logger_->log_info(
             "Remove connection %s into Processor %s outgoing connection for relationship %s",
@@ -259,7 +259,7 @@ bool Processor::flowFilesQueued() {
 bool Processor::flowFilesOutGoingFull() {
   std::lock_guard<std::mutex> lock(mutex_);
 
-  for (auto &&connection : _outGoingConnections) {
+  for (auto &&connection : out_going_connections_) {
     // We already has connection for this relationship
     std::set<std::shared_ptr<Connectable>> existedConnection = connection.second;
     for (const auto conn : existedConnection) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/core/Property.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Property.cpp b/libminifi/src/core/Property.cpp
index aa002af..dd3a9cb 100644
--- a/libminifi/src/core/Property.cpp
+++ b/libminifi/src/core/Property.cpp
@@ -18,6 +18,7 @@
 
 #include "core/Property.h"
 #include <string>
+#include <vector>
 namespace org {
 namespace apache {
 namespace nifi {
@@ -34,11 +35,27 @@ std::string Property::getDescription() {
 }
 // Get value for the property
 std::string Property::getValue() const {
-  return value_;
+  if (!values_.empty())
+    return values_.front();
+  else
+    return "";
+}
+
+std::vector<std::string> &Property::getValues() {
+  return values_;
 }
 // Set value for the property
 void Property::setValue(std::string value) {
-  value_ = value;
+  if (!isCollection) {
+    values_.clear();
+    values_.push_back(std::string(value.c_str()));
+  } else {
+    values_.push_back(std::string(value.c_str()));
+  }
+}
+
+void Property::addValue(const std::string &value) {
+  values_.push_back(std::string(value.c_str()));
 }
 // Compare
 bool Property::operator <(const Property & right) const {
@@ -47,7 +64,8 @@ bool Property::operator <(const Property & right) const {
 
 const Property &Property::operator=(const Property &other) {
   name_ = other.name_;
-  value_ = other.value_;
+  values_ = other.values_;
+  isCollection = other.isCollection;
   return *this;
 }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/core/controller/ControllerServiceNode.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/controller/ControllerServiceNode.cpp b/libminifi/src/core/controller/ControllerServiceNode.cpp
new file mode 100644
index 0000000..12e3653
--- /dev/null
+++ b/libminifi/src/core/controller/ControllerServiceNode.cpp
@@ -0,0 +1,49 @@
+/**
+ *
+ * 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.
+ */
+
+#include "core/controller/ControllerServiceNode.h"
+#include <memory>
+#include <vector>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace controller {
+
+std::shared_ptr<ControllerService> &ControllerServiceNode::getControllerServiceImplementation() {
+  return controller_service_;
+}
+
+std::vector<std::shared_ptr<ControllerServiceNode> > &ControllerServiceNode::getLinkedControllerServices() {
+  return linked_controller_services_;
+}
+
+std::vector<std::shared_ptr<ConfigurableComponent> > &ControllerServiceNode::getLinkedComponents() {
+  return linked_components_;
+}
+
+
+
+} /* namespace controller */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/core/controller/ControllerServiceProvider.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/controller/ControllerServiceProvider.cpp b/libminifi/src/core/controller/ControllerServiceProvider.cpp
new file mode 100644
index 0000000..da5c6a1
--- /dev/null
+++ b/libminifi/src/core/controller/ControllerServiceProvider.cpp
@@ -0,0 +1,50 @@
+/**
+ *
+ * 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.
+ */
+
+#include "core/controller/ControllerServiceProvider.h"
+#include <memory>
+#include <string>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace controller {
+
+/**
+ * @param identifier of controller service
+ * @return the ControllerService that is registered with the given
+ * identifier
+ */
+std::shared_ptr<ControllerService> ControllerServiceProvider::getControllerService(
+    const std::string &identifier) {
+  auto service = controller_map_->getControllerServiceNode(identifier);
+  if (service != nullptr) {
+    return service->getControllerServiceImplementation();
+  } else {
+    return nullptr;
+  }
+}
+
+} /* namespace controller */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/core/controller/StandardControllerServiceNode.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/controller/StandardControllerServiceNode.cpp b/libminifi/src/core/controller/StandardControllerServiceNode.cpp
new file mode 100644
index 0000000..26804f6
--- /dev/null
+++ b/libminifi/src/core/controller/StandardControllerServiceNode.cpp
@@ -0,0 +1,69 @@
+/**
+ *
+ * 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.
+ */
+
+#include "core/controller/StandardControllerServiceNode.h"
+#include <memory>
+#include <mutex>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace controller {
+std::shared_ptr<core::ProcessGroup> &StandardControllerServiceNode::getProcessGroup() {
+  std::lock_guard<std::mutex> lock(mutex_);
+  return process_group_;
+}
+
+void StandardControllerServiceNode::setProcessGroup(
+    std::shared_ptr<ProcessGroup> &processGroup) {
+  std::lock_guard<std::mutex> lock(mutex_);
+  process_group_ = processGroup;
+}
+
+bool StandardControllerServiceNode::enable() {
+  Property property("Linked Services", "Referenced Controller Services");
+  controller_service_->setState(ENABLED);
+  logger_->log_trace("Enabling CSN %s", getName());
+  if (getProperty(property.getName(), property)) {
+    active = true;
+    for (auto linked_service : property.getValues()) {
+      std::shared_ptr<ControllerServiceNode> csNode = provider
+          ->getControllerServiceNode(linked_service);
+      if (nullptr != csNode) {
+        std::lock_guard<std::mutex> lock(mutex_);
+        linked_controller_services_.push_back(csNode);
+      }
+    }
+  } else {
+  }
+  std::shared_ptr<ControllerService> impl =
+      getControllerServiceImplementation();
+  if (nullptr != impl) {
+    impl->onEnable();
+  }
+  return true;
+}
+
+} /* namespace controller */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/core/yaml/YamlConfiguration.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/yaml/YamlConfiguration.cpp b/libminifi/src/core/yaml/YamlConfiguration.cpp
index b7c8246..c2c4950 100644
--- a/libminifi/src/core/yaml/YamlConfiguration.cpp
+++ b/libminifi/src/core/yaml/YamlConfiguration.cpp
@@ -21,6 +21,8 @@
 #include <string>
 #include <vector>
 #include <set>
+#include "core/reporting/SiteToSiteProvenanceReportingTask.h"
+#include "io/validation.h"
 namespace org {
 namespace apache {
 namespace nifi {
@@ -31,13 +33,14 @@ 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);
 
-  logger_->log_debug(
-      "parseRootProcessGroup: id => [%s], name => [%s]", id, flowName);
+  logger_->log_debug("parseRootProcessGroup: id => [%s], name => [%s]", id,
+                     flowName);
   std::unique_ptr<core::ProcessGroup> group =
       FlowConfiguration::createRootProcessGroup(flowName, uuid);
 
@@ -172,13 +175,16 @@ void YamlConfiguration::parseProcessorNodeYaml(
 
         if (procCfg.schedulingStrategy == "TIMER_DRIVEN") {
           processor->setSchedulingStrategy(core::TIMER_DRIVEN);
-          logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+          logger_->log_debug("setting scheduling strategy as %s",
+                             procCfg.schedulingStrategy);
         } else if (procCfg.schedulingStrategy == "EVENT_DRIVEN") {
           processor->setSchedulingStrategy(core::EVENT_DRIVEN);
-          logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+          logger_->log_debug("setting scheduling strategy as %s",
+                             procCfg.schedulingStrategy);
         } else {
           processor->setSchedulingStrategy(core::CRON_DRIVEN);
-          logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+          logger_->log_debug("setting scheduling strategy as %s",
+                             procCfg.schedulingStrategy);
         }
 
         int64_t maxConcurrentTasks;
@@ -203,11 +209,14 @@ void YamlConfiguration::parseProcessorNodeYaml(
 
         parentGroup->addProcessor(processor);
       }
+    } else {
+      throw new std::invalid_argument(
+          "Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
     }
   } else {
     throw new std::invalid_argument(
         "Cannot instantiate a MiNiFi instance without a defined "
-            "Processors configuration node.");
+        "Processors configuration node.");
   }
 }
 
@@ -323,14 +332,16 @@ void YamlConfiguration::parseProvenanceReportingYaml(
   std::shared_ptr<core::Processor> processor = nullptr;
   processor = createProvenanceReportTask();
   std::shared_ptr<core::reporting::SiteToSiteProvenanceReportingTask> reportTask =
-      std::static_pointer_cast < core::reporting::SiteToSiteProvenanceReportingTask
-      > (processor);
+      std::static_pointer_cast<
+          core::reporting::SiteToSiteProvenanceReportingTask>(processor);
 
   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>();
@@ -346,21 +357,21 @@ void YamlConfiguration::parseProvenanceReportingYaml(
   processor->setScheduledState(core::RUNNING);
 
   core::TimeUnit unit;
-  if (core::Property::StringToTime(schedulingPeriodStr, schedulingPeriod, unit) &&
-      core::Property::ConvertTimeUnitToNS(schedulingPeriod, unit, schedulingPeriod)) {
-    logger_->log_debug(
-        "ProvenanceReportingTask schedulingPeriod %d ns",
-        schedulingPeriod);
+  if (core::Property::StringToTime(schedulingPeriodStr, schedulingPeriod, unit)
+      && core::Property::ConvertTimeUnitToNS(schedulingPeriod, unit,
+                                             schedulingPeriod)) {
+    logger_->log_debug("ProvenanceReportingTask schedulingPeriod %d ns",
+                       schedulingPeriod);
     processor->setSchedulingPeriodNano(schedulingPeriod);
   }
 
   if (schedulingStrategyStr == "TIMER_DRIVEN") {
-     processor->setSchedulingStrategy(core::TIMER_DRIVEN);
-     logger_->log_debug(
-         "ProvenanceReportingTask scheduling strategy %s", schedulingStrategyStr);
+    processor->setSchedulingStrategy(core::TIMER_DRIVEN);
+    logger_->log_debug("ProvenanceReportingTask scheduling strategy %s",
+                       schedulingStrategyStr);
   } else {
     throw std::invalid_argument(
-        "Invalid scheduling strategy " +  schedulingStrategyStr);
+        "Invalid scheduling strategy " + schedulingStrategyStr);
   }
 
   reportTask->setHost(hostStr);
@@ -370,6 +381,7 @@ void YamlConfiguration::parseProvenanceReportingYaml(
     logger_->log_debug("ProvenanceReportingTask port %d", (uint16_t) lvalue);
     reportTask->setPort((uint16_t) lvalue);
   }
+
   logger_->log_debug("ProvenanceReportingTask port uuid %s", portUUIDStr);
   uuid_parse(portUUIDStr.c_str(), port_uuid);
   reportTask->setPortUUID(port_uuid);
@@ -378,9 +390,62 @@ void YamlConfiguration::parseProvenanceReportingYaml(
   }
 }
 
-void YamlConfiguration::parseConnectionYaml(
-    YAML::Node *connectionsNode,
-    core::ProcessGroup *parent) {
+void YamlConfiguration::parseControllerServices(
+    YAML::Node *controllerServicesNode) {
+  if (!IsNullOrEmpty(controllerServicesNode)) {
+    if (controllerServicesNode->IsSequence()) {
+      for (auto iter : *controllerServicesNode) {
+        YAML::Node controllerServiceNode = iter.as<YAML::Node>();
+        try {
+          checkRequiredField(&controllerServiceNode, "name",
+                             CONFIG_YAML_CONTROLLER_SERVICES_KEY);
+          checkRequiredField(&controllerServiceNode, "id",
+                             CONFIG_YAML_CONTROLLER_SERVICES_KEY);
+          checkRequiredField(&controllerServiceNode, "class",
+                             CONFIG_YAML_CONTROLLER_SERVICES_KEY);
+
+          auto name = controllerServiceNode["name"].as<std::string>();
+          auto id = controllerServiceNode["id"].as<std::string>();
+          auto type = controllerServiceNode["class"].as<std::string>();
+
+          uuid_t uuid;
+          uuid_parse(id.c_str(), uuid);
+          auto controller_service_node = createControllerService(type, name,
+                                                                 uuid);
+          if (nullptr != controller_service_node) {
+            logger_->log_debug(
+                "Created Controller Service with UUID %s and name %s", id,
+                name);
+            controller_service_node->initialize();
+            YAML::Node propertiesNode = controllerServiceNode["Properties"];
+            // we should propogate propertiets to the node and to the implementation
+            parsePropertiesNodeYaml(
+                &propertiesNode,
+                std::static_pointer_cast<core::ConfigurableComponent>(
+                    controller_service_node));
+            if (controller_service_node->getControllerServiceImplementation()
+                != nullptr) {
+              parsePropertiesNodeYaml(
+                  &propertiesNode,
+                  std::static_pointer_cast<core::ConfigurableComponent>(
+                      controller_service_node
+                          ->getControllerServiceImplementation()));
+            }
+          }
+          controller_services_->put(id, controller_service_node);
+          controller_services_->put(name, controller_service_node);
+        } catch (YAML::InvalidNode &in) {
+          throw Exception(
+              ExceptionType::GENERAL_EXCEPTION,
+              "Name, id, and class must be specified for controller services");
+        }
+      }
+    }
+  }
+}
+
+void YamlConfiguration::parseConnectionYaml(YAML::Node *connectionsNode,
+                                            core::ProcessGroup *parent) {
   if (!parent) {
     logger_->log_error("parseProcessNode: no parent group was provided");
     return;
@@ -400,15 +465,15 @@ void YamlConfiguration::parseConnectionYaml(
         std::string id = getOrGenerateId(&connectionNode);
         uuid_parse(id.c_str(), uuid);
         connection = this->createConnection(name, uuid);
-        logger_->log_debug(
-            "Created connection with UUID %s and name %s", id, name);
-
+        logger_->log_debug("Created connection with UUID %s and name %s", id,
+                           name);
 
         // Configure connection source
         checkRequiredField(&connectionNode, "source relationship name", CONFIG_YAML_CONNECTIONS_KEY);
         auto rawRelationship = connectionNode["source relationship name"].as<std::string>();
         core::Relationship relationship(rawRelationship, "");
-        logger_->log_debug("parseConnection: relationship => [%s]", rawRelationship);
+        logger_->log_debug("parseConnection: relationship => [%s]",
+                           rawRelationship);
         if (connection) {
           connection->setRelationship(relationship);
         }
@@ -462,7 +527,7 @@ void YamlConfiguration::parseConnectionYaml(
           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 "
@@ -508,13 +573,18 @@ 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", "The field 'id' is required for "
-      "the port named '" + nameStr + "' in the YAML Config. If this port "
-      "is an input port for a NiFi Remote Process Group, the port "
-      "id should match the corresponding id specified in the NiFi configuration. "
-      "This is a UUID of the format XXXXXXXX-XXXX-XXXX-XXXX-XXXXXXXXXXXX.");
+  checkRequiredField(
+      &inputPortsObj,
+      "id",
+      "The field 'id' is required for "
+          "the port named '" + nameStr
+          + "' in the YAML Config. If this port "
+              "is an input port for a NiFi Remote Process Group, the port "
+              "id should match the corresponding id specified in the NiFi configuration. "
+              "This is a UUID of the format XXXXXXXX-XXXX-XXXX-XXXX-XXXXXXXXXXXX.");
   auto portId = inputPortsObj["id"].as<std::string>();
   uuid_parse(portId.c_str(), uuid);
 
@@ -530,7 +600,9 @@ void YamlConfiguration::parsePortYaml(YAML::Node *portNode,
   // handle port properties
   YAML::Node nodeVal = portNode->as<YAML::Node>();
   YAML::Node propertiesNode = nodeVal["Properties"];
-  parsePropertiesNodeYaml(&propertiesNode, processor);
+  parsePropertiesNodeYaml(
+      &propertiesNode,
+      std::static_pointer_cast<core::ConfigurableComponent>(processor));
 
   // add processor to parent
   parent->addProcessor(processor);
@@ -548,28 +620,51 @@ void YamlConfiguration::parsePortYaml(YAML::Node *portNode,
 }
 
 void YamlConfiguration::parsePropertiesNodeYaml(
-    YAML::Node *propertiesNode, std::shared_ptr<core::Processor> processor) {
+    YAML::Node *propertiesNode,
+    std::shared_ptr<core::ConfigurableComponent> processor) {
   // Treat generically as a YAML node so we can perform inspection on entries to ensure they are populated
   for (YAML::const_iterator propsIter = propertiesNode->begin();
       propsIter != propertiesNode->end(); ++propsIter) {
     std::string propertyName = propsIter->first.as<std::string>();
     YAML::Node propertyValueNode = propsIter->second;
     if (!propertyValueNode.IsNull() && propertyValueNode.IsDefined()) {
-      std::string rawValueString = propertyValueNode.as<std::string>();
-      if (!processor->setProperty(propertyName, rawValueString)) {
-        logger_->log_warn(
-            "Received property %s with value %s but it is not one of the properties for %s",
-            propertyName,
-            rawValueString,
-            processor->getName());
+      if (propertyValueNode.IsSequence()) {
+        for (auto iter : propertyValueNode) {
+          if (iter.IsDefined()) {
+            YAML::Node nodeVal = iter.as<YAML::Node>();
+            YAML::Node propertiesNode = nodeVal["value"];
+            // must insert the sequence in differently.
+            std::string rawValueString = propertiesNode.as<std::string>();
+            logger_->log_info("Found %s=%s", propertyName, rawValueString);
+            if (!processor->updateProperty(propertyName, rawValueString)) {
+              std::shared_ptr<core::Connectable> proc =
+                  std::dynamic_pointer_cast<core::Connectable>(processor);
+              if (proc != 0) {
+                logger_->log_warn(
+                    "Received property %s with value %s but is not one of the properties for %s",
+                    propertyName, rawValueString, proc->getName());
+              }
+            }
+          }
+        }
+      } else {
+        std::string rawValueString = propertyValueNode.as<std::string>();
+        if (!processor->setProperty(propertyName, rawValueString)) {
+          std::shared_ptr<core::Connectable> proc = std::dynamic_pointer_cast<
+              core::Connectable>(processor);
+          if (proc != 0) {
+            logger_->log_warn(
+                "Received property %s with value %s but is not one of the properties for %s",
+                propertyName, rawValueString, proc->getName());
+          }
+        }
       }
     }
   }
 }
 
-std::string YamlConfiguration::getOrGenerateId(
-    YAML::Node *yamlNode,
-    const std::string &idField) {
+std::string YamlConfiguration::getOrGenerateId(YAML::Node *yamlNode,
+                                               const std::string &idField) {
   std::string id;
   YAML::Node node = yamlNode->as<YAML::Node>();
 
@@ -579,7 +674,7 @@ std::string YamlConfiguration::getOrGenerateId(
     } else {
       throw std::invalid_argument(
           "getOrGenerateId: idField is expected to reference YAML::Node "
-              "of YAML::NodeType::Scalar.");
+          "of YAML::NodeType::Scalar.");
     }
   } else {
     uuid_t uuid;
@@ -592,12 +687,10 @@ std::string YamlConfiguration::getOrGenerateId(
   return id;
 }
 
-void YamlConfiguration::checkRequiredField(
-    YAML::Node *yamlNode,
-    const std::string &fieldName,
-    const std::string &yamlSection,
-    const std::string &errorMessage) {
-
+void YamlConfiguration::checkRequiredField(YAML::Node *yamlNode,
+                                           const std::string &fieldName,
+                                           const std::string &yamlSection,
+                                           const std::string &errorMessage) {
   std::string errMsg = errorMessage;
   if (!yamlNode->as<YAML::Node>()[fieldName]) {
     if (errMsg.empty()) {
@@ -605,14 +698,13 @@ void YamlConfiguration::checkRequiredField(
       // invalid YAML config file, using the component name if present
       errMsg =
           yamlNode->as<YAML::Node>()["name"] ?
-          "Unable to parse configuration file for component named '" +
-              yamlNode->as<YAML::Node>()["name"].as<std::string>() +
-              "' as required field '" + fieldName + "' is missing" :
-          "Unable to parse configuration file as required field '" +
-              fieldName + "' is missing";
+              "Unable to parse configuration file for component named '"
+                  + yamlNode->as<YAML::Node>()["name"].as<std::string>()
+                  + "' as required field '" + fieldName + "' is missing" :
+              "Unable to parse configuration file as required field '"
+                  + fieldName + "' is missing";
       if (!yamlSection.empty()) {
-        errMsg += " [in '" + yamlSection +
-            "' section of configuration file]";
+        errMsg += " [in '" + yamlSection + "' section of configuration file]";
       }
     }
     logger_->log_error(errMsg.c_str());

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/processors/GetFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/GetFile.cpp b/libminifi/src/processors/GetFile.cpp
index dcabb5d..e39afec 100644
--- a/libminifi/src/processors/GetFile.cpp
+++ b/libminifi/src/processors/GetFile.cpp
@@ -43,6 +43,10 @@ namespace apache {
 namespace nifi {
 namespace minifi {
 namespace processors {
+
+
+
+
 core::Property GetFile::BatchSize(
     "Batch Size", "The maximum number of files to pull in each iteration",
     "10");

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/src/processors/InvokeHTTP.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/InvokeHTTP.cpp b/libminifi/src/processors/InvokeHTTP.cpp
index 17226a8..295560f 100644
--- a/libminifi/src/processors/InvokeHTTP.cpp
+++ b/libminifi/src/processors/InvokeHTTP.cpp
@@ -17,6 +17,7 @@
  */
 
 #include "processors/InvokeHTTP.h"
+#include <regex.h>
 #include <curl/curlbuild.h>
 #include <curl/easy.h>
 #include <uuid/uuid.h>
@@ -42,12 +43,6 @@
 #include "ResourceClaim.h"
 #include "utils/StringUtils.h"
 
-#if  (__GNUC__ >= 4)
-#if (__GNUC_MINOR__ < 9)
-#include <regex.h>
-#endif
-#endif
-
 namespace org {
 namespace apache {
 namespace nifi {
@@ -156,7 +151,8 @@ core::Relationship InvokeHTTP::RelFailure(
 
 void InvokeHTTP::set_request_method(CURL *curl, const std::string &method) {
   std::string my_method = method;
-  std::transform(my_method.begin(), my_method.end(), my_method.begin(), ::toupper);
+  std::transform(my_method.begin(), my_method.end(), my_method.begin(),
+                 ::toupper);
   if (my_method == "POST") {
     curl_easy_setopt(curl, CURLOPT_POST, 1);
   } else if (my_method == "PUT") {
@@ -280,6 +276,17 @@ void InvokeHTTP::onSchedule(core::ProcessContext *context,
   }
 
   utils::StringUtils::StringToBool(penalize_no_retry, penalize_no_retry_);
+
+  std::string context_name;
+  if (context->getProperty(SSLContext.getName(), context_name)
+      && !IsNullOrEmpty(context_name)) {
+    std::shared_ptr<core::controller::ControllerService> service = context
+        ->getControllerService(context_name);
+    if (nullptr != service) {
+      ssl_context_service_ = std::static_pointer_cast<
+          minifi::controllers::SSLContextService>(service);
+    }
+  }
 }
 
 InvokeHTTP::~InvokeHTTP() {
@@ -288,9 +295,9 @@ InvokeHTTP::~InvokeHTTP() {
 
 inline bool InvokeHTTP::matches(const std::string &value,
                                 const std::string &sregex) {
-#ifdef __GNUC__
-#if (__GNUC__ >= 4)
-#if (__GNUC_MINOR__ < 9)
+  if (sregex == ".*")
+    return true;
+
   regex_t regex;
   int ret = regcomp(&regex, sregex.c_str(), 0);
   if (ret)
@@ -299,24 +306,7 @@ inline bool InvokeHTTP::matches(const std::string &value,
   regfree(&regex);
   if (ret)
     return false;
-#else
-  try {
-    std::regex re(sregex);
 
-    if (!std::regex_match(value, re)) {
-      return false;
-    }
-  } catch (std::regex_error e) {
-    logger_->log_error("Invalid File Filter regex: %s.", e.what());
-    return false;
-  }
-#endif
-#endif
-#else
-  logger_->log_info("Cannot support regex filtering");
-  if (regex == ".*")
-  return true;
-#endif
   return true;
 }
 
@@ -346,6 +336,33 @@ struct curl_slist *InvokeHTTP::build_header_list(
   }
   return list;
 }
+
+bool InvokeHTTP::isSecure(const std::string &url) {
+  if (url.find("https") != std::string::npos) {
+    return true;
+  }
+  return false;
+}
+
+CURLcode InvokeHTTP::configure_ssl_context(CURL *curl, void *ctx, void *param) {
+  minifi::controllers::SSLContextService *ssl_context_service =
+      static_cast<minifi::controllers::SSLContextService*>(param);
+  if (!ssl_context_service->configure_ssl_context(static_cast<SSL_CTX*>(ctx))) {
+    return CURLE_FAILED_INIT;
+  }
+  return CURLE_OK;
+}
+
+void InvokeHTTP::configure_secure_connection(CURL *http_session) {
+  logger_->log_debug("InvokeHTTP -- Using certificate file %s",
+                     ssl_context_service_->getCertificateFile());
+  curl_easy_setopt(http_session, CURLOPT_VERBOSE, 1L);
+  curl_easy_setopt(http_session, CURLOPT_SSL_CTX_FUNCTION,
+                   &InvokeHTTP::configure_ssl_context);
+  curl_easy_setopt(http_session, CURLOPT_SSL_CTX_DATA,
+                   static_cast<void*>(ssl_context_service_.get()));
+}
+
 void InvokeHTTP::onTrigger(core::ProcessContext *context,
                            core::ProcessSession *session) {
   std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<
@@ -371,6 +388,10 @@ void InvokeHTTP::onTrigger(core::ProcessContext *context,
   CURL *http_session = curl_easy_init();
   // set the HTTP request method from libCURL
   set_request_method(http_session, method_);
+  if (isSecure(url_) && ssl_context_service_ != nullptr) {
+    configure_secure_connection(http_session);
+  }
+
   curl_easy_setopt(http_session, CURLOPT_URL, url_.c_str());
 
   if (connect_timeout_ > 0) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/FlowFileRecordTest.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/FlowFileRecordTest.cpp b/libminifi/test/FlowFileRecordTest.cpp
deleted file mode 100644
index 09a3d33..0000000
--- a/libminifi/test/FlowFileRecordTest.cpp
+++ /dev/null
@@ -1,28 +0,0 @@
-/**
- * @file MiNiFiMain.cpp 
- * MiNiFiMain implementation 
- *
- * 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.
- */
-#include <vector>
-#include <queue>
-#include <map>
-
-#include "FlowFileRecord.h"
-
-int main(int argc, char **argv)
-{
-}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/HttpGetIntegrationTest.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/HttpGetIntegrationTest.cpp b/libminifi/test/HttpGetIntegrationTest.cpp
deleted file mode 100644
index 90505b4..0000000
--- a/libminifi/test/HttpGetIntegrationTest.cpp
+++ /dev/null
@@ -1,120 +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.
- */
-
-#include <cassert>
-#include <chrono>
-#include <fstream>
-#include <memory>
-#include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <sys/stat.h>
-#include "utils/StringUtils.h"
-#include "../include/core/Core.h"
-#include "../include/core/logging/LogAppenders.h"
-#include "../include/core/logging/BaseLogger.h"
-#include "../include/core/logging/Logger.h"
-#include "../include/core/ProcessGroup.h"
-#include "../include/core/yaml/YamlConfiguration.h"
-#include "../include/FlowController.h"
-#include "../include/properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "../include/io/StreamFactory.h"
-
-std::string test_file_location;
-
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(10));
-}
-
-int main(int argc, char **argv) {
-
-  if (argc > 1) {
-    test_file_location = argv[1];
-  }
-  mkdir("content_repository", S_IRWXU | S_IRWXG | S_IROTH | S_IXOTH);
-  std::ostringstream oss;
-  std::unique_ptr<logging::BaseLogger> outputLogger = std::unique_ptr<
-      logging::BaseLogger>(
-      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,
-                                                                         0));
-  std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
-  logger->updateLogger(std::move(outputLogger));
-  logger->setLogLevel("debug");
-
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::shared_ptr<core::Repository> test_repo =
-      std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<
-      TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file,
-                     test_file_location);
-
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = std::make_shared<minifi::io::StreamFactory>(configuration);
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<
-      core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, stream_factory, test_file_location));
-  std::shared_ptr<TestRepository> repo =
-      std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<
-      minifi::FlowController>(test_repo, test_flow_repo, std::make_shared<minifi::Configure>(), std::move(yaml_ptr),
-  DEFAULT_ROOT_GROUP_NAME,
-                              true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, stream_factory, test_file_location);
-
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(
-      test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(
-      ptr.get());
-  ptr.release();
-
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
-
-  controller->waitUnload(60000);
-  std::string logs = oss.str();
-  assert(logs.find("key:filename value:") != std::string::npos);
-  assert(
-      logs.find(
-          "key:invokehttp.request.url value:https://curl.haxx.se/libcurl/c/httpput.html")
-          != std::string::npos);
-  assert(logs.find("Size:8970 Offset:0") != std::string::npos);
-  assert(
-      logs.find("key:invokehttp.status.code value:200") != std::string::npos);
-  std::string stringtofind = "Resource Claim created ./content_repository/";
-
-  size_t loc = logs.find(stringtofind);
-  while (loc > 0) {
-    std::string id = logs.substr(loc + stringtofind.size(), 36);
-
-    loc = logs.find(stringtofind, loc+1);
-    std::string path = "content_repository/" + id;
-    unlink(path.c_str());
-
-    if ( loc == std::string::npos)
-      break;
-  }
-  rmdir("./content_repository");
-  return 0;
-}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/HttpPostIntegrationTest.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/HttpPostIntegrationTest.cpp b/libminifi/test/HttpPostIntegrationTest.cpp
deleted file mode 100644
index 73d21e6..0000000
--- a/libminifi/test/HttpPostIntegrationTest.cpp
+++ /dev/null
@@ -1,120 +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.
- */
-
-#include <cassert>
-#include <chrono>
-#include <fstream>
-#include <memory>
-#include <string>
-#include <thread>
-#include <type_traits>
-#include <vector>
-#include <sys/stat.h>
-#include "utils/StringUtils.h"
-#include "../include/core/Core.h"
-#include "../include/core/logging/LogAppenders.h"
-#include "../include/core/logging/BaseLogger.h"
-#include "../include/core/logging/Logger.h"
-#include "../include/core/ProcessGroup.h"
-#include "../include/core/yaml/YamlConfiguration.h"
-#include "../include/FlowController.h"
-#include "../include/properties/Configure.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "../include/io/StreamFactory.h"
-#include "../include/properties/Configure.h"
-
-std::string test_file_location;
-
-void waitToVerifyProcessor() {
-  std::this_thread::sleep_for(std::chrono::seconds(2));
-}
-
-int main(int argc, char **argv) {
-
-  if (argc > 1) {
-    test_file_location = argv[1];
-  }
-  mkdir("/tmp/aljr39/",S_IRWXU | S_IRWXG | S_IROTH | S_IXOTH);
-  std::ofstream myfile;
-  myfile.open ("/tmp/aljr39/example.txt");
-  myfile << "Hello world" << std::endl;
-  myfile.close();
-  mkdir("content_repository", S_IRWXU | S_IRWXG | S_IROTH | S_IXOTH);
-  std::ostringstream oss;
-  std::unique_ptr<logging::BaseLogger> outputLogger = std::unique_ptr<
-      logging::BaseLogger>(
-      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,
-                                                                         0));
-  std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
-  logger->updateLogger(std::move(outputLogger));
-  logger->setLogLevel("debug");
-
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-
-  std::shared_ptr<core::Repository> test_repo =
-      std::make_shared<TestRepository>();
-  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<
-      TestFlowRepository>();
-
-  configuration->set(minifi::Configure::nifi_flow_configuration_file,
-                     test_file_location);
-  std::shared_ptr<minifi::io::StreamFactory> stream_factory = std::make_shared<minifi::io::StreamFactory>(configuration);
-
-  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<
-      core::YamlConfiguration>(
-      new core::YamlConfiguration(test_repo, test_repo, stream_factory, test_file_location));
-  std::shared_ptr<TestRepository> repo =
-      std::static_pointer_cast<TestRepository>(test_repo);
-
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<
-      minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr),
-  DEFAULT_ROOT_GROUP_NAME,
-                              true);
-
-  core::YamlConfiguration yaml_config(test_repo, test_repo, stream_factory, test_file_location);
-
-  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(
-      test_file_location);
-  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(
-      ptr.get());
-  ptr.release();
-
-  controller->load();
-  controller->start();
-  waitToVerifyProcessor();
-
-  controller->waitUnload(60000);
-  std::string logs = oss.str();
-  assert(logs.find("curl performed") != std::string::npos);
-  assert(logs.find("Import offset 0 length 12") != std::string::npos);
-
-  std::string stringtofind = "Resource Claim created ./content_repository/";
-
-  size_t loc = logs.find(stringtofind);
-  while (loc > 0 && loc != std::string::npos) {
-    std::string id = logs.substr(loc + stringtofind.size(), 36);
-    loc = logs.find(stringtofind, loc+1);
-    std::string path = "content_repository/" + id;
-    unlink(path.c_str());
-    if ( loc == std::string::npos)
-      break;
-  }
-
-  rmdir("./content_repository");
-  return 0;
-}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/ProcessorTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/ProcessorTests.cpp b/libminifi/test/ProcessorTests.cpp
deleted file mode 100644
index dfdcf47..0000000
--- a/libminifi/test/ProcessorTests.cpp
+++ /dev/null
@@ -1,408 +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 <uuid/uuid.h>
-#include <fstream>
-#include "unit/ProvenanceTestHelper.h"
-#include "TestBase.h"
-#include "core/logging/LogAppenders.h"
-#include "core/logging/BaseLogger.h"
-#include "processors/GetFile.h"
-#include "core/Core.h"
-#include "core/FlowFile.h"
-#include "core/Processor.h"
-#include "core/ProcessContext.h"
-#include "core/ProcessSession.h"
-#include "core/ProcessorNode.h"
-#include "core/reporting/SiteToSiteProvenanceReportingTask.h"
-
-
-
-TEST_CASE("Test Creation of GetFile", "[getfileCreate]") {
-  std::shared_ptr<core::Processor> processor = std::make_shared<
-        org::apache::nifi::minifi::processors::GetFile>("processorname");
-  REQUIRE(processor->getName() == "processorname");
-}
-
-TEST_CASE("Test Find file", "[getfileCreate2]") {
-
-  TestController testController;
-
-  testController.enableDebug();
-
-  std::shared_ptr<core::Processor> processor = std::make_shared<
-      org::apache::nifi::minifi::processors::GetFile>("getfileCreate2");
-
-  std::shared_ptr<core::Processor> processorReport =
-      std::make_shared<
-          org::apache::nifi::minifi::core::reporting::SiteToSiteProvenanceReportingTask>(std::make_shared<org::apache::nifi::minifi::io::StreamFactory>(std::make_shared<org::apache::nifi::minifi::Configure>()));
-
-  std::shared_ptr<core::Repository> test_repo =
-      std::make_shared<TestRepository>();
-
-  std::shared_ptr<TestRepository> repo =
-      std::static_pointer_cast<TestRepository>(test_repo);
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<
-      TestFlowController>(test_repo, test_repo);
-
-  char format[] = "/tmp/gt.XXXXXX";
-  char *dir = testController.createTempDirectory(format);
-
-  uuid_t processoruuid;
-  REQUIRE(true == processor->getUUID(processoruuid));
-
-  std::shared_ptr<minifi::Connection> connection = std::make_shared<
-      minifi::Connection>(test_repo, "getfileCreate2Connection");
-  connection->setRelationship(core::Relationship("success", "description"));
-
-  // link the connections so that we can test results at the end for this
-  connection->setSource(processor);
-  connection->setDestination(processor);
-
-  connection->setSourceUUID(processoruuid);
-  connection->setDestinationUUID(processoruuid);
-
-  processor->addConnection(connection);
-  REQUIRE(dir != NULL);
-
-  core::ProcessorNode node(processor);
-
-  core::ProcessContext context(node, test_repo);
-  core::ProcessSessionFactory factory(&context);
-  context.setProperty(org::apache::nifi::minifi::processors::GetFile::Directory,
-                      dir);
-  core::ProcessSession session(&context);
-
-  processor->onSchedule(&context, &factory);
-  REQUIRE(processor->getName() == "getfileCreate2");
-
-  std::shared_ptr<core::FlowFile> record;
-  processor->setScheduledState(core::ScheduledState::RUNNING);
-  processor->onTrigger(&context, &session);
-
-  provenance::ProvenanceReporter *reporter = session.getProvenanceReporter();
-  std::set<provenance::ProvenanceEventRecord*> records = reporter->getEvents();
-  record = session.get();
-  REQUIRE(record == nullptr);
-  REQUIRE(records.size() == 0);
-
-  std::fstream file;
-  std::stringstream ss;
-  ss << dir << "/" << "tstFile.ext";
-  file.open(ss.str(), std::ios::out);
-  file << "tempFile";
-  file.close();
-
-  processor->incrementActiveTasks();
-  processor->setScheduledState(core::ScheduledState::RUNNING);
-  processor->onTrigger(&context, &session);
-  unlink(ss.str().c_str());
-  reporter = session.getProvenanceReporter();
-
-  REQUIRE(processor->getName() == "getfileCreate2");
-
-  records = reporter->getEvents();
-
-  for (provenance::ProvenanceEventRecord *provEventRecord : records) {
-    REQUIRE(provEventRecord->getComponentType() == processor->getName());
-  }
-  session.commit();
-  std::shared_ptr<core::FlowFile> ffr = session.get();
-
-  ffr->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
-  REQUIRE(2 == repo->getRepoMap().size());
-
-  for (auto entry : repo->getRepoMap()) {
-    provenance::ProvenanceEventRecord newRecord;
-    newRecord.DeSerialize((uint8_t*) entry.second.data(),
-                          entry.second.length());
-
-    bool found = false;
-    for (auto provRec : records) {
-      if (provRec->getEventId() == newRecord.getEventId()) {
-        REQUIRE(provRec->getEventId() == newRecord.getEventId());
-        REQUIRE(provRec->getComponentId() == newRecord.getComponentId());
-        REQUIRE(provRec->getComponentType() == newRecord.getComponentType());
-        REQUIRE(provRec->getDetails() == newRecord.getDetails());
-        REQUIRE(provRec->getEventDuration() == newRecord.getEventDuration());
-        found = true;
-        break;
-      }
-    }
-    if (!found)
-      throw std::runtime_error("Did not find record");
-
-  }
-
-  core::ProcessorNode nodeReport(processorReport);
-  core::ProcessContext contextReport(nodeReport, test_repo);
-  core::ProcessSessionFactory factoryReport(&contextReport);
-  core::ProcessSession sessionReport(&contextReport);
-  processorReport->onSchedule(&contextReport, &factoryReport);
-  std::shared_ptr<
-      org::apache::nifi::minifi::core::reporting::SiteToSiteProvenanceReportingTask> taskReport =
-      std::static_pointer_cast<
-          org::apache::nifi::minifi::core::reporting::SiteToSiteProvenanceReportingTask>(
-          processorReport);
-  taskReport->setBatchSize(1);
-  std::vector<std::shared_ptr<provenance::ProvenanceEventRecord>> recordsReport;
-  processorReport->incrementActiveTasks();
-  processorReport->setScheduledState(core::ScheduledState::RUNNING);
-  std::string jsonStr;
-  repo->getProvenanceRecord(recordsReport, 1);
-  taskReport->getJsonReport(&contextReport, &sessionReport, recordsReport,
-                            jsonStr);
-  REQUIRE(recordsReport.size() == 1);
-  REQUIRE(
-      taskReport->getName()
-          == std::string(
-              org::apache::nifi::minifi::core::reporting::SiteToSiteProvenanceReportingTask::ReportTaskName));
-  REQUIRE(
-      jsonStr.find("\"componentType\" : \"getfileCreate2\"")
-          != std::string::npos);
-}
-
-TEST_CASE("Test GetFileLikeIt'sThreaded", "[getfileCreate3]") {
-
-  TestController testController;
-
-  testController.enableDebug();
-
-  std::shared_ptr<core::Processor> processor = std::make_shared<
-      org::apache::nifi::minifi::processors::GetFile>("getfileCreate2");
-
-  std::shared_ptr<core::Repository> test_repo =
-      std::make_shared<TestRepository>();
-
-  std::shared_ptr<TestRepository> repo =
-      std::static_pointer_cast<TestRepository>(test_repo);
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<
-      TestFlowController>(test_repo, test_repo);
-
-  char format[] = "/tmp/gt.XXXXXX";
-  char *dir = testController.createTempDirectory(format);
-
-  uuid_t processoruuid;
-  REQUIRE(true == processor->getUUID(processoruuid));
-
-  std::shared_ptr<minifi::Connection> connection = std::make_shared<
-      minifi::Connection>(test_repo, "getfileCreate2Connection");
-  connection->setRelationship(core::Relationship("success", "description"));
-
-  // link the connections so that we can test results at the end for this
-  connection->setSource(processor);
-  connection->setDestination(processor);
-
-  connection->setSourceUUID(processoruuid);
-  connection->setDestinationUUID(processoruuid);
-
-  processor->addConnection(connection);
-  REQUIRE(dir != NULL);
-
-  core::ProcessorNode node(processor);
-  core::ProcessContext context(node, test_repo);
-  core::ProcessSessionFactory factory(&context);
-  context.setProperty(org::apache::nifi::minifi::processors::GetFile::Directory,
-                      dir);
-  // replicate 10 threads
-  processor->setScheduledState(core::ScheduledState::RUNNING);
-  processor->onSchedule(&context, &factory);
-
-  int prev = 0;
-  for (int i = 0; i < 10; i++) {
-
-    core::ProcessSession session(&context);
-    REQUIRE(processor->getName() == "getfileCreate2");
-
-    std::shared_ptr<core::FlowFile> record;
-
-    processor->onTrigger(&context, &session);
-
-    provenance::ProvenanceReporter *reporter = session.getProvenanceReporter();
-    std::set<provenance::ProvenanceEventRecord*> records =
-        reporter->getEvents();
-    record = session.get();
-    REQUIRE(record == nullptr);
-    REQUIRE(records.size() == 0);
-
-    std::fstream file;
-    std::stringstream ss;
-    ss << dir << "/" << "tstFile.ext";
-    file.open(ss.str(), std::ios::out);
-    file << "tempFile";
-    file.close();
-
-    processor->incrementActiveTasks();
-    processor->setScheduledState(core::ScheduledState::RUNNING);
-    processor->onTrigger(&context, &session);
-    unlink(ss.str().c_str());
-    reporter = session.getProvenanceReporter();
-
-    REQUIRE(processor->getName() == "getfileCreate2");
-
-    records = reporter->getEvents();
-
-    for (provenance::ProvenanceEventRecord *provEventRecord : records) {
-      REQUIRE(provEventRecord->getComponentType() == processor->getName());
-    }
-    session.commit();
-    std::shared_ptr<core::FlowFile> ffr = session.get();
-
-    REQUIRE((repo->getRepoMap().size() % 2) == 0);
-    REQUIRE(repo->getRepoMap().size() == (prev + 2));
-    prev += 2;
-
-  }
-
-}
-
-TEST_CASE("LogAttributeTest", "[getfileCreate3]") {
-  std::ostringstream oss;
-  std::unique_ptr<logging::BaseLogger> outputLogger = std::unique_ptr<
-      logging::BaseLogger>(
-      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,
-                                                                         0));
-  std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
-  logger->updateLogger(std::move(outputLogger));
-
-  TestController testController;
-
-  testController.enableDebug();
-
-  std::shared_ptr<core::Repository> repo = std::make_shared<TestRepository>();
-
-  std::shared_ptr<core::Processor> processor = std::make_shared<
-      org::apache::nifi::minifi::processors::GetFile>("getfileCreate2");
-
-  std::shared_ptr<core::Processor> logAttribute = std::make_shared<
-      org::apache::nifi::minifi::processors::LogAttribute>("logattribute");
-
-  char format[] = "/tmp/gt.XXXXXX";
-  char *dir = testController.createTempDirectory(format);
-
-  uuid_t processoruuid;
-  REQUIRE(true == processor->getUUID(processoruuid));
-
-  uuid_t logattribute_uuid;
-  REQUIRE(true == logAttribute->getUUID(logattribute_uuid));
-
-  std::shared_ptr<minifi::Connection> connection = std::make_shared<
-      minifi::Connection>(repo, "getfileCreate2Connection");
-  connection->setRelationship(core::Relationship("success", "description"));
-
-  std::shared_ptr<minifi::Connection> connection2 = std::make_shared<
-      minifi::Connection>(repo, "logattribute");
-  connection2->setRelationship(core::Relationship("success", "description"));
-
-  // link the connections so that we can test results at the end for this
-  connection->setSource(processor);
-
-  // link the connections so that we can test results at the end for this
-  connection->setDestination(logAttribute);
-
-  connection2->setSource(logAttribute);
-
-  connection2->setSourceUUID(logattribute_uuid);
-  connection->setSourceUUID(processoruuid);
-  connection->setDestinationUUID(logattribute_uuid);
-
-  processor->addConnection(connection);
-  logAttribute->addConnection(connection);
-  logAttribute->addConnection(connection2);
-  REQUIRE(dir != NULL);
-
-  core::ProcessorNode node(processor);
-  core::ProcessorNode node2(logAttribute);
-
-  core::ProcessContext context(node, repo);
-  core::ProcessContext context2(node2, repo);
-  context.setProperty(org::apache::nifi::minifi::processors::GetFile::Directory,
-                      dir);
-  core::ProcessSession session(&context);
-  core::ProcessSession session2(&context2);
-
-  REQUIRE(processor->getName() == "getfileCreate2");
-
-  std::shared_ptr<core::FlowFile> record;
-  processor->setScheduledState(core::ScheduledState::RUNNING);
-
-  core::ProcessSessionFactory factory(&context);
-  processor->onSchedule(&context, &factory);
-  processor->onTrigger(&context, &session);
-
-  logAttribute->incrementActiveTasks();
-  logAttribute->setScheduledState(core::ScheduledState::RUNNING);
-  core::ProcessSessionFactory factory2(&context2);
-  logAttribute->onSchedule(&context2, &factory2);
-  logAttribute->onTrigger(&context2, &session2);
-
-  provenance::ProvenanceReporter *reporter = session.getProvenanceReporter();
-  std::set<provenance::ProvenanceEventRecord*> records = reporter->getEvents();
-  record = session.get();
-  REQUIRE(record == nullptr);
-  REQUIRE(records.size() == 0);
-
-  std::fstream file;
-  std::stringstream ss;
-  ss << dir << "/" << "tstFile.ext";
-  file.open(ss.str(), std::ios::out);
-  file << "tempFile";
-  file.close();
-
-  processor->incrementActiveTasks();
-  processor->setScheduledState(core::ScheduledState::RUNNING);
-  processor->onTrigger(&context, &session);
-  unlink(ss.str().c_str());
-  reporter = session.getProvenanceReporter();
-
-  records = reporter->getEvents();
-  session.commit();
-  oss.str("");
-  oss.clear();
-
-  logAttribute->incrementActiveTasks();
-  logAttribute->setScheduledState(core::ScheduledState::RUNNING);
-  logAttribute->onTrigger(&context2, &session2);
-
-  //session2.commit();
-  records = reporter->getEvents();
-
-  std::string log_attribute_output = oss.str();
-  REQUIRE(
-      log_attribute_output.find("key:absolute.path value:" + ss.str())
-          != std::string::npos);
-  REQUIRE(log_attribute_output.find("Size:8 Offset:0") != std::string::npos);
-  REQUIRE(
-      log_attribute_output.find("key:path value:" + std::string(dir))
-          != std::string::npos);
-
-  outputLogger = std::unique_ptr<logging::BaseLogger>(
-      new org::apache::nifi::minifi::core::logging::NullAppender());
-  logger->updateLogger(std::move(outputLogger));
-
-}
-
-int fileSize(const char *add) {
-  std::ifstream mySource;
-  mySource.open(add, std::ios_base::binary);
-  mySource.seekg(0, std::ios_base::end);
-  int size = mySource.tellg();
-  mySource.close();
-  return size;
-}
-

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/SocketTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/SocketTests.cpp b/libminifi/test/SocketTests.cpp
deleted file mode 100644
index 2e5013b..0000000
--- a/libminifi/test/SocketTests.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 "TestBase.h"
-#include "io/ClientSocket.h"
-
-using namespace org::apache::nifi::minifi::io;
-TEST_CASE("TestSocket", "[TestSocket1]") {
-
-  Socket socket(std::make_shared<SocketContext>(std::make_shared<minifi::Configure>()), "localhost", 8183);
-  REQUIRE(-1 == socket.initialize());
-  REQUIRE("localhost" == socket.getHostname());
-  socket.closeStream();
-
-}
-
-TEST_CASE("TestSocketWriteTest1", "[TestSocket2]") {
-
-  Socket socket(std::make_shared<SocketContext>(std::make_shared<minifi::Configure>()), "localhost", 8183);
-  REQUIRE(-1 == socket.initialize());
-
-  socket.writeData(0, 0);
-
-  std::vector<uint8_t> buffer;
-  buffer.push_back('a');
-
-  REQUIRE(-1 == socket.writeData(buffer, 1));
-
-  socket.closeStream();
-
-}
-
-TEST_CASE("TestSocketWriteTest2", "[TestSocket3]") {
-
-  std::vector<uint8_t> buffer;
-  buffer.push_back('a');
-  
-  std::shared_ptr<SocketContext> socket_context = std::make_shared<SocketContext>(std::make_shared<minifi::Configure>());
-
-  Socket server(socket_context, "localhost", 9183, 1);
-
-  REQUIRE(-1 != server.initialize());
-
-  Socket client(socket_context, "localhost", 9183);
-
-  REQUIRE(-1 != client.initialize());
-
-  REQUIRE(1 == client.writeData(buffer, 1));
-
-  std::vector<uint8_t> readBuffer;
-  readBuffer.resize(1);
-
-  REQUIRE(1 == server.readData(readBuffer, 1));
-
-  REQUIRE(readBuffer == buffer);
-
-  server.closeStream();
-
-  client.closeStream();
-
-}
-
-TEST_CASE("TestGetHostName", "[TestSocket4]") {
-
-  REQUIRE(Socket::getMyHostName().length() > 0);
-
-}
-
-TEST_CASE("TestWriteEndian64", "[TestSocket4]") {
-
-  std::vector<uint8_t> buffer;
-  buffer.push_back('a');
-  
-  std::shared_ptr<SocketContext> socket_context = std::make_shared<SocketContext>(std::make_shared<minifi::Configure>());
-
-  Socket server(socket_context, "localhost", 9183, 1);
-
-  REQUIRE(-1 != server.initialize());
-
-  Socket client(socket_context, "localhost", 9183);
-
-  REQUIRE(-1 != client.initialize());
-
-  uint64_t negative_one = -1;
-  REQUIRE(8 == client.write(negative_one));
-
-  uint64_t negative_two = 0;
-  REQUIRE(8 == server.read(negative_two));
-
-  REQUIRE(negative_two == negative_one);
-
-  server.closeStream();
-
-  client.closeStream();
-
-}
-
-TEST_CASE("TestWriteEndian32", "[TestSocket5]") {
-
-  std::vector<uint8_t> buffer;
-  buffer.push_back('a');
-
-  std::shared_ptr<SocketContext> socket_context = std::make_shared<SocketContext>(std::make_shared<minifi::Configure>());
-  
-  Socket server(socket_context, "localhost", 9183, 1);
-
-  REQUIRE(-1 != server.initialize());
-
-  Socket client(socket_context, "localhost", 9183);
-
-  REQUIRE(-1 != client.initialize());
-
-  {
-    uint32_t negative_one = -1;
-    REQUIRE(4 == client.write(negative_one));
-
-    uint32_t negative_two = 0;
-    REQUIRE(4 == server.read(negative_two));
-
-    REQUIRE(negative_two == negative_one);
-  }
-
-  {
-    uint16_t negative_one = -1;
-    REQUIRE(2 == client.write(negative_one));
-
-    uint16_t negative_two = 0;
-    REQUIRE(2 == server.read(negative_two));
-
-    REQUIRE(negative_two == negative_one);
-  }
-  server.closeStream();
-
-  client.closeStream();
-
-}
-
-TEST_CASE("TestSocketWriteTestAfterClose", "[TestSocket6]") {
-
-  std::vector<uint8_t> buffer;
-  buffer.push_back('a');
-
-  std::shared_ptr<SocketContext> socket_context = std::make_shared<SocketContext>(std::make_shared<minifi::Configure>());
-  
-  Socket server(socket_context, "localhost", 9183, 1);
-
-  REQUIRE(-1 != server.initialize());
-
-  Socket client(socket_context, "localhost", 9183);
-
-  REQUIRE(-1 != client.initialize());
-
-  REQUIRE(1 == client.writeData(buffer, 1));
-
-  std::vector<uint8_t> readBuffer;
-  readBuffer.resize(1);
-
-  REQUIRE(1 == server.readData(readBuffer, 1));
-
-  REQUIRE(readBuffer == buffer);
-
-  client.closeStream();
-
-  REQUIRE(-1 == client.writeData(buffer, 1));
-
-  server.closeStream();
-
-}


[2/5] nifi-minifi-cpp git commit: MINIFI-226: Add controller services capabilities along with unit tests Fix test failures Update Travis YML Update readme to link to MiNiFi licensing information

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/TestExecuteProcess.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/TestExecuteProcess.cpp b/libminifi/test/TestExecuteProcess.cpp
deleted file mode 100644
index 31509d7..0000000
--- a/libminifi/test/TestExecuteProcess.cpp
+++ /dev/null
@@ -1,131 +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.
- */
-
-#include <uuid/uuid.h>
-#include <fstream>
-#include "FlowController.h"
-#include "unit/ProvenanceTestHelper.h"
-#include "core/logging/LogAppenders.h"
-#include "core/logging/BaseLogger.h"
-#include "processors/GetFile.h"
-#include "core/Core.h"
-#include "core/FlowFile.h"
-#include "core/Processor.h"
-#include "core/ProcessContext.h"
-#include "core/ProcessSession.h"
-#include "core/ProcessorNode.h"
-
-int main(int argc, char  **argv)
-{
-
-  std::ostringstream oss;
-  std::unique_ptr<logging::BaseLogger> outputLogger = std::unique_ptr<
-      logging::BaseLogger>(
-      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,
-                                                                         0));
-  std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
-  logger->updateLogger(std::move(outputLogger));
-
-
-  outputLogger = std::unique_ptr<logging::BaseLogger>(
-        new org::apache::nifi::minifi::core::logging::NullAppender());
-    logger->updateLogger(std::move(outputLogger));
-
-  std::shared_ptr<core::Processor> processor = std::make_shared<
-      org::apache::nifi::minifi::processors::ExecuteProcess>("executeProcess");
-  processor->setMaxConcurrentTasks(1);
-
-  std::shared_ptr<core::Repository> test_repo =
-      std::make_shared<TestRepository>();
-
-  std::shared_ptr<TestRepository> repo =
-      std::static_pointer_cast<TestRepository>(test_repo);
-  std::shared_ptr<minifi::FlowController> controller = std::make_shared<
-      TestFlowController>(test_repo, test_repo);
-
-  uuid_t processoruuid;
-  assert(true == processor->getUUID(processoruuid));
-
-  std::shared_ptr<minifi::Connection> connection = std::make_shared<
-      minifi::Connection>(test_repo, "executeProcessConnection");
-  connection->setRelationship(core::Relationship("success", "description"));
-
-  // link the connections so that we can test results at the end for this
-  connection->setSource(processor);
-  connection->setDestination(processor);
-
-  connection->setSourceUUID(processoruuid);
-  connection->setDestinationUUID(processoruuid);
-
-  processor->addConnection(connection);
-  assert(processor->getName() == "executeProcess");
-
-  std::shared_ptr<core::FlowFile> record;
-  processor->setScheduledState(core::ScheduledState::RUNNING);
-
-  processor->initialize();
-
-  std::atomic<bool> is_ready(false);
-
-  std::vector<std::thread> processor_workers;
-
-  core::ProcessorNode node2(processor);
-  std::shared_ptr<core::ProcessContext> contextset = std::make_shared<
-      core::ProcessContext>(node2, test_repo);
-  core::ProcessSessionFactory factory(contextset.get());
-  processor->onSchedule(contextset.get(), &factory);
-
-  for (int i = 0; i < 1; i++) {
-    //
-    processor_workers.push_back(
-        std::thread(
-            [processor,test_repo,&is_ready]()
-            {
-              core::ProcessorNode node(processor);
-              std::shared_ptr<core::ProcessContext> context = std::make_shared<core::ProcessContext>(node, test_repo);
-              context->setProperty(org::apache::nifi::minifi::processors::ExecuteProcess::Command,"sleep 0.5");
-              //context->setProperty(org::apache::nifi::minifi::processors::ExecuteProcess::CommandArguments," 1 >>" + ss.str());
-              std::shared_ptr<core::ProcessSession> session = std::make_shared<core::ProcessSession>(context.get());
-              while(!is_ready.load(std::memory_order_relaxed)) {
-
-              }
-
-              processor->onTrigger(context.get(), session.get());
-
-            }));
-  }
-
-  is_ready.store(true, std::memory_order_relaxed);
-  //is_ready.store(true);
-
-  std::for_each(processor_workers.begin(), processor_workers.end(),
-                [](std::thread &t)
-                {
-                  t.join();
-                });
-
-    outputLogger = std::unique_ptr<logging::BaseLogger>(
-      new org::apache::nifi::minifi::core::logging::NullAppender());
-  logger->updateLogger(std::move(outputLogger));
-
-
-  std::shared_ptr<org::apache::nifi::minifi::processors::ExecuteProcess> execp =
-      std::static_pointer_cast<
-          org::apache::nifi::minifi::processors::ExecuteProcess>(processor);
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/integration/ControllerServiceIntegrationTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/integration/ControllerServiceIntegrationTests.cpp b/libminifi/test/integration/ControllerServiceIntegrationTests.cpp
new file mode 100644
index 0000000..00add78
--- /dev/null
+++ b/libminifi/test/integration/ControllerServiceIntegrationTests.cpp
@@ -0,0 +1,186 @@
+/**
+ *
+ * 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.
+ */
+
+#include <cassert>
+#include <chrono>
+#include <fstream>
+#include <memory>
+#include <string>
+#include <thread>
+#include <type_traits>
+#include "core/logging/LogAppenders.h"
+#include "core/logging/BaseLogger.h"
+#include <vector>
+
+#include "../include/core/controller/ControllerServiceMap.h"
+#include "../include/core/controller/StandardControllerServiceNode.h"
+#include "../include/core/controller/StandardControllerServiceProvider.h"
+#include "controllers/SSLContextService.h"
+#include "../include/core/Core.h"
+#include "../include/core/logging/Logger.h"
+#include "../include/core/ProcessGroup.h"
+#include "../include/core/Resource.h"
+#include "../include/core/yaml/YamlConfiguration.h"
+#include "../include/FlowController.h"
+#include "../include/properties/Configure.h"
+#include "../unit/MockClasses.h"
+#include "../unit/ProvenanceTestHelper.h"
+
+REGISTER_RESOURCE(MockControllerService);
+REGISTER_RESOURCE(MockProcessor);
+std::string test_file_location;
+std::string key_dir;
+
+std::shared_ptr<core::controller::StandardControllerServiceNode> newCsNode(
+    std::shared_ptr<core::controller::ControllerServiceProvider> provider,
+    const std::string id) {
+  std::shared_ptr<core::controller::ControllerService> service =
+      std::make_shared<MockControllerService>();
+  std::shared_ptr<core::controller::StandardControllerServiceNode> testNode =
+      std::make_shared<core::controller::StandardControllerServiceNode>(
+          service, provider, id, std::make_shared<minifi::Configure>());
+
+  return testNode;
+}
+
+void waitToVerifyProcessor() {
+  std::this_thread::sleep_for(std::chrono::seconds(2));
+}
+
+int main(int argc, char **argv) {
+
+  if (argc > 2) {
+    test_file_location = argv[1];
+    key_dir = argv[1];
+  }
+
+  logging::Logger::getLogger()->setLogLevel("trace");
+
+  std::shared_ptr<minifi::Configure> configuration = std::make_shared<
+      minifi::Configure>();
+
+  std::shared_ptr<core::Repository> test_repo =
+      std::make_shared<TestRepository>();
+  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<
+      TestFlowRepository>();
+
+  configuration->set(minifi::Configure::nifi_flow_configuration_file,
+                     test_file_location);
+  /*
+   * nifi.security.client.certificate=/Users/mparisi/Downloads/nifi-toolkit-1.1.1/bin/cn.crt.pem
+   nifi.security.client.private.key=/Users/mparisi/Downloads/nifi-toolkit-1.1.1/bin/cn.ckey.pem
+   nifi.security.client.pass.phrase=/Users/mparisi/Downloads/nifi-toolkit-1.1.1/bin/cn.pass
+   nifi.security.client.ca.certificate=/Users/mparisi/Downloads/nifi-toolkit-1.1.1/bin/nifi-cert.pem
+   */
+  std::string client_cert = "cn.crt.pem";
+  std::string priv_key_file = "cn.ckey.pem";
+  std::string passphrase = "cn.pass";
+  std::string ca_cert = "nifi-cert.pem";
+  configuration->set(minifi::Configure::nifi_security_client_certificate,
+                     test_file_location);
+  configuration->set(minifi::Configure::nifi_security_client_private_key,
+                     priv_key_file);
+  configuration->set(minifi::Configure::nifi_security_client_pass_phrase,
+                     passphrase);
+  configuration->set(minifi::Configure::nifi_default_directory, key_dir);
+
+  std::shared_ptr<minifi::io::StreamFactory> stream_factory = std::make_shared<
+      minifi::io::StreamFactory>(configuration);
+
+  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<
+      core::YamlConfiguration>(
+      new core::YamlConfiguration(test_repo, test_repo, stream_factory,
+                                  configuration, test_file_location));
+  std::shared_ptr<TestRepository> repo =
+      std::static_pointer_cast<TestRepository>(test_repo);
+
+  std::shared_ptr<minifi::FlowController> controller = std::make_shared<
+      minifi::FlowController>(test_repo, test_flow_repo, configuration,
+                              std::move(yaml_ptr),
+                              DEFAULT_ROOT_GROUP_NAME,
+                              true);
+
+  disabled = false;
+  std::shared_ptr<core::controller::ControllerServiceMap> map =
+      std::make_shared<core::controller::ControllerServiceMap>();
+
+  core::YamlConfiguration yaml_config(test_repo, test_repo, stream_factory,
+                                      configuration, test_file_location);
+
+  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(
+      test_file_location);
+  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(
+      ptr.get());
+  ptr.release();
+
+  std::shared_ptr<core::controller::StandardControllerServiceProvider> provider =
+      std::make_shared<core::controller::StandardControllerServiceProvider>(
+          map, pg, std::make_shared<minifi::Configure>());
+  std::shared_ptr<core::controller::ControllerServiceNode> mockNode = pg
+      ->findControllerService("MockItLikeIts1995");
+  assert(mockNode != nullptr);
+  mockNode->enable();
+  std::vector<std::shared_ptr<core::controller::ControllerServiceNode> > linkedNodes =
+      mockNode->getLinkedControllerServices();
+  assert(linkedNodes.size() == 1);
+
+  std::shared_ptr<core::controller::ControllerServiceNode> notexistNode = pg
+      ->findControllerService("MockItLikeItsWrong");
+  assert(notexistNode == nullptr);
+  logging::Logger::getLogger()->log_info(
+      "STARTING FLOW CONTROLLER INTEGRATION TEST");
+  controller->load();
+  controller->start();
+  waitToVerifyProcessor();
+  std::shared_ptr<core::controller::ControllerServiceNode> ssl_client_cont =
+      controller->getControllerServiceNode("SSLClientServiceTest");
+  ssl_client_cont->enable();
+  assert(ssl_client_cont != nullptr);
+  assert(ssl_client_cont->getControllerServiceImplementation() != nullptr);
+  std::shared_ptr<minifi::controllers::SSLContextService> ssl_client =
+      std::static_pointer_cast<minifi::controllers::SSLContextService>(
+          ssl_client_cont->getControllerServiceImplementation());
+
+  assert(ssl_client->getCACertificate().length() > 0);
+
+  // now let's disable one of the controller services.
+  std::shared_ptr<core::controller::ControllerServiceNode> cs_id = controller
+      ->getControllerServiceNode("ID");
+  assert(cs_id != nullptr);
+  controller->disableControllerService(cs_id);
+  disabled = true;
+  waitToVerifyProcessor();
+  controller->enableControllerService(cs_id);
+  disabled = false;
+  waitToVerifyProcessor();
+  std::shared_ptr<core::controller::ControllerServiceNode> mock_cont =
+      controller->getControllerServiceNode("MockItLikeIts1995");
+  assert(cs_id->enabled());
+
+  controller->disableReferencingServices(mock_cont);
+  disabled = true;
+  waitToVerifyProcessor();
+  assert(cs_id->enabled() == false);
+  controller->enableReferencingServices(mock_cont);
+  disabled = false;
+  waitToVerifyProcessor();
+  assert(cs_id->enabled() == true);
+
+  controller->waitUnload(60000);
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/integration/HttpGetIntegrationTest.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/integration/HttpGetIntegrationTest.cpp b/libminifi/test/integration/HttpGetIntegrationTest.cpp
new file mode 100644
index 0000000..aa24dfe
--- /dev/null
+++ b/libminifi/test/integration/HttpGetIntegrationTest.cpp
@@ -0,0 +1,128 @@
+/**
+ *
+ * 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.
+ */
+
+#include <cassert>
+#include <chrono>
+#include <fstream>
+#include <memory>
+#include <string>
+#include <thread>
+#include <type_traits>
+#include <vector>
+#include <sys/stat.h>
+
+#include "utils/StringUtils.h"
+#include "core/Core.h"
+#include "core/logging/LogAppenders.h"
+#include "core/logging/BaseLogger.h"
+#include "core/logging/Logger.h"
+#include "core/ProcessGroup.h"
+#include "core/yaml/YamlConfiguration.h"
+#include "FlowController.h"
+#include "properties/Configure.h"
+#include "../unit/ProvenanceTestHelper.h"
+#include "io/StreamFactory.h"
+
+
+void waitToVerifyProcessor() {
+  std::this_thread::sleep_for(std::chrono::seconds(10));
+}
+
+int main(int argc, char **argv) {
+
+  std::string key_dir,test_file_location;
+  if (argc > 1) {
+    test_file_location = argv[1];
+    key_dir = argv[2];
+  }
+  std::shared_ptr<minifi::Configure> configuration = std::make_shared<
+        minifi::Configure>();
+  configuration->set(minifi::Configure::nifi_default_directory, key_dir);
+  mkdir("content_repository", S_IRWXU | S_IRWXG | S_IROTH | S_IXOTH);
+  std::ostringstream oss;
+  std::unique_ptr<logging::BaseLogger> outputLogger = std::unique_ptr<
+      logging::BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,
+                                                                         0));
+  std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
+  logger->updateLogger(std::move(outputLogger));
+  logger->setLogLevel("trace");
+
+  std::shared_ptr<core::Repository> test_repo =
+      std::make_shared<TestRepository>();
+  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<
+      TestFlowRepository>();
+
+  configuration->set(minifi::Configure::nifi_flow_configuration_file,
+                     test_file_location);
+
+  std::shared_ptr<minifi::io::StreamFactory> stream_factory = std::make_shared<
+      minifi::io::StreamFactory>(configuration);
+  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<
+      core::YamlConfiguration>(
+      new core::YamlConfiguration(test_repo, test_repo, stream_factory,
+                                  configuration, test_file_location));
+  std::shared_ptr<TestRepository> repo =
+      std::static_pointer_cast<TestRepository>(test_repo);
+
+  std::shared_ptr<minifi::FlowController> controller = std::make_shared<
+      minifi::FlowController>(test_repo, test_flow_repo, configuration,
+                              std::move(yaml_ptr),
+                              DEFAULT_ROOT_GROUP_NAME,
+                              true);
+
+  core::YamlConfiguration yaml_config(test_repo, test_repo, stream_factory,configuration,
+                                      test_file_location);
+
+  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(
+      test_file_location);
+  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(
+      ptr.get());
+  ptr.release();
+
+  controller->load();
+  controller->start();
+  waitToVerifyProcessor();
+
+  controller->waitUnload(60000);
+  std::string logs = oss.str();
+  std::cout << logs << std::endl;
+  assert(logs.find("key:filename value:") != std::string::npos);
+  assert(
+      logs.find(
+          "key:invokehttp.request.url value:https://curl.haxx.se/libcurl/c/httpput.html")
+          != std::string::npos);
+  assert(logs.find("Size:8970 Offset:0") != std::string::npos);
+  assert(
+      logs.find("key:invokehttp.status.code value:200") != std::string::npos);
+  std::string stringtofind = "Resource Claim created ./content_repository/";
+
+  size_t loc = logs.find(stringtofind);
+  while (loc > 0) {
+    std::string id = logs.substr(loc + stringtofind.size(), 36);
+
+    loc = logs.find(stringtofind, loc + 1);
+    std::string path = "content_repository/" + id;
+    unlink(path.c_str());
+
+    if (loc == std::string::npos)
+      break;
+  }
+  rmdir("./content_repository");
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/integration/HttpPostIntegrationTest.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/integration/HttpPostIntegrationTest.cpp b/libminifi/test/integration/HttpPostIntegrationTest.cpp
new file mode 100644
index 0000000..45abd34
--- /dev/null
+++ b/libminifi/test/integration/HttpPostIntegrationTest.cpp
@@ -0,0 +1,126 @@
+/**
+ *
+ * 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.
+ */
+
+#include <cassert>
+#include <chrono>
+#include <fstream>
+#include <memory>
+#include <string>
+#include <thread>
+#include <type_traits>
+#include <vector>
+#include <sys/stat.h>
+
+#include "utils/StringUtils.h"
+#include "core/Core.h"
+#include "core/logging/LogAppenders.h"
+#include "core/logging/BaseLogger.h"
+#include "core/logging/Logger.h"
+#include "core/ProcessGroup.h"
+#include "core/yaml/YamlConfiguration.h"
+#include "FlowController.h"
+#include "properties/Configure.h"
+#include "../unit/ProvenanceTestHelper.h"
+#include "io/StreamFactory.h"
+#include "properties/Configure.h"
+
+std::string test_file_location;
+
+void waitToVerifyProcessor() {
+  std::this_thread::sleep_for(std::chrono::seconds(2));
+}
+
+int main(int argc, char **argv) {
+
+  if (argc > 1) {
+    test_file_location = argv[1];
+  }
+  mkdir("/tmp/aljr39/", S_IRWXU | S_IRWXG | S_IROTH | S_IXOTH);
+  std::ofstream myfile;
+  myfile.open("/tmp/aljr39/example.txt");
+  myfile << "Hello world" << std::endl;
+  myfile.close();
+  mkdir("content_repository", S_IRWXU | S_IRWXG | S_IROTH | S_IXOTH);
+  std::ostringstream oss;
+  std::unique_ptr<logging::BaseLogger> outputLogger = std::unique_ptr<
+      logging::BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,
+                                                                         0));
+  std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
+  logger->updateLogger(std::move(outputLogger));
+  logger->setLogLevel("debug");
+
+  std::shared_ptr<minifi::Configure> configuration = std::make_shared<
+      minifi::Configure>();
+
+  std::shared_ptr<core::Repository> test_repo =
+      std::make_shared<TestRepository>();
+  std::shared_ptr<core::Repository> test_flow_repo = std::make_shared<
+      TestFlowRepository>();
+
+  configuration->set(minifi::Configure::nifi_flow_configuration_file,
+                     test_file_location);
+  std::shared_ptr<minifi::io::StreamFactory> stream_factory = std::make_shared<
+      minifi::io::StreamFactory>(configuration);
+
+  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<
+      core::YamlConfiguration>(
+      new core::YamlConfiguration(test_repo, test_repo, stream_factory,
+                                  configuration, test_file_location));
+  std::shared_ptr<TestRepository> repo =
+      std::static_pointer_cast<TestRepository>(test_repo);
+
+  std::shared_ptr<minifi::FlowController> controller = std::make_shared<
+      minifi::FlowController>(test_repo, test_flow_repo, configuration,
+                              std::move(yaml_ptr),
+                              DEFAULT_ROOT_GROUP_NAME,
+                              true);
+
+  core::YamlConfiguration yaml_config(test_repo, test_repo, stream_factory,
+                                      configuration, test_file_location);
+
+  std::unique_ptr<core::ProcessGroup> ptr = yaml_config.getRoot(
+      test_file_location);
+  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup>(
+      ptr.get());
+  ptr.release();
+
+  controller->load();
+  controller->start();
+  waitToVerifyProcessor();
+
+  controller->waitUnload(60000);
+  std::string logs = oss.str();
+  assert(logs.find("curl performed") != std::string::npos);
+  assert(logs.find("Import offset 0 length 12") != std::string::npos);
+
+  std::string stringtofind = "Resource Claim created ./content_repository/";
+
+  size_t loc = logs.find(stringtofind);
+  while (loc > 0 && loc != std::string::npos) {
+    std::string id = logs.substr(loc + stringtofind.size(), 36);
+    loc = logs.find(stringtofind, loc + 1);
+    std::string path = "content_repository/" + id;
+    unlink(path.c_str());
+    if (loc == std::string::npos)
+      break;
+  }
+
+  rmdir("./content_repository");
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/integration/ProcessorTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/integration/ProcessorTests.cpp b/libminifi/test/integration/ProcessorTests.cpp
new file mode 100644
index 0000000..1c447a8
--- /dev/null
+++ b/libminifi/test/integration/ProcessorTests.cpp
@@ -0,0 +1,411 @@
+
+/**
+ *
+ * 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 <uuid/uuid.h>
+#include <fstream>
+#include "../unit/ProvenanceTestHelper.h"
+#include "../TestBase.h"
+#include "core/logging/LogAppenders.h"
+#include "core/logging/BaseLogger.h"
+#include "processors/ListenHTTP.h"
+#include "processors/LogAttribute.h"
+#include "processors/GetFile.h"
+#include "core/Core.h"
+#include "core/FlowFile.h"
+#include "core/Processor.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/ProcessorNode.h"
+#include "core/reporting/SiteToSiteProvenanceReportingTask.h"
+
+
+
+TEST_CASE("Test Creation of GetFile", "[getfileCreate]") {
+  std::shared_ptr<core::Processor> processor = std::make_shared<
+        org::apache::nifi::minifi::processors::GetFile>("processorname");
+  REQUIRE(processor->getName() == "processorname");
+}
+
+TEST_CASE("Test Find file", "[getfileCreate2]") {
+
+  TestController testController;
+
+  testController.enableDebug();
+
+  std::shared_ptr<core::Processor> processor = std::make_shared<
+      org::apache::nifi::minifi::processors::GetFile>("getfileCreate2");
+
+  std::shared_ptr<core::Processor> processorReport =
+      std::make_shared<
+          org::apache::nifi::minifi::core::reporting::SiteToSiteProvenanceReportingTask>(std::make_shared<org::apache::nifi::minifi::io::StreamFactory>(std::make_shared<org::apache::nifi::minifi::Configure>()));
+
+  std::shared_ptr<core::Repository> test_repo =
+      std::make_shared<TestRepository>();
+
+  std::shared_ptr<TestRepository> repo =
+      std::static_pointer_cast<TestRepository>(test_repo);
+
+  char format[] = "/tmp/gt.XXXXXX";
+  char *dir = testController.createTempDirectory(format);
+
+  uuid_t processoruuid;
+  REQUIRE(true == processor->getUUID(processoruuid));
+
+  std::shared_ptr<minifi::Connection> connection = std::make_shared<
+      minifi::Connection>(test_repo, "getfileCreate2Connection");
+  connection->setRelationship(core::Relationship("success", "description"));
+
+  // link the connections so that we can test results at the end for this
+  connection->setSource(processor);
+  connection->setDestination(processor);
+
+  connection->setSourceUUID(processoruuid);
+  connection->setDestinationUUID(processoruuid);
+
+  processor->addConnection(connection);
+  REQUIRE(dir != NULL);
+
+  core::ProcessorNode node(processor);
+  std::shared_ptr<core::controller::ControllerServiceProvider> controller_services_provider =
+      nullptr;
+  core::ProcessContext context(node, controller_services_provider, test_repo);
+  core::ProcessSessionFactory factory(&context);
+  context.setProperty(org::apache::nifi::minifi::processors::GetFile::Directory,
+                      dir);
+  core::ProcessSession session(&context);
+
+  processor->onSchedule(&context, &factory);
+  REQUIRE(processor->getName() == "getfileCreate2");
+
+  std::shared_ptr<core::FlowFile> record;
+  processor->setScheduledState(core::ScheduledState::RUNNING);
+  processor->onTrigger(&context, &session);
+
+  provenance::ProvenanceReporter *reporter = session.getProvenanceReporter();
+  std::set<provenance::ProvenanceEventRecord*> records = reporter->getEvents();
+  record = session.get();
+  REQUIRE(record == nullptr);
+  REQUIRE(records.size() == 0);
+
+  std::fstream file;
+  std::stringstream ss;
+  ss << dir << "/" << "tstFile.ext";
+  file.open(ss.str(), std::ios::out);
+  file << "tempFile";
+  file.close();
+
+  processor->incrementActiveTasks();
+  processor->setScheduledState(core::ScheduledState::RUNNING);
+  processor->onTrigger(&context, &session);
+  unlink(ss.str().c_str());
+  reporter = session.getProvenanceReporter();
+
+  REQUIRE(processor->getName() == "getfileCreate2");
+
+  records = reporter->getEvents();
+
+  for (provenance::ProvenanceEventRecord *provEventRecord : records) {
+    REQUIRE(provEventRecord->getComponentType() == processor->getName());
+  }
+  session.commit();
+  std::shared_ptr<core::FlowFile> ffr = session.get();
+
+  ffr->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
+  REQUIRE(2 == repo->getRepoMap().size());
+
+  for (auto entry : repo->getRepoMap()) {
+    provenance::ProvenanceEventRecord newRecord;
+    newRecord.DeSerialize((uint8_t*) entry.second.data(),
+                          entry.second.length());
+
+    bool found = false;
+    for (auto provRec : records) {
+      if (provRec->getEventId() == newRecord.getEventId()) {
+        REQUIRE(provRec->getEventId() == newRecord.getEventId());
+        REQUIRE(provRec->getComponentId() == newRecord.getComponentId());
+        REQUIRE(provRec->getComponentType() == newRecord.getComponentType());
+        REQUIRE(provRec->getDetails() == newRecord.getDetails());
+        REQUIRE(provRec->getEventDuration() == newRecord.getEventDuration());
+        found = true;
+        break;
+      }
+    }
+    if (!found)
+      throw std::runtime_error("Did not find record");
+
+  }
+
+  core::ProcessorNode nodeReport(processorReport);
+  core::ProcessContext contextReport(nodeReport,controller_services_provider, test_repo);
+  core::ProcessSessionFactory factoryReport(&contextReport);
+  core::ProcessSession sessionReport(&contextReport);
+  processorReport->onSchedule(&contextReport, &factoryReport);
+  std::shared_ptr<
+      org::apache::nifi::minifi::core::reporting::SiteToSiteProvenanceReportingTask> taskReport =
+      std::static_pointer_cast<
+          org::apache::nifi::minifi::core::reporting::SiteToSiteProvenanceReportingTask>(
+          processorReport);
+  taskReport->setBatchSize(1);
+  std::vector<std::shared_ptr<provenance::ProvenanceEventRecord>> recordsReport;
+  processorReport->incrementActiveTasks();
+  processorReport->setScheduledState(core::ScheduledState::RUNNING);
+  std::string jsonStr;
+  repo->getProvenanceRecord(recordsReport, 1);
+  taskReport->getJsonReport(&contextReport, &sessionReport, recordsReport,
+                            jsonStr);
+  REQUIRE(recordsReport.size() == 1);
+  REQUIRE(
+      taskReport->getName()
+          == std::string(
+              org::apache::nifi::minifi::core::reporting::SiteToSiteProvenanceReportingTask::ReportTaskName));
+  REQUIRE(
+      jsonStr.find("\"componentType\" : \"getfileCreate2\"")
+          != std::string::npos);
+}
+
+TEST_CASE("Test GetFileLikeIt'sThreaded", "[getfileCreate3]") {
+
+  TestController testController;
+
+  testController.enableDebug();
+
+  std::shared_ptr<core::Processor> processor = std::make_shared<
+      org::apache::nifi::minifi::processors::GetFile>("getfileCreate2");
+
+  std::shared_ptr<core::Repository> test_repo =
+      std::make_shared<TestRepository>();
+
+  std::shared_ptr<TestRepository> repo =
+      std::static_pointer_cast<TestRepository>(test_repo);
+
+  char format[] = "/tmp/gt.XXXXXX";
+  char *dir = testController.createTempDirectory(format);
+
+  uuid_t processoruuid;
+  REQUIRE(true == processor->getUUID(processoruuid));
+
+  std::shared_ptr<minifi::Connection> connection = std::make_shared<
+      minifi::Connection>(test_repo, "getfileCreate2Connection");
+  connection->setRelationship(core::Relationship("success", "description"));
+
+  // link the connections so that we can test results at the end for this
+  connection->setSource(processor);
+  connection->setDestination(processor);
+
+  connection->setSourceUUID(processoruuid);
+  connection->setDestinationUUID(processoruuid);
+
+  processor->addConnection(connection);
+  REQUIRE(dir != NULL);
+
+  core::ProcessorNode node(processor);
+  std::shared_ptr<core::controller::ControllerServiceProvider> controller_services_provider =
+      nullptr;
+  core::ProcessContext context(node, controller_services_provider, test_repo);
+  core::ProcessSessionFactory factory(&context);
+  context.setProperty(org::apache::nifi::minifi::processors::GetFile::Directory,
+                      dir);
+  // replicate 10 threads
+  processor->setScheduledState(core::ScheduledState::RUNNING);
+  processor->onSchedule(&context, &factory);
+
+  int prev = 0;
+  for (int i = 0; i < 10; i++) {
+
+    core::ProcessSession session(&context);
+    REQUIRE(processor->getName() == "getfileCreate2");
+
+    std::shared_ptr<core::FlowFile> record;
+
+    processor->onTrigger(&context, &session);
+
+    provenance::ProvenanceReporter *reporter = session.getProvenanceReporter();
+    std::set<provenance::ProvenanceEventRecord*> records =
+        reporter->getEvents();
+    record = session.get();
+    REQUIRE(record == nullptr);
+    REQUIRE(records.size() == 0);
+
+    std::fstream file;
+    std::stringstream ss;
+    ss << dir << "/" << "tstFile.ext";
+    file.open(ss.str(), std::ios::out);
+    file << "tempFile";
+    file.close();
+
+    processor->incrementActiveTasks();
+    processor->setScheduledState(core::ScheduledState::RUNNING);
+    processor->onTrigger(&context, &session);
+    unlink(ss.str().c_str());
+    reporter = session.getProvenanceReporter();
+
+    REQUIRE(processor->getName() == "getfileCreate2");
+
+    records = reporter->getEvents();
+
+    for (provenance::ProvenanceEventRecord *provEventRecord : records) {
+      REQUIRE(provEventRecord->getComponentType() == processor->getName());
+    }
+    session.commit();
+    std::shared_ptr<core::FlowFile> ffr = session.get();
+
+    REQUIRE((repo->getRepoMap().size() % 2) == 0);
+    REQUIRE(repo->getRepoMap().size() == (prev + 2));
+    prev += 2;
+
+  }
+
+}
+
+TEST_CASE("LogAttributeTest", "[getfileCreate3]") {
+  std::ostringstream oss;
+  std::unique_ptr<logging::BaseLogger> outputLogger = std::unique_ptr<
+      logging::BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,
+                                                                         0));
+  std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
+  logger->updateLogger(std::move(outputLogger));
+
+  TestController testController;
+
+  testController.enableDebug();
+
+  std::shared_ptr<core::Repository> repo = std::make_shared<TestRepository>();
+
+  std::shared_ptr<core::Processor> processor = std::make_shared<
+      org::apache::nifi::minifi::processors::GetFile>("getfileCreate2");
+
+  std::shared_ptr<core::Processor> logAttribute = std::make_shared<
+      org::apache::nifi::minifi::processors::LogAttribute>("logattribute");
+
+  char format[] = "/tmp/gt.XXXXXX";
+  char *dir = testController.createTempDirectory(format);
+
+  uuid_t processoruuid;
+  REQUIRE(true == processor->getUUID(processoruuid));
+
+  uuid_t logattribute_uuid;
+  REQUIRE(true == logAttribute->getUUID(logattribute_uuid));
+
+  std::shared_ptr<minifi::Connection> connection = std::make_shared<
+      minifi::Connection>(repo, "getfileCreate2Connection");
+  connection->setRelationship(core::Relationship("success", "description"));
+
+  std::shared_ptr<minifi::Connection> connection2 = std::make_shared<
+      minifi::Connection>(repo, "logattribute");
+  connection2->setRelationship(core::Relationship("success", "description"));
+
+  // link the connections so that we can test results at the end for this
+  connection->setSource(processor);
+
+  // link the connections so that we can test results at the end for this
+  connection->setDestination(logAttribute);
+
+  connection2->setSource(logAttribute);
+
+  connection2->setSourceUUID(logattribute_uuid);
+  connection->setSourceUUID(processoruuid);
+  connection->setDestinationUUID(logattribute_uuid);
+
+  processor->addConnection(connection);
+  logAttribute->addConnection(connection);
+  logAttribute->addConnection(connection2);
+  REQUIRE(dir != NULL);
+
+  core::ProcessorNode node(processor);
+  core::ProcessorNode node2(logAttribute);
+  std::shared_ptr<core::controller::ControllerServiceProvider> controller_services_provider =
+      nullptr;
+  core::ProcessContext context(node, controller_services_provider, repo);
+  core::ProcessContext context2(node2, controller_services_provider, repo);
+  context.setProperty(org::apache::nifi::minifi::processors::GetFile::Directory,
+                      dir);
+  core::ProcessSession session(&context);
+  core::ProcessSession session2(&context2);
+
+  REQUIRE(processor->getName() == "getfileCreate2");
+
+  std::shared_ptr<core::FlowFile> record;
+  processor->setScheduledState(core::ScheduledState::RUNNING);
+
+  core::ProcessSessionFactory factory(&context);
+  processor->onSchedule(&context, &factory);
+  processor->onTrigger(&context, &session);
+
+  logAttribute->incrementActiveTasks();
+  logAttribute->setScheduledState(core::ScheduledState::RUNNING);
+  core::ProcessSessionFactory factory2(&context2);
+  logAttribute->onSchedule(&context2, &factory2);
+  logAttribute->onTrigger(&context2, &session2);
+
+  provenance::ProvenanceReporter *reporter = session.getProvenanceReporter();
+  std::set<provenance::ProvenanceEventRecord*> records = reporter->getEvents();
+  record = session.get();
+  REQUIRE(record == nullptr);
+  REQUIRE(records.size() == 0);
+
+  std::fstream file;
+  std::stringstream ss;
+  ss << dir << "/" << "tstFile.ext";
+  file.open(ss.str(), std::ios::out);
+  file << "tempFile";
+  file.close();
+
+  processor->incrementActiveTasks();
+  processor->setScheduledState(core::ScheduledState::RUNNING);
+  processor->onTrigger(&context, &session);
+  unlink(ss.str().c_str());
+  reporter = session.getProvenanceReporter();
+
+  records = reporter->getEvents();
+  session.commit();
+  oss.str("");
+  oss.clear();
+
+  logAttribute->incrementActiveTasks();
+  logAttribute->setScheduledState(core::ScheduledState::RUNNING);
+  logAttribute->onTrigger(&context2, &session2);
+
+  //session2.commit();
+  records = reporter->getEvents();
+
+  std::string log_attribute_output = oss.str();
+  REQUIRE(
+      log_attribute_output.find("key:absolute.path value:" + ss.str())
+          != std::string::npos);
+  REQUIRE(log_attribute_output.find("Size:8 Offset:0") != std::string::npos);
+  REQUIRE(
+      log_attribute_output.find("key:path value:" + std::string(dir))
+          != std::string::npos);
+
+  outputLogger = std::unique_ptr<logging::BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::NullAppender());
+  logger->updateLogger(std::move(outputLogger));
+
+}
+
+int fileSize(const char *add) {
+  std::ifstream mySource;
+  mySource.open(add, std::ios_base::binary);
+  mySource.seekg(0, std::ios_base::end);
+  int size = mySource.tellg();
+  mySource.close();
+  return size;
+}
+

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/integration/SocketTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/integration/SocketTests.cpp b/libminifi/test/integration/SocketTests.cpp
new file mode 100644
index 0000000..157e685
--- /dev/null
+++ b/libminifi/test/integration/SocketTests.cpp
@@ -0,0 +1,185 @@
+/**
+ *
+ * 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 "../TestBase.h"
+#include "io/ClientSocket.h"
+
+using namespace org::apache::nifi::minifi::io;
+TEST_CASE("TestSocket", "[TestSocket1]") {
+
+  Socket socket(std::make_shared<SocketContext>(std::make_shared<minifi::Configure>()), "localhost", 8183);
+  REQUIRE(-1 == socket.initialize());
+  REQUIRE("localhost" == socket.getHostname());
+  socket.closeStream();
+
+}
+
+TEST_CASE("TestSocketWriteTest1", "[TestSocket2]") {
+
+  Socket socket(std::make_shared<SocketContext>(std::make_shared<minifi::Configure>()), "localhost", 8183);
+  REQUIRE(-1 == socket.initialize());
+
+  socket.writeData(0, 0);
+
+  std::vector<uint8_t> buffer;
+  buffer.push_back('a');
+
+  REQUIRE(-1 == socket.writeData(buffer, 1));
+
+  socket.closeStream();
+
+}
+
+TEST_CASE("TestSocketWriteTest2", "[TestSocket3]") {
+
+  std::vector<uint8_t> buffer;
+  buffer.push_back('a');
+  
+  std::shared_ptr<SocketContext> socket_context = std::make_shared<SocketContext>(std::make_shared<minifi::Configure>());
+
+  Socket server(socket_context, "localhost", 9183, 1);
+
+  REQUIRE(-1 != server.initialize());
+
+  Socket client(socket_context, "localhost", 9183);
+
+  REQUIRE(-1 != client.initialize());
+
+  REQUIRE(1 == client.writeData(buffer, 1));
+
+  std::vector<uint8_t> readBuffer;
+  readBuffer.resize(1);
+
+  REQUIRE(1 == server.readData(readBuffer, 1));
+
+  REQUIRE(readBuffer == buffer);
+
+  server.closeStream();
+
+  client.closeStream();
+
+}
+
+TEST_CASE("TestGetHostName", "[TestSocket4]") {
+
+  REQUIRE(Socket::getMyHostName().length() > 0);
+
+}
+
+TEST_CASE("TestWriteEndian64", "[TestSocket4]") {
+
+  std::vector<uint8_t> buffer;
+  buffer.push_back('a');
+  
+  std::shared_ptr<SocketContext> socket_context = std::make_shared<SocketContext>(std::make_shared<minifi::Configure>());
+
+  Socket server(socket_context, "localhost", 9183, 1);
+
+  REQUIRE(-1 != server.initialize());
+
+  Socket client(socket_context, "localhost", 9183);
+
+  REQUIRE(-1 != client.initialize());
+
+  uint64_t negative_one = -1;
+  REQUIRE(8 == client.write(negative_one));
+
+  uint64_t negative_two = 0;
+  REQUIRE(8 == server.read(negative_two));
+
+  REQUIRE(negative_two == negative_one);
+
+  server.closeStream();
+
+  client.closeStream();
+
+}
+
+TEST_CASE("TestWriteEndian32", "[TestSocket5]") {
+
+  std::vector<uint8_t> buffer;
+  buffer.push_back('a');
+
+  std::shared_ptr<SocketContext> socket_context = std::make_shared<SocketContext>(std::make_shared<minifi::Configure>());
+  
+  Socket server(socket_context, "localhost", 9183, 1);
+
+  REQUIRE(-1 != server.initialize());
+
+  Socket client(socket_context, "localhost", 9183);
+
+  REQUIRE(-1 != client.initialize());
+
+  {
+    uint32_t negative_one = -1;
+    REQUIRE(4 == client.write(negative_one));
+
+    uint32_t negative_two = 0;
+    REQUIRE(4 == server.read(negative_two));
+
+    REQUIRE(negative_two == negative_one);
+  }
+
+  {
+    uint16_t negative_one = -1;
+    REQUIRE(2 == client.write(negative_one));
+
+    uint16_t negative_two = 0;
+    REQUIRE(2 == server.read(negative_two));
+
+    REQUIRE(negative_two == negative_one);
+  }
+  server.closeStream();
+
+  client.closeStream();
+
+}
+
+TEST_CASE("TestSocketWriteTestAfterClose", "[TestSocket6]") {
+
+  std::vector<uint8_t> buffer;
+  buffer.push_back('a');
+
+  std::shared_ptr<SocketContext> socket_context = std::make_shared<SocketContext>(std::make_shared<minifi::Configure>());
+  
+  Socket server(socket_context, "localhost", 9183, 1);
+
+  REQUIRE(-1 != server.initialize());
+
+  Socket client(socket_context, "localhost", 9183);
+
+  REQUIRE(-1 != client.initialize());
+
+  REQUIRE(1 == client.writeData(buffer, 1));
+
+  std::vector<uint8_t> readBuffer;
+  readBuffer.resize(1);
+
+  REQUIRE(1 == server.readData(readBuffer, 1));
+
+  REQUIRE(readBuffer == buffer);
+
+  client.closeStream();
+
+  REQUIRE(-1 == client.writeData(buffer, 1));
+
+  server.closeStream();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/integration/TestExecuteProcess.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/integration/TestExecuteProcess.cpp b/libminifi/test/integration/TestExecuteProcess.cpp
new file mode 100644
index 0000000..a7c6da6
--- /dev/null
+++ b/libminifi/test/integration/TestExecuteProcess.cpp
@@ -0,0 +1,137 @@
+/**
+ *
+ * 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.
+ */
+
+#include <uuid/uuid.h>
+#include <fstream>
+
+#include "../unit/ProvenanceTestHelper.h"
+#include "FlowController.h"
+#include "core/logging/LogAppenders.h"
+#include "core/logging/BaseLogger.h"
+#include "processors/GetFile.h"
+#include "core/Core.h"
+#include "core/FlowFile.h"
+#include "core/Processor.h"
+#include "core/controller/ControllerServiceNode.h"
+#include "core/controller/ControllerServiceProvider.h"
+#include "processors/ExecuteProcess.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/ProcessorNode.h"
+
+int main(int argc, char  **argv)
+{
+
+  std::ostringstream oss;
+  std::unique_ptr<logging::BaseLogger> outputLogger = std::unique_ptr<
+      logging::BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,
+                                                                         0));
+  std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
+  logger->updateLogger(std::move(outputLogger));
+
+
+  outputLogger = std::unique_ptr<logging::BaseLogger>(
+        new org::apache::nifi::minifi::core::logging::NullAppender());
+    logger->updateLogger(std::move(outputLogger));
+
+  std::shared_ptr<core::Processor> processor = std::make_shared<
+      org::apache::nifi::minifi::processors::ExecuteProcess>("executeProcess");
+  processor->setMaxConcurrentTasks(1);
+
+  std::shared_ptr<core::Repository> test_repo =
+      std::make_shared<TestRepository>();
+
+  std::shared_ptr<TestRepository> repo =
+      std::static_pointer_cast<TestRepository>(test_repo);
+  std::shared_ptr<minifi::FlowController> controller = std::make_shared<
+      TestFlowController>(test_repo, test_repo);
+
+  uuid_t processoruuid;
+  assert(true == processor->getUUID(processoruuid));
+
+  std::shared_ptr<minifi::Connection> connection = std::make_shared<
+      minifi::Connection>(test_repo, "executeProcessConnection");
+  connection->setRelationship(core::Relationship("success", "description"));
+
+  // link the connections so that we can test results at the end for this
+  connection->setSource(processor);
+  connection->setDestination(processor);
+
+  connection->setSourceUUID(processoruuid);
+  connection->setDestinationUUID(processoruuid);
+
+  processor->addConnection(connection);
+  assert(processor->getName() == "executeProcess");
+
+  std::shared_ptr<core::FlowFile> record;
+  processor->setScheduledState(core::ScheduledState::RUNNING);
+
+  processor->initialize();
+
+  std::atomic<bool> is_ready(false);
+
+  std::vector<std::thread> processor_workers;
+
+  core::ProcessorNode node2(processor);
+  std::shared_ptr<core::controller::ControllerServiceProvider> controller_services_provider = nullptr;
+  std::shared_ptr<core::ProcessContext> contextset = std::make_shared<
+      core::ProcessContext>(node2,controller_services_provider, test_repo);
+  core::ProcessSessionFactory factory(contextset.get());
+  processor->onSchedule(contextset.get(), &factory);
+
+  for (int i = 0; i < 1; i++) {
+    //
+    processor_workers.push_back(
+        std::thread(
+            [processor,test_repo,&is_ready]()
+            {
+              core::ProcessorNode node(processor);
+              std::shared_ptr<core::controller::ControllerServiceProvider> controller_services_provider = nullptr;
+              std::shared_ptr<core::ProcessContext> context = std::make_shared<core::ProcessContext>(node,controller_services_provider, test_repo);
+              context->setProperty(org::apache::nifi::minifi::processors::ExecuteProcess::Command,"sleep 0.5");
+              //context->setProperty(org::apache::nifi::minifi::processors::ExecuteProcess::CommandArguments," 1 >>" + ss.str());
+              std::shared_ptr<core::ProcessSession> session = std::make_shared<core::ProcessSession>(context.get());
+              while(!is_ready.load(std::memory_order_relaxed)) {
+
+              }
+
+              processor->onTrigger(context.get(), session.get());
+
+            }));
+  }
+
+  is_ready.store(true, std::memory_order_relaxed);
+  //is_ready.store(true);
+
+  std::for_each(processor_workers.begin(), processor_workers.end(),
+                [](std::thread &t)
+                {
+                  t.join();
+                });
+
+    outputLogger = std::unique_ptr<logging::BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::NullAppender());
+  logger->updateLogger(std::move(outputLogger));
+
+
+  std::shared_ptr<org::apache::nifi::minifi::processors::ExecuteProcess> execp =
+      std::static_pointer_cast<
+          org::apache::nifi::minifi::processors::ExecuteProcess>(processor);
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/resources/TestControllerServices.yml
----------------------------------------------------------------------
diff --git a/libminifi/test/resources/TestControllerServices.yml b/libminifi/test/resources/TestControllerServices.yml
new file mode 100644
index 0000000..3923c87
--- /dev/null
+++ b/libminifi/test/resources/TestControllerServices.yml
@@ -0,0 +1,62 @@
+Flow Controller:
+    name: MiNiFi Flow
+    id: 2438e3c8-015a-1000-79ca-83af40ec1990
+    
+Processors: 
+    - name: MockProcessorIto
+      id: 1438e3c8-015a-1000-79ca-83af40ec1991
+      class: org.apache.nifi.processors.standard.MockProcessor
+      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:
+        linkedService: MockItLikeIts1995
+
+Connections:
+    - name: TransferFilesToRPG
+      id: 1438e3c8-015a-1000-79ca-83af40ec1997
+      source name: MockProcessorIto
+      source id: 1438e3c8-015a-1000-79ca-83af40ec1991
+      source relationship name: success
+      destination name: MockProcessorIto
+      destination id: 1438e3c8-015a-1000-79ca-83af40ec1991
+      destination relationship name: success
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+      
+Controller Services:
+    - name: MockItLikeIts1995
+      id: 2438e3c8-015a-1000-79ca-83af40ec1991
+      class: MockControllerService
+      Properties:
+          Linked Services: 
+            - value: ID
+    - name: SSLClientServiceTest
+      id: 2438e3c8-015a-1000-79ca-83af40ec1994
+      class: SSLContextService
+      Properties:
+        Client Certificate:
+            - value: cn.crt.pem
+        Private Key:
+            - value: cn.ckey.pem
+        Passphrase:
+            - value: cn.pass
+        CA Certificate:
+            - value: nifi-cert.pem
+    - name: ID
+      id: 2438e3c8-015a-1000-79ca-83af40ec1992
+      class: MockControllerService
+      Properties:
+          Linked Services: 
+    - name: MockItLikeItsWrong
+      id: 2438e3c8-015a-1000-79ca-83af40ec1993
+      class: MockControllerSer
+      Properties:
+          Linked Services: 
+            - value: ID
+

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/resources/TestHTTPGet.yml
----------------------------------------------------------------------
diff --git a/libminifi/test/resources/TestHTTPGet.yml b/libminifi/test/resources/TestHTTPGet.yml
new file mode 100644
index 0000000..0783b8e
--- /dev/null
+++ b/libminifi/test/resources/TestHTTPGet.yml
@@ -0,0 +1,73 @@
+#
+# 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/c9940e94/libminifi/test/resources/TestHTTPGetSecure.yml
----------------------------------------------------------------------
diff --git a/libminifi/test/resources/TestHTTPGetSecure.yml b/libminifi/test/resources/TestHTTPGetSecure.yml
new file mode 100644
index 0000000..2bb4f23
--- /dev/null
+++ b/libminifi/test/resources/TestHTTPGetSecure.yml
@@ -0,0 +1,88 @@
+#
+# 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:
+          SSL Context Service: SSLContextService
+          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
+
+Controller Services:
+    - name: SSLContextService
+      id: 2438e3c8-015a-1000-79ca-83af40ec1994
+      class: SSLContextService
+      Properties:
+        Client Certificate:
+            - value: cn.crt.pem
+        Private Key:
+            - value: cn.ckey.pem
+        Passphrase:
+            - value: cn.pass
+        CA Certificate:
+            - value: nifi-cert.pem
+
+Remote Processing Groups:
+    
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/resources/TestHTTPPost.yml
----------------------------------------------------------------------
diff --git a/libminifi/test/resources/TestHTTPPost.yml b/libminifi/test/resources/TestHTTPPost.yml
new file mode 100644
index 0000000..837194d
--- /dev/null
+++ b/libminifi/test/resources/TestHTTPPost.yml
@@ -0,0 +1,87 @@
+#
+# 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/c9940e94/libminifi/test/resources/cn.ckey.pem
----------------------------------------------------------------------
diff --git a/libminifi/test/resources/cn.ckey.pem b/libminifi/test/resources/cn.ckey.pem
new file mode 100644
index 0000000..23017fa
--- /dev/null
+++ b/libminifi/test/resources/cn.ckey.pem
@@ -0,0 +1,31 @@
+Bag Attributes
+    friendlyName: nifi-key
+    localKeyID: 73 E6 90 32 31 08 F5 87 C2 CE 8D 17 10 32 05 F2 95 6A 9E 9C 
+Key Attributes: <No Attributes>
+-----BEGIN RSA PRIVATE KEY-----
+MIIEowIBAAKCAQEAn9zCw5/EDWKe/jZWIou8aohC0wDhe8azYAORb45kEEqB2Q1p
+JJDHIIoNgDukXhrcPWFz2FxdZ56so7kkrjkN507t11dJn+wuFwfVot0us35NvOcl
+t3rHNy38nid4gV4+F01mXksrNWCnvTNSmp4XIWlRHxFkiZyKVTRxyo5U3fr2zmtm
+80LtopoBEf5Lr29WYZrjgA+A8iDMKvVlgmpriau44Vie9L3DIot4eAO7eN3o5UG+
+oYE7RQ6BdXSo9+D/tbma0rZvS9WHw3Qin7gDygaGKbwWEFuwhSYO+lcsywsB5hT3
+uxGTRnskafE29/vvzmsygMlNoRXk5NaRlnf/NQIDAQABAoIBAD0Z6tNMiGm5TR3K
+9OclU7HAPRJbHtPpYXwMQRVLblGPuMuWRZ61O3bjbT+bDDrvt0g6rEWBIC5PxpqZ
+VDzHD9peH6gT888NPZyN4Qjr00LFvr659n+HjQc5K3njijqgI8lSSxM1VeGD/Pji
+2erBDAQ4NcpTIoM6cuEYN3n9JtDk1B/nJ4vWD+1Ak/sS91hmaIy/1hOUofgaqp4W
+8+pnur/lEn4d9yCfEbDOfkfep49si8kDw8SdBH6uFpr1s71E2q9PfGv5WN4LGxc+
+fX3zHHnxFRB9XRivo42bIxCRyf/U/GNeD0ldlXzmfUkVKw/V1CLioRKP+KaJ7aT6
+FYZE1GECgYEA6Wk4t7kkNGszWiFJpGlUmB50CmmH0zdb+qeiBhxuAAgcPveglXJX
+qNAJOxDY29nSQqhtuRe/vhQEk88axNVfTIHBFF+29P8ODQPkQsOn5pyCeii+MTmJ
+R1Kq8/5F08CPCJ2n/Rfd1xYKsiB6QJDeRCxbUHuqlOwwkuASNnHVPI0CgYEAr1Vd
+SXToxlNOesCuvfK+IV+7qanAag49MSoHKYuzii4YhaS/9yC8HGNUNU4ZOr3gyOCo
+rlPGsJx+7HGAIylg7M+w0xrAOlwcBIfD6DHSoFSCbpxKQGOdav5PpKvKmXDbnCeq
+hvlJRSRepC9raZePRqHMeO+y0SCMCW/3kCvBZ0kCgYBPVEXEm5YkgrLHVTz0NiBc
+Oy1xRIC3Vr9aJ5M3xH3WYOtIs1zOrjgA1gPtIUKzJgsCQMBaAipW7fLIUcFlWKe1
+FvyKciV6ix05azMvRdRQ2qy13BcYH9f6d6xGlQsidvWmFf88U9o/+evUspkaTKJe
+RT0b04wcZVH0+Vb3TiUNGQKBgQCBmpxGfEDEuTe9zUhd6F1a1iUmGPC+dKAPA34f
+28BvV/uQbmPoDS5/3XyVmpFTtp++XTLTCeJJEqVPqeaKF0AqTKvsoE2P1h5vytup
+e4qyMUXo9WZ9iiUGvc2pb8NnAOGU4E1RUFw029mJi/PeLcYe8+LtUmHf7CsXnshk
+rbNc4QKBgFAIoRZonaHDU4AYXx/6Xem2EneJ3tXyxMZ8WEjNQKaWmA5rkGbbSijk
+0EaJ3bMexlyp1qgv86f4LrfLbZLdqTDSzhqlTV0x0b6D7MRBLuzcQUqsKmDmxkxq
+CkJUqnNhVWM0SMG8cy9K1WMBecdot8I2OH8L00uwKkkM4pQ1En+o
+-----END RSA PRIVATE KEY-----

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/resources/cn.crt.pem
----------------------------------------------------------------------
diff --git a/libminifi/test/resources/cn.crt.pem b/libminifi/test/resources/cn.crt.pem
new file mode 100644
index 0000000..3a786db
--- /dev/null
+++ b/libminifi/test/resources/cn.crt.pem
@@ -0,0 +1,25 @@
+Bag Attributes
+    friendlyName: nifi-key
+    localKeyID: 73 E6 90 32 31 08 F5 87 C2 CE 8D 17 10 32 05 F2 95 6A 9E 9C 
+subject=/OU=NIFI/CN=test
+issuer=/OU=NIFI/CN=localhost
+-----BEGIN CERTIFICATE-----
+MIIDQTCCAimgAwIBAgIKAVpnU2lhAAAAADANBgkqhkiG9w0BAQsFADAjMQ0wCwYD
+VQQLDAROSUZJMRIwEAYDVQQDDAlsb2NhbGhvc3QwHhcNMTcwMjIyMTkzNjQ0WhcN
+MjAwMjIyMTkzNjQ0WjAeMQ0wCwYDVQQLDAROSUZJMQ0wCwYDVQQDDAR0ZXN0MIIB
+IjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAn9zCw5/EDWKe/jZWIou8aohC
+0wDhe8azYAORb45kEEqB2Q1pJJDHIIoNgDukXhrcPWFz2FxdZ56so7kkrjkN507t
+11dJn+wuFwfVot0us35NvOclt3rHNy38nid4gV4+F01mXksrNWCnvTNSmp4XIWlR
+HxFkiZyKVTRxyo5U3fr2zmtm80LtopoBEf5Lr29WYZrjgA+A8iDMKvVlgmpriau4
+4Vie9L3DIot4eAO7eN3o5UG+oYE7RQ6BdXSo9+D/tbma0rZvS9WHw3Qin7gDygaG
+KbwWEFuwhSYO+lcsywsB5hT3uxGTRnskafE29/vvzmsygMlNoRXk5NaRlnf/NQID
+AQABo3wwejAdBgNVHQ4EFgQUc+aQMjEI9YfCzo0XEDIF8pVqnpwwHwYDVR0jBBgw
+FoAUtz2ps5YYKDRnj1ixtUA8NOjmo2IwDgYDVR0PAQH/BAQDAgP4MAkGA1UdEwQC
+MAAwHQYDVR0lBBYwFAYIKwYBBQUHAwIGCCsGAQUFBwMBMA0GCSqGSIb3DQEBCwUA
+A4IBAQBbpGba97qG7eXIQm3hiMKsuGpMNoybKxUlazOEeqG5QemMxSDOy2TDMBgM
+2oQU3RImrNMLbfWCxnQ1biWclis9exR5IkUintb8hEE8Vbq/nTLCi7IvpEg+2oDc
+kvSs8UTrI17EQAzh/vc9ljbgtvJD0L8khO8oPPSEpweD+vk/ZragQJOzeA7ohYMM
+kJDJ/a9HNNe6nPx+NOCWQH2hJS8Ok2z57hBgy1oym/xJoT9z3vfjQcHtNK4qLNlx
+P+o2mWNC2ZYNe84ghUfMD+btlnA3MrPEEOJHXOoE9UgaeNZXjwuF6N1qI2RCWSVW
+IU4uwXjW1icAEYZbCK9f2l5cK+Vu
+-----END CERTIFICATE-----

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/resources/cn.pass
----------------------------------------------------------------------
diff --git a/libminifi/test/resources/cn.pass b/libminifi/test/resources/cn.pass
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/resources/nifi-cert.pem
----------------------------------------------------------------------
diff --git a/libminifi/test/resources/nifi-cert.pem b/libminifi/test/resources/nifi-cert.pem
new file mode 100644
index 0000000..4e404cd
--- /dev/null
+++ b/libminifi/test/resources/nifi-cert.pem
@@ -0,0 +1,20 @@
+-----BEGIN CERTIFICATE-----
+MIIDSTCCAjGgAwIBAgIKAVpnU2gBAAAAADANBgkqhkiG9w0BAQsFADAjMQ0wCwYD
+VQQLDAROSUZJMRIwEAYDVQQDDAlsb2NhbGhvc3QwHhcNMTcwMjIyMTkzNjQzWhcN
+MjAwMjIyMTkzNjQzWjAjMQ0wCwYDVQQLDAROSUZJMRIwEAYDVQQDDAlsb2NhbGhv
+c3QwggEiMA0GCSqGSIb3DQEBAQUAA4IBDwAwggEKAoIBAQDAIXpNyG57u1HroE+D
+zzpY5g0+BLA1qrUolBK4ZwQcrbCUAS5h+JjP0tuMUtlEtFjYIX2YgZvrzdvKue84
+xyrpr5h1cNo4dmeQoFKwj/FPNJxdO/cQpK55nSwy4PsvTL/YKzaV/yF4FQRSqY17
+Xkn3K8RDUaknSPuIIKd4piON7MVaCVFK//2y6+pULEjGviP2Bc4ODJTfB3AjGRBQ
+EleTga0OnTdHFzHoKHRUHCNwaRgeFywYdHODp1yvv3m0t85A0D9WWN7T5G0f8pT9
+ZeDOX2cq3EPtdTBnKGgSXM1x6E3RaYoOACYBKdy/Lv5hjMpTr0fXhD7af9ql2irH
++RNBAgMBAAGjfzB9MA4GA1UdDwEB/wQEAwIB/jAMBgNVHRMEBTADAQH/MB0GA1Ud
+DgQWBBS3PamzlhgoNGePWLG1QDw06OajYjAfBgNVHSMEGDAWgBS3PamzlhgoNGeP
+WLG1QDw06OajYjAdBgNVHSUEFjAUBggrBgEFBQcDAgYIKwYBBQUHAwEwDQYJKoZI
+hvcNAQELBQADggEBAKcXKG4rPnEuYS/AJfHs5RlGsg2h/VnbCT50A4OsUNcv5KSe
+WFeP2ctp793MsIsUKA3FgGi+23aaUTtlAclQQs/xlBiDq6XmkED/DjLVzt5dTHrl
+gS6aFtDHffMn5tLaUKOkNETOG/Fnc+XPAENB8GyqBd2Q/1L0SWSHXSTecX+aA9P6
+lvrRtWOqyGHiRoaRE5+VUjyO+0ToEgj9E+3rV8JL66BT7SWQusLGqbX1OoANCMTj
+BRYeqB0g0PrXU+6chh6StpNSnYzkQdoxLUIDYYZx2XGsbkjDh/k6ni6bgJEKEOCu
+T3Z2tyvGpc+PjLRXW/WyXCpg/xfr3+GSVKI6ark=
+-----END CERTIFICATE-----

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/unit/ClassLoaderTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/ClassLoaderTests.cpp b/libminifi/test/unit/ClassLoaderTests.cpp
new file mode 100644
index 0000000..f5cb6e9
--- /dev/null
+++ b/libminifi/test/unit/ClassLoaderTests.cpp
@@ -0,0 +1,35 @@
+/**
+ *
+ * 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.
+ */
+
+#include "../../include/core/ClassLoader.h"
+#include "../TestBase.h"
+#include "io/ClientSocket.h"
+#include "core/Processor.h"
+#include "core/ClassLoader.h"
+#include "processors/AppendHostInfo.h"
+#include "core/logging/LogAppenders.h"
+
+using namespace org::apache::nifi::minifi::io;
+TEST_CASE("TestLoader", "[TestLoader]") {
+  
+REQUIRE ( nullptr != core::ClassLoader::getDefaultClassLoader().instantiate("AppendHostInfo","hosty"));
+REQUIRE ( nullptr != core::ClassLoader::getDefaultClassLoader().instantiate("ListenHTTP","hosty2"));
+REQUIRE ( nullptr == core::ClassLoader::getDefaultClassLoader().instantiate("Don'tExist","hosty3"));
+REQUIRE ( nullptr == core::ClassLoader::getDefaultClassLoader().instantiate("","EmptyEmpty"));
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/c9940e94/libminifi/test/unit/ControllerServiceTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/ControllerServiceTests.cpp b/libminifi/test/unit/ControllerServiceTests.cpp
new file mode 100644
index 0000000..e8fb4ce
--- /dev/null
+++ b/libminifi/test/unit/ControllerServiceTests.cpp
@@ -0,0 +1,89 @@
+/**
+ *
+ * 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.
+ */
+#include <uuid/uuid.h>
+#include <fstream>
+#include "FlowController.h"
+#include "../TestBase.h"
+#include "properties/Configure.h"
+#include "core/logging/LogAppenders.h"
+#include "core/logging/BaseLogger.h"
+#include "processors/GetFile.h"
+#include "core/Core.h"
+#include "Exception.h"
+#include "../../include/core/FlowFile.h"
+#include "../unit/MockClasses.h"
+#include "../unit/ProvenanceTestHelper.h"
+#include "core/yaml/YamlConfiguration.h"
+#include "core/Processor.h"
+#include "core/controller/ControllerServiceMap.h"
+#include "core/controller/StandardControllerServiceNode.h"
+namespace ControllerServiceTests {
+
+TEST_CASE("Test ControllerServicesMap", "[cs1]") {
+  core::controller::ControllerServiceMap map;
+  REQUIRE(0 == map.getAllControllerServices().size());
+
+  std::shared_ptr<core::controller::ControllerService> service =
+      std::make_shared<MockControllerService>();
+  std::shared_ptr<core::controller::StandardControllerServiceNode> testNode =
+      std::make_shared<core::controller::StandardControllerServiceNode>(
+          service, "ID", std::make_shared<minifi::Configure>());
+
+  map.put("ID", testNode);
+  REQUIRE(1 == map.getAllControllerServices().size());
+
+  REQUIRE(nullptr != map.getControllerServiceNode("ID"));
+
+  REQUIRE(false== map.put("",testNode));
+  REQUIRE(false== map.put("",nullptr));
+
+  // ensure the pointer is the same
+
+  REQUIRE(
+      service.get()
+          == map.getControllerServiceNode("ID")
+              ->getControllerServiceImplementation().get());
+}
+
+TEST_CASE("Test StandardControllerServiceNode nullPtr", "[cs1]") {
+  core::controller::ControllerServiceMap map;
+
+  try {
+    std::shared_ptr<core::controller::StandardControllerServiceNode> testNode =
+        std::make_shared<core::controller::StandardControllerServiceNode>(
+            nullptr, "ID", std::make_shared<minifi::Configure>());
+  } catch (minifi::Exception &ex) {
+    return;
+  }
+
+  FAIL("Should have encountered exception");
+}
+
+std::shared_ptr<core::controller::StandardControllerServiceNode> newCsNode(
+    const std::string id) {
+  std::shared_ptr<core::controller::ControllerService> service =
+      std::make_shared<MockControllerService>();
+  std::shared_ptr<core::controller::StandardControllerServiceNode> testNode =
+      std::make_shared<core::controller::StandardControllerServiceNode>(
+          service, id, std::make_shared<minifi::Configure>());
+
+  return testNode;
+}
+
+
+}